From d8dccdf1aef355b3d5638bf025299749b5a6467e Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Fri, 20 Oct 2023 11:26:18 -0700 Subject: [PATCH 01/12] work on avro support --- Cargo.lock | 132 ++++++++++++++++++++++++++-- arroyo-api/src/connection_tables.rs | 72 --------------- arroyo-rpc/Cargo.toml | 2 + arroyo-rpc/src/formats.rs | 29 +++++- arroyo-rpc/src/lib.rs | 2 + arroyo-rpc/src/schema_resolver.rs | 109 +++++++++++++++++++++++ arroyo-worker/Cargo.toml | 1 + arroyo-worker/src/formats.rs | 33 +++++-- 8 files changed, 293 insertions(+), 87 deletions(-) create mode 100644 arroyo-rpc/src/schema_resolver.rs diff --git a/Cargo.lock b/Cargo.lock index 89bde32b2..2aedfaa95 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -156,6 +156,29 @@ dependencies = [ "backtrace", ] +[[package]] +name = "apache-avro" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ceb7c683b2f8f40970b70e39ff8be514c95b96fcb9c4af87e1ed2cb2e10801a0" +dependencies = [ + "digest 0.10.7", + "lazy_static", + "libflate 2.0.0", + "log", + "num-bigint", + "quad-rand", + "rand", + "regex-lite", + "serde", + "serde_json", + "strum 0.25.0", + "strum_macros 0.25.2", + "thiserror", + "typed-builder", + "uuid", +] + [[package]] name = "arbitrary" version = "1.3.0" @@ -638,8 +661,10 @@ dependencies = [ "anyhow", "arroyo-types", "bincode 2.0.0-rc.3", + "log", "nanoid", "prost", + "reqwest", "serde", "serde_json", "tokio", @@ -806,6 +831,7 @@ name = "arroyo-worker" version = "0.7.0" dependencies = [ "anyhow", + "apache-avro", "arrow", "arrow-array", "arroyo-macro", @@ -2014,6 +2040,15 @@ version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" +[[package]] +name = "core2" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b49ba7ef1ad6107f8824dbe97de947cbaac53c44e7f9756a1fba0d37c1eec505" +dependencies = [ + "memchr", +] + [[package]] name = "cornucopia" version = "0.9.0" @@ -2406,6 +2441,12 @@ dependencies = [ "syn 2.0.33", ] +[[package]] +name = "dary_heap" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7762d17f1241643615821a8455a0b2c3e803784b058693d990b11f2dce25a0ca" + [[package]] name = "dashmap" version = "5.5.3" @@ -4511,7 +4552,20 @@ checksum = "5ff4ae71b685bbad2f2f391fe74f6b7659a34871c08b210fdc039e43bee07d18" dependencies = [ "adler32", "crc32fast", - "libflate_lz77", + "libflate_lz77 1.2.0", +] + +[[package]] +name = "libflate" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f7d5654ae1795afc7ff76f4365c2c8791b0feb18e8996a96adad8ffd7c3b2bf" +dependencies = [ + "adler32", + "core2", + "crc32fast", + "dary_heap", + "libflate_lz77 2.0.0", ] [[package]] @@ -4523,6 +4577,17 @@ dependencies = [ "rle-decode-fast", ] +[[package]] +name = "libflate_lz77" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be5f52fb8c451576ec6b79d3f4deb327398bc05bbdbd99021a6e77a4c855d524" +dependencies = [ + "core2", + "hashbrown 0.13.2", + "rle-decode-fast", +] + [[package]] name = "libm" version = "0.2.7" @@ -5867,7 +5932,7 @@ checksum = "ac8a53ce01af1087eaeee6ce7c4fbf50ea4040ab1825c0115c4bafa039644ba9" dependencies = [ "json", "libc", - "libflate", + "libflate 1.4.0", "log", "names", "prost", @@ -5889,6 +5954,12 @@ dependencies = [ "thiserror", ] +[[package]] +name = "quad-rand" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "658fa1faf7a4cc5f057c9ee5ef560f717ad9d8dc66d975267f709624d6e1ab88" + [[package]] name = "quanta" version = "0.11.1" @@ -6137,6 +6208,12 @@ dependencies = [ "regex-syntax 0.7.5", ] +[[package]] +name = "regex-lite" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30b661b2f27137bdbc16f00eda72866a92bb28af1753ffbd56744fb6e2e9cd8e" + [[package]] name = "regex-syntax" version = "0.6.29" @@ -6161,9 +6238,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.11.20" +version = "0.11.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e9ad3fe7488d7e34558a2033d45a0c90b72d97b4f80705666fea71472e2e6a1" +checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" dependencies = [ "base64 0.21.4", "bytes", @@ -6189,6 +6266,7 @@ dependencies = [ "serde", "serde_json", "serde_urlencoded", + "system-configuration", "tokio", "tokio-native-tls", "tokio-rustls 0.24.1", @@ -6724,9 +6802,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.106" +version = "1.0.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cc66a619ed80bf7a0f6b17dd063a84b88f6dea1813737cf469aef1d081142c2" +checksum = "6b420ce6e3d8bd882e9b243c6eed35dbc9a6110c9769e74b584e0d68d1f20c65" dependencies = [ "indexmap 2.0.0", "itoa", @@ -7305,6 +7383,27 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" +[[package]] +name = "system-configuration" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba3a3adc5c275d719af8cb4272ea1c4a6d668a777f37e115f6d11ddbc1c8e0e7" +dependencies = [ + "bitflags 1.3.2", + "core-foundation", + "system-configuration-sys", +] + +[[package]] +name = "system-configuration-sys" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75fb188eb626b924683e3b95e3a48e63551fcfb51949de2f06a9d91dbee93c9" +dependencies = [ + "core-foundation-sys", + "libc", +] + [[package]] name = "target-lexicon" version = "0.12.11" @@ -7983,6 +8082,26 @@ dependencies = [ "static_assertions", ] +[[package]] +name = "typed-builder" +version = "0.16.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34085c17941e36627a879208083e25d357243812c30e7d7387c3b954f30ade16" +dependencies = [ + "typed-builder-macro", +] + +[[package]] +name = "typed-builder-macro" +version = "0.16.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.33", +] + [[package]] name = "typenum" version = "1.16.0" @@ -8185,6 +8304,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "88ad59a7560b41a70d191093a945f0b87bc1deeda46fb237479708a1d6b6cdfc" dependencies = [ "getrandom", + "serde", ] [[package]] diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index 2950290a6..b524b72c2 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -498,79 +498,7 @@ pub(crate) async fn get_confluent_schema( query_params: Query, ) -> Result, ErrorResp> { // TODO: ensure only external URLs can be hit - let url = format!( - "{}/subjects/{}-value/versions/latest", - query_params.endpoint, query_params.topic - ); - let resp = reqwest::get(url).await.map_err(|e| { - warn!("Got error response from schema registry: {:?}", e); - match e.status() { - Some(StatusCode::NOT_FOUND) => bad_request(format!( - "Could not find value schema for topic '{}'", - query_params.topic - )), - - Some(code) => bad_request(format!("Schema registry returned error: {}", code)), - None => { - warn!( - "Unknown error connecting to schema registry {}: {:?}", - query_params.endpoint, e - ); - bad_request(format!( - "Could not connect to Schema Registry at {}: unknown error", - query_params.endpoint - )) - } - } - })?; - - if !resp.status().is_success() { - let message = format!( - "Received an error status code from the provided endpoint: {} {}", - resp.status().as_u16(), - resp.bytes() - .await - .map(|bs| String::from_utf8_lossy(&bs).to_string()) - .unwrap_or_else(|_| "".to_string()) - ); - return Err(bad_request(message)); - } - let value: serde_json::Value = resp.json().await.map_err(|e| { - warn!("Invalid json from schema registry: {:?}", e); - bad_request(format!( - "Schema registry returned invalid JSON: {}", - e.to_string() - )) - })?; - - let schema_type = value - .get("schemaType") - .ok_or_else(|| { - bad_request( - "The JSON returned from this endpoint was unexpected. Please confirm that the URL is correct." - .to_string(), - ) - })? - .as_str(); - - if schema_type != Some("JSON") { - return Err(bad_request( - "Only JSON schema types are supported currently".to_string(), - )); - } - - let schema = value - .get("schema") - .ok_or_else(|| { - return bad_request("Missing 'schema' field in schema registry response".to_string()); - })? - .as_str() - .ok_or_else(|| { - return bad_request( - "The 'schema' field in the schema registry response is not a string".to_string(), - ); - })?; if let Err(e) = convert_json_schema(&query_params.topic, schema) { warn!( diff --git a/arroyo-rpc/Cargo.toml b/arroyo-rpc/Cargo.toml index 4ffb1667d..d7b7df2f9 100644 --- a/arroyo-rpc/Cargo.toml +++ b/arroyo-rpc/Cargo.toml @@ -17,6 +17,8 @@ serde_json = "1.0" nanoid = "0.4" utoipa = "3" anyhow = "1.0.75" +reqwest = "0.11.22" +log = "0.4.20" [build-dependencies] tonic-build = { workspace = true } diff --git a/arroyo-rpc/src/formats.rs b/arroyo-rpc/src/formats.rs index 7114ac545..3ea169114 100644 --- a/arroyo-rpc/src/formats.rs +++ b/arroyo-rpc/src/formats.rs @@ -1,7 +1,10 @@ use serde::{Deserialize, Serialize}; use std::collections::HashMap; +use std::hash::{Hash, Hasher}; use std::str::FromStr; +use serde_json::Value; use utoipa::ToSchema; +use arroyo_types::UserError; #[derive( Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Default, Hash, PartialOrd, ToSchema, @@ -91,9 +94,31 @@ impl JsonFormat { #[serde(rename_all = "camelCase")] pub struct RawStringFormat {} +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash, PartialOrd, ToSchema)] +pub struct ConfluentSchemaRegistryConfig { + endpoint: String, +} + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash, PartialOrd, ToSchema)] #[serde(rename_all = "camelCase")] -pub struct AvroFormat {} +pub struct AvroFormat { + pub writer_schema: Option, + pub confluent_schema_registry: Option, +} + +impl AvroFormat { + pub fn from_opts(opts: &mut HashMap) -> Result { + let confluent_schema_registry = opts.remove("avro.confluent_schema_registry.endpoint") + .map(|endpoint| ConfluentSchemaRegistryConfig { + endpoint + }); + + Ok(Self { + writer_schema: None, + confluent_schema_registry, + }) + } +} #[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash, PartialOrd, ToSchema)] #[serde(rename_all = "camelCase")] @@ -118,7 +143,7 @@ impl Format { "json" => Format::Json(JsonFormat::from_opts(false, opts)?), "debezium_json" => Format::Json(JsonFormat::from_opts(true, opts)?), "protobuf" => return Err("protobuf is not yet supported".to_string()), - "avro" => return Err("avro is not yet supported".to_string()), + "avro" => Format::Avro(AvroFormat::from_opts(opts)?), "raw_string" => Format::RawString(RawStringFormat {}), "parquet" => Format::Parquet(ParquetFormat {}), f => return Err(format!("Unknown format '{}'", f)), diff --git a/arroyo-rpc/src/lib.rs b/arroyo-rpc/src/lib.rs index ca4431030..821e61928 100644 --- a/arroyo-rpc/src/lib.rs +++ b/arroyo-rpc/src/lib.rs @@ -1,7 +1,9 @@ pub mod api_types; pub mod formats; +pub mod schema_resolver; pub mod public_ids; + use std::{fs, time::SystemTime}; use crate::api_types::connections::PrimitiveType; diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs new file mode 100644 index 000000000..4ebac9549 --- /dev/null +++ b/arroyo-rpc/src/schema_resolver.rs @@ -0,0 +1,109 @@ +use std::time::Duration; +use anyhow::{anyhow, bail}; +use arroyo_types::UserError; +use reqwest::{Client, StatusCode, Url}; +use serde_json::Value; +use log::warn; +use serde::{Deserialize, Serialize}; + +pub trait SchemaResolver { + fn resolve_schema(&self, id: [u8; 4]) -> Result, UserError>; +} + +pub struct FailingSchemaResolver { +} + +impl SchemaResolver for FailingSchemaResolver { + fn resolve_schema(&self, id: [u8; 4]) -> Result, UserError> { + Err(UserError { + name: "Could not deserialize".to_string(), + details: format!("Schema with id {:?} not available, and no schema registry configured", id), + }) + } +} + +#[derive(Clone, Debug, Serialize, Deserialize)] +#[serde(rename_all = "UPPERCASE")] +pub enum ConfluentSchemaType { + Avro, + Json, + Protobuf +} + +#[derive(Clone, Debug, Serialize, Deserialize)] +#[serde(rename_all = "camelCase")] +pub struct ConfluentSchemaResponse { + pub name: String, + pub version: String, + pub schema_type: ConfluentSchemaType, + pub schema: String, +} + +struct ConfluentSchemaResolver { + endpoint: Url, + topic: String, + client: Client, +} + +impl ConfluentSchemaResolver { + pub fn new(endpoint: &str, topic: &str) -> anyhow::Result { + let client = Client::builder() + .timeout(Duration::from_secs(5)) + .build().unwrap(); + + + let endpoint: Url = + format!("{}/subjects/{}-value/versions/", endpoint, topic).as_str().try_into() + .map_err(|e| anyhow!("{} is not a valid url", endpoint))?; + + Ok(Self { + client, + topic: topic.to_string(), + endpoint, + }) + } + + + async fn get_schema(&self, version: Option) -> anyhow::Result { + let url = self.endpoint.join( + &version.map(|v| format!("{}", v)).unwrap_or_else(|| "latest".to_string())).unwrap(); + + let resp = reqwest::get(url).await.map_err(|e| { + warn!("Got error response from schema registry: {:?}", e); + match e.status() { + Some(StatusCode::NOT_FOUND) => anyhow!( + "Could not find value schema for topic '{}'", + self.topic), + + Some(code) => anyhow!("Schema registry returned error: {}", code), + None => { + warn!("Unknown error connecting to schema registry {}: {:?}", + self.endpoint, e + ); + anyhow!( + "Could not connect to Schema Registry at {}: unknown error", + self.endpoint + ) + } + } + })?; + + if !resp.status().is_success() { + bail!( + "Received an error status code from the provided endpoint: {} {}", + resp.status().as_u16(), + resp.bytes() + .await + .map(|bs| String::from_utf8_lossy(&bs).to_string()) + .unwrap_or_else(|_| "".to_string()) + ); + } + + resp.json().await.map_err(|e| { + warn!("Invalid json from schema registry: {:?}", e); + anyhow!( + "Schema registry response could not be deserialied: {}", e + ) + }) + } +} \ No newline at end of file diff --git a/arroyo-worker/Cargo.toml b/arroyo-worker/Cargo.toml index f5f1adcb0..b469cc3ec 100644 --- a/arroyo-worker/Cargo.toml +++ b/arroyo-worker/Cargo.toml @@ -71,6 +71,7 @@ fluvio-future = "0.6.0" object_store = {workspace = true } reqwest = "0.11.20" memchr = "2.6.3" +apache-avro = "0.16.0" [dev-dependencies] test-case = "3" diff --git a/arroyo-worker/src/formats.rs b/arroyo-worker/src/formats.rs index 95ffdef7f..443d241e8 100644 --- a/arroyo-worker/src/formats.rs +++ b/arroyo-worker/src/formats.rs @@ -1,11 +1,13 @@ use std::sync::Arc; use std::{collections::HashMap, marker::PhantomData}; +use apache_avro::Schema; use arrow::datatypes::{Field, Fields}; -use arroyo_rpc::formats::{Format, Framing, FramingMethod, JsonFormat}; +use arroyo_rpc::formats::{AvroFormat, Format, Framing, FramingMethod, JsonFormat}; use arroyo_types::UserError; use serde::de::DeserializeOwned; use serde_json::{json, Value}; +use tokio::sync::Mutex; use crate::SchemaData; @@ -54,6 +56,14 @@ fn deserialize_raw_string(msg: &[u8]) -> Result Ok(serde_json::from_value(json).unwrap()) } +fn deserialize_slice_avro( + format: &AvroFormat, + schema_registry: Arc>>, + msg: &[u8], +) { + +} + pub struct FramingIterator<'a> { framing: Option>, buf: &'a [u8], @@ -105,33 +115,42 @@ impl<'a> Iterator for FramingIterator<'a> { } } +#[derive(Clone)] pub struct DataDeserializer { format: Arc, framing: Option>, + schema_registry: Arc>>, + schema_resolver: Arc>, _t: PhantomData, } + impl DataDeserializer { - pub fn new(format: Format, framing: Option) -> Self { + pub fn new(format: Format, framing: Option,) -> Self { + if let Format::Avro(avro) = &format { + + }; + Self { format: Arc::new(format), framing: framing.map(|f| Arc::new(f)), _t: PhantomData, } } + pub fn deserialize_slice<'a>( &self, msg: &'a [u8], ) -> impl Iterator> + 'a { - let format = self.format.clone(); + let new_self = self.clone(); FramingIterator::new(self.framing.clone(), msg) - .map(move |t| Self::deserialize_single(format.clone(), t)) + .map(move |t| new_self.deserialize_single(t)) } - fn deserialize_single(format: Arc, msg: &[u8]) -> Result { - match &*format { + fn deserialize_single(&self, msg: &[u8]) -> Result { + match &*self.format { Format::Json(json) => deserialize_slice_json(json, msg), - Format::Avro(_) => todo!(), + Format::Avro(avro) => deserialie_slice_avro(), Format::Parquet(_) => todo!(), Format::RawString(_) => deserialize_raw_string(msg), } From d4a40bd69702642eacc1745661ea5d7ecf42690b Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Fri, 20 Oct 2023 14:22:56 -0700 Subject: [PATCH 02/12] work --- arroyo-api/src/connection_tables.rs | 39 ++----------------------- arroyo-api/src/lib.rs | 4 +-- arroyo-api/src/rest.rs | 6 +--- arroyo-connectors/src/kafka.rs | 1 + arroyo-rpc/src/schema_resolver.rs | 2 +- arroyo-sql/src/lib.rs | 1 + connector-schemas/kafka/connection.json | 14 +++++++++ 7 files changed, 21 insertions(+), 46 deletions(-) diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index b524b72c2..3c83fd5b4 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -18,7 +18,7 @@ use tracing::warn; use arroyo_connectors::{connector_for_type, ErasedConnector}; use arroyo_rpc::api_types::connections::{ - ConfluentSchema, ConfluentSchemaQueryParams, ConnectionProfile, ConnectionSchema, + ConnectionProfile, ConnectionSchema, ConnectionTable, ConnectionTablePost, SchemaDefinition, }; use arroyo_rpc::api_types::{ConnectionTableCollection, PaginationQueryParams}; @@ -479,39 +479,4 @@ pub(crate) async fn test_schema( Ok(()) } } -} - -/// Get a Confluent Schema -#[utoipa::path( - get, - path = "/v1/connection_tables/schemas/confluent", - tag = "connection_tables", - params( - ("topic" = String, Query, description = "Confluent topic name"), - ("endpoint" = String, Query, description = "Confluent schema registry endpoint"), - ), - responses( - (status = 200, description = "Got Confluent Schema", body = ConfluentSchema), - ), -)] -pub(crate) async fn get_confluent_schema( - query_params: Query, -) -> Result, ErrorResp> { - // TODO: ensure only external URLs can be hit - - - if let Err(e) = convert_json_schema(&query_params.topic, schema) { - warn!( - "Schema from schema registry is not valid: '{}': {}", - schema, e - ); - return Err(bad_request(format!( - "Schema from schema registry is not valid: {}", - e - ))); - } - - Ok(Json(ConfluentSchema { - schema: schema.to_string(), - })) -} +} \ No newline at end of file diff --git a/arroyo-api/src/lib.rs b/arroyo-api/src/lib.rs index 1c3f2702e..c54ceb395 100644 --- a/arroyo-api/src/lib.rs +++ b/arroyo-api/src/lib.rs @@ -8,7 +8,7 @@ use crate::connection_profiles::{ __path_create_connection_profile, __path_get_connection_profiles, }; use crate::connection_tables::{ - __path_create_connection_table, __path_delete_connection_table, __path_get_confluent_schema, + __path_create_connection_table, __path_delete_connection_table, __path_get_connection_tables, __path_test_connection_table, __path_test_schema, }; use crate::connectors::__path_get_connectors; @@ -148,7 +148,6 @@ pub(crate) fn to_micros(dt: OffsetDateTime) -> u64 { delete_connection_table, test_connection_table, test_schema, - get_confluent_schema, get_checkpoint_details, ), components(schemas( @@ -193,7 +192,6 @@ pub(crate) fn to_micros(dt: OffsetDateTime) -> u64 { PrimitiveType, SchemaDefinition, TestSourceMessage, - ConfluentSchema, JsonFormat, AvroFormat, ParquetFormat, diff --git a/arroyo-api/src/rest.rs b/arroyo-api/src/rest.rs index b0403a327..684f82334 100644 --- a/arroyo-api/src/rest.rs +++ b/arroyo-api/src/rest.rs @@ -19,7 +19,7 @@ use utoipa_swagger_ui::SwaggerUi; use crate::connection_profiles::{create_connection_profile, get_connection_profiles}; use crate::connection_tables::{ - create_connection_table, delete_connection_table, get_confluent_schema, get_connection_tables, + create_connection_table, delete_connection_table, get_connection_tables, test_connection_table, test_schema, }; use crate::connectors::get_connectors; @@ -112,10 +112,6 @@ pub fn create_rest_app(pool: Pool, controller_addr: &str) -> Router { .route("/connection_tables", post(create_connection_table)) .route("/connection_tables/test", post(test_connection_table)) .route("/connection_tables/schemas/test", post(test_schema)) - .route( - "/connection_tables/schemas/confluent", - get(get_confluent_schema), - ) .route("/connection_tables/:id", delete(delete_connection_table)) .route("/pipelines", post(post_pipeline)) .route("/pipelines", get(get_pipelines)) diff --git a/arroyo-connectors/src/kafka.rs b/arroyo-connectors/src/kafka.rs index eb524d839..dea6e3014 100644 --- a/arroyo-connectors/src/kafka.rs +++ b/arroyo-connectors/src/kafka.rs @@ -154,6 +154,7 @@ impl Connector for KafkaConnector { let connection = KafkaConfig { authentication: auth, bootstrap_servers: BootstrapServers(pull_opt("bootstrap_servers", opts)?), + schema_registry: None, }; let typ = pull_opt("type", opts)?; diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs index 4ebac9549..bc72433e0 100644 --- a/arroyo-rpc/src/schema_resolver.rs +++ b/arroyo-rpc/src/schema_resolver.rs @@ -64,7 +64,7 @@ impl ConfluentSchemaResolver { } - async fn get_schema(&self, version: Option) -> anyhow::Result { + async fn get_schema(&self, version: Option) -> anyhow::Result { let url = self.endpoint.join( &version.map(|v| format!("{}", v)).unwrap_or_else(|| "latest".to_string())).unwrap(); diff --git a/arroyo-sql/src/lib.rs b/arroyo-sql/src/lib.rs index 3f536ab9f..f58a1b4dd 100644 --- a/arroyo-sql/src/lib.rs +++ b/arroyo-sql/src/lib.rs @@ -637,6 +637,7 @@ pub fn get_test_expression( KafkaConfig { authentication: arroyo_connectors::kafka::KafkaConfigAuthentication::None {}, bootstrap_servers: "localhost:9092".to_string().try_into().unwrap(), + schema_registry: None, }, KafkaTable { topic: "test_topic".to_string(), diff --git a/connector-schemas/kafka/connection.json b/connector-schemas/kafka/connection.json index edf3957ae..56e109753 100644 --- a/connector-schemas/kafka/connection.json +++ b/connector-schemas/kafka/connection.json @@ -49,6 +49,20 @@ "additionalProperties": false } ] + }, + "schemaRegistry": { + "type": "object", + "properties": { + "endpoint": { + "title": "Endpoint", + "type": "string", + "description": "The endpoint for your Confluent Schema Registry", + "examples": [ + "http://localhost:8081" + ], + "format": "uri" + } + } } }, "required": [ From a3a91dd8d0e0f357db15c1f0ecbbff64369a4f04 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Fri, 20 Oct 2023 18:03:16 -0700 Subject: [PATCH 03/12] progress --- arroyo-console/package.json | 2 +- arroyo-console/src/gen/api-types.ts | 38 +----- .../connections/ConfluentSchemaEditor.tsx | 109 ++---------------- .../src/routes/connections/DefineSchema.tsx | 23 +++- .../src/routes/connections/JsonForm.tsx | 3 +- .../routes/connections/JsonSchemaEditor.tsx | 1 - arroyo-rpc/src/formats.rs | 7 -- arroyo-rpc/src/schema_resolver.rs | 6 +- connector-schemas/kafka/connection.json | 6 +- 9 files changed, 41 insertions(+), 154 deletions(-) diff --git a/arroyo-console/package.json b/arroyo-console/package.json index 0fdab3292..766e8ed86 100644 --- a/arroyo-console/package.json +++ b/arroyo-console/package.json @@ -9,7 +9,7 @@ "preview": "vite preview", "format": "npx prettier --write src/ && npx eslint --fix --ext .js,.jsx,.ts,.tsx src", "check": "npx prettier --check src/ && npx eslint --ext .js,.jsx,.ts,.tsx src", - "openapi": "cargo build && npx openapi-typescript $(pwd)/../arroyo-openapi/api-spec.json --output $(pwd)/src/gen/api-types.ts" + "openapi": "cargo build --package arroyo-openapi && npx openapi-typescript $(pwd)/../arroyo-openapi/api-spec.json --output $(pwd)/src/gen/api-types.ts" }, "dependencies": { "@babel/core": "^7.22.5", diff --git a/arroyo-console/src/gen/api-types.ts b/arroyo-console/src/gen/api-types.ts index 002ff3817..d7135192a 100644 --- a/arroyo-console/src/gen/api-types.ts +++ b/arroyo-console/src/gen/api-types.ts @@ -34,13 +34,6 @@ export interface paths { */ post: operations["create_connection_table"]; }; - "/v1/connection_tables/schemas/confluent": { - /** - * Get a Confluent Schema - * @description Get a Confluent Schema - */ - get: operations["get_confluent_schema"]; - }; "/v1/connection_tables/schemas/test": { /** * Test a Connection Schema @@ -183,7 +176,10 @@ export type webhooks = Record; export interface components { schemas: { - AvroFormat: Record; + AvroFormat: { + confluentSchemaRegistry?: components["schemas"]["ConfluentSchemaRegistryConfig"] | null; + writerSchema?: string | null; + }; Checkpoint: { backend: string; /** Format: int32 */ @@ -206,8 +202,8 @@ export interface components { }; /** @enum {string} */ CheckpointSpanType: "alignment" | "sync" | "async" | "committing"; - ConfluentSchema: { - schema: string; + ConfluentSchemaRegistryConfig: { + endpoint: string; }; ConnectionProfile: { config: unknown; @@ -588,28 +584,6 @@ export interface operations { }; }; }; - /** - * Get a Confluent Schema - * @description Get a Confluent Schema - */ - get_confluent_schema: { - parameters: { - query: { - /** @description Confluent topic name */ - topic: string; - /** @description Confluent schema registry endpoint */ - endpoint: string; - }; - }; - responses: { - /** @description Got Confluent Schema */ - 200: { - content: { - "application/json": components["schemas"]["ConfluentSchema"]; - }; - }; - }; - }; /** * Test a Connection Schema * @description Test a Connection Schema diff --git a/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx b/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx index 7076001db..49d465314 100644 --- a/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx +++ b/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx @@ -1,20 +1,9 @@ -import { Dispatch, useState } from 'react'; +import { Dispatch } from 'react'; import { CreateConnectionState } from './CreateConnection'; import { - Alert, - AlertIcon, - Box, Button, - Code, - FormControl, - FormHelperText, - FormLabel, - Heading, - Input, - Stack, + Stack, Text, } from '@chakra-ui/react'; -import { get } from '../../lib/data_fetching'; -import { formatError } from '../../lib/util'; export function ConfluentSchemaEditor({ state, @@ -25,99 +14,15 @@ export function ConfluentSchemaEditor({ setState: Dispatch; next: () => void; }) { - const [endpoint, setEndpoint] = useState(''); - const [schema, setSchema] = useState(null); - const [error, setError] = useState(null); - const [loading, setLoading] = useState(false); - - const topic = state.table.topic; - - const fetchSchema = async () => { - setError(null); - setLoading(true); - const { data, error } = await get('/v1/connection_tables/schemas/confluent', { - params: { - query: { - endpoint, - topic, - }, - }, - }); - - if (error) { - setError(formatError(error)); - } - - if (data) { - setSchema(data.schema); - setState({ - ...state, - schema: { - ...state.schema, - definition: { json_schema: data.schema }, - fields: [], - format: { json: { confluentSchemaRegistry: true } }, - }, - }); - } - setLoading(false); - }; - - let errorBox = null; - if (error != null) { - errorBox = ( - - - - {error} - - - ); - } - - let successBox = null; - if (schema != null) { - successBox = ( - - Fetched schema for topic {topic} - -
{JSON.stringify(JSON.parse(schema), null, 2)}
-
-
- ); - } - return ( - - Schema Registry Endpoint - setEndpoint(e.target.value)} - /> - Provide the endpoint for your Confluent Schema Registry - - - {errorBox} + + Schemas will be loaded from the configured Confluent Schema Registry + - - - {successBox} - - {schema != null && ( - - )} ); } diff --git a/arroyo-console/src/routes/connections/DefineSchema.tsx b/arroyo-console/src/routes/connections/DefineSchema.tsx index 1c0f4bb49..53d734555 100644 --- a/arroyo-console/src/routes/connections/DefineSchema.tsx +++ b/arroyo-console/src/routes/connections/DefineSchema.tsx @@ -12,17 +12,19 @@ import { import React, { ChangeEvent, ChangeEventHandler, Dispatch, ReactElement, useState } from 'react'; import { CreateConnectionState } from './CreateConnection'; import { JsonSchemaEditor } from './JsonSchemaEditor'; -import { Connector } from '../../lib/data_fetching'; +import {ConnectionProfile, Connector, useConnectionProfiles} from '../../lib/data_fetching'; import { ConfluentSchemaEditor } from './ConfluentSchemaEditor'; import { components } from '../../gen/api-types'; const JsonEditor = ({ connector, + connectionProfiles, state, setState, next, }: { connector: Connector; + connectionProfiles: Array, state: CreateConnectionState; setState: Dispatch; next: () => void; @@ -32,7 +34,13 @@ const JsonEditor = ({ { name: 'JSON Schema', value: 'json' }, { name: 'Unstructured JSON', value: 'unstructured' }, ]; - if (connector.id == 'kafka') { + + let connectionProfile = null; + if (state.connectionProfileId != null) { + connectionProfile = connectionProfiles.find(c => c.id == state.connectionProfileId); + } + + if (connector.id == 'kafka' && connectionProfile != null && (connectionProfile.config as any).schemaRegistry != null) { schemaTypeOptions.push({ name: 'Confluent Schema Registry', value: 'confluent' }); } @@ -84,7 +92,7 @@ const JsonEditor = ({ ...state, schema: { ...state.schema, - definition: { json_schema: '' }, + definition: null, fields: [], format: { json: { unstructured: false, confluentSchemaRegistry: true } }, }, @@ -192,11 +200,18 @@ export const DefineSchema = ({ const [selectedFormat, setSelectedFormat] = useState(undefined); const [selectedFraming, setSelectedFraming] = useState(undefined); + let { connectionProfiles, connectionProfilesLoading } = useConnectionProfiles(); + + if (connectionProfilesLoading) { + return <>; + } + + const formats: DataFormatOption[] = [ { name: 'JSON', value: 'json', - el: , + el: , }, { name: 'Raw String', diff --git a/arroyo-console/src/routes/connections/JsonForm.tsx b/arroyo-console/src/routes/connections/JsonForm.tsx index e005e1b09..60133a27a 100644 --- a/arroyo-console/src/routes/connections/JsonForm.tsx +++ b/arroyo-console/src/routes/connections/JsonForm.tsx @@ -344,7 +344,6 @@ export function FormInner({ onChange={onChange} /> ); - 342; } case 'array': { return ( @@ -401,7 +400,7 @@ export function FormInner({ ); - } else if ((values[key].properties?.length || 0) > 0) { + } else if (property.properties != undefined) { return (
{ - console.log('testing schema'); setTesting(true); setErrors(null); const { error } = await post('/v1/connection_tables/schemas/test', { diff --git a/arroyo-rpc/src/formats.rs b/arroyo-rpc/src/formats.rs index 3ea169114..0230fc60f 100644 --- a/arroyo-rpc/src/formats.rs +++ b/arroyo-rpc/src/formats.rs @@ -103,19 +103,12 @@ pub struct ConfluentSchemaRegistryConfig { #[serde(rename_all = "camelCase")] pub struct AvroFormat { pub writer_schema: Option, - pub confluent_schema_registry: Option, } impl AvroFormat { pub fn from_opts(opts: &mut HashMap) -> Result { - let confluent_schema_registry = opts.remove("avro.confluent_schema_registry.endpoint") - .map(|endpoint| ConfluentSchemaRegistryConfig { - endpoint - }); - Ok(Self { writer_schema: None, - confluent_schema_registry, }) } } diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs index bc72433e0..a4398e426 100644 --- a/arroyo-rpc/src/schema_resolver.rs +++ b/arroyo-rpc/src/schema_resolver.rs @@ -22,7 +22,7 @@ impl SchemaResolver for FailingSchemaResolver { } } -#[derive(Clone, Debug, Serialize, Deserialize)] +#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)] #[serde(rename_all = "UPPERCASE")] pub enum ConfluentSchemaType { Avro, @@ -39,7 +39,7 @@ pub struct ConfluentSchemaResponse { pub schema: String, } -struct ConfluentSchemaResolver { +pub struct ConfluentSchemaResolver { endpoint: Url, topic: String, client: Client, @@ -64,7 +64,7 @@ impl ConfluentSchemaResolver { } - async fn get_schema(&self, version: Option) -> anyhow::Result { + pub async fn get_schema(&self, version: Option) -> anyhow::Result { let url = self.endpoint.join( &version.map(|v| format!("{}", v)).unwrap_or_else(|| "latest".to_string())).unwrap(); diff --git a/connector-schemas/kafka/connection.json b/connector-schemas/kafka/connection.json index 56e109753..96405d6ad 100644 --- a/connector-schemas/kafka/connection.json +++ b/connector-schemas/kafka/connection.json @@ -52,17 +52,19 @@ }, "schemaRegistry": { "type": "object", + "title": "Schema Registry", "properties": { "endpoint": { "title": "Endpoint", "type": "string", - "description": "The endpoint for your Confluent Schema Registry", + "description": "The endpoint for your Confluent Schema Registry if you have one", "examples": [ "http://localhost:8081" ], "format": "uri" } - } + }, + "required": ["endpoint"] } }, "required": [ From bf7bcaff95f98cb6336f4b941d8e350d40e52693 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Sat, 21 Oct 2023 10:15:08 -0700 Subject: [PATCH 04/12] re-implemented json confluent schema support --- Cargo.lock | 13 ++++ arroyo-api/Cargo.toml | 2 +- arroyo-api/src/connection_tables.rs | 81 +++++++++++++++++++------ arroyo-console/src/lib/data_fetching.ts | 2 +- arroyo-rpc/src/schema_resolver.rs | 7 ++- 5 files changed, 82 insertions(+), 23 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2aedfaa95..e7072659d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1418,6 +1418,7 @@ checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" dependencies = [ "async-trait", "axum-core", + "axum-macros", "bitflags 1.3.2", "bytes", "futures-util", @@ -1481,6 +1482,18 @@ dependencies = [ "tower-service", ] +[[package]] +name = "axum-macros" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cdca6a10ecad987bda04e95606ef85a5417dcaac1a78455242d72e031e2b6b62" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.33", +] + [[package]] name = "backoff" version = "0.4.0" diff --git a/arroyo-api/Cargo.toml b/arroyo-api/Cargo.toml index bca408689..9ce5e6361 100644 --- a/arroyo-api/Cargo.toml +++ b/arroyo-api/Cargo.toml @@ -37,7 +37,7 @@ petgraph = {version = "0.6", features = ["serde-1"]} http = "0.2" tower-http = {version = "0.4", features = ["trace", "fs", "cors", "validate-request", "auth"]} -axum = {version = "0.6.12", features = ["headers", "tokio"]} +axum = {version = "0.6.12", features = ["headers", "tokio", "macros"]} axum-extra = "0.7.4" thiserror = "1.0.40" utoipa = "3" diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index 3c83fd5b4..08ebd2ca6 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -10,19 +10,22 @@ use cornucopia_async::GenericClient; use cornucopia_async::Params; use futures_util::stream::Stream; use http::StatusCode; -use serde_json::json; +use serde_json::{json, Value}; use std::convert::Infallible; use tokio::sync::mpsc::channel; use tokio_stream::wrappers::ReceiverStream; use tracing::warn; -use arroyo_connectors::{connector_for_type, ErasedConnector}; +use arroyo_connectors::{Connector, connector_for_type, ErasedConnector}; +use arroyo_connectors::kafka::{KafkaConfig, KafkaConnector, KafkaTable}; use arroyo_rpc::api_types::connections::{ ConnectionProfile, ConnectionSchema, ConnectionTable, ConnectionTablePost, SchemaDefinition, }; use arroyo_rpc::api_types::{ConnectionTableCollection, PaginationQueryParams}; +use arroyo_rpc::formats::{Format, JsonFormat}; use arroyo_rpc::public_ids::{generate_id, IdTypes}; +use arroyo_rpc::schema_resolver::{ConfluentSchemaResolver, ConfluentSchemaType}; use arroyo_sql::json_schema::convert_json_schema; use arroyo_sql::types::{StructField, TypeDef}; @@ -95,8 +98,9 @@ async fn get_and_validate_connector( .transpose() .map_err(|e| bad_request(format!("Invalid schema: {}", e)))?; - let schema = if let Some(schema) = &schema { - Some(expand_schema(&req.name, schema)?) + let schema = if let Some(schema) = schema { + let name = connector.name(); + Some(expand_schema(&req.name, name, schema, &req.config, &profile_config).await?) } else { None }; @@ -137,6 +141,7 @@ pub(crate) async fn delete_connection_table( } /// Test a Connection Table +#[axum::debug_handler] #[utoipa::path( post, path = "/v1/connection_tables/test", @@ -413,31 +418,70 @@ pub(crate) async fn get_connection_tables( // attempts to fill in the SQL schema from a schema object that may just have a json-schema or // other source schema. schemas stored in the database should always be expanded first. -pub(crate) fn expand_schema( +pub(crate) async fn expand_schema( name: &str, - schema: &ConnectionSchema, + connector: &str, + schema: ConnectionSchema, + table_config: &Value, + profile_config: &Value, ) -> Result { - let mut schema = schema.clone(); + let Some(format) = schema.format.as_ref() else { + return Ok(schema); + }; + + match format { + Format::Json(_) => expand_json_schema(name, connector, schema, table_config, profile_config).await, + Format::Avro(_) => todo!(), + Format::Parquet(_) => Ok(schema), + Format::RawString(_) => Ok(schema), + } + +} + +async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionSchema, table_config: &Value, profile_config: &Value) -> Result { + if let Some(Format::Json(JsonFormat { confluent_schema_registry: true, .. })) = &schema.format { + if connector != "kafka" { + return Err(bad_request("confluent schema registry can only be used for Kafka connections".to_string())); + } + + // we unwrap here because this should already have been validated + let profile: KafkaConfig = serde_json::from_value(profile_config.clone()) + .expect("invalid kafka config"); + + let table: KafkaTable = serde_json::from_value(table_config.clone()) + .expect("invalid kafka table"); + + let schema_registry = profile.schema_registry.as_ref().ok_or_else(|| + bad_request("schema registry must be configured on the Kafka connection profile".to_string()))?; + + let resolver = ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic) + .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e)))?; + + let schema_response = resolver.get_schema(None).await + .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e)))?; + + if schema_response.schema_type != ConfluentSchemaType::Json { + return Err(bad_request(format!("Format configured is json, but confluent schema repository returned a {:?} schema", + schema_response.schema_type))); + } + + schema.definition = Some(SchemaDefinition::JsonSchema(schema_response.schema)); + } if let Some(d) = &schema.definition { let fields = match d { - SchemaDefinition::JsonSchema(json) => convert_json_schema(name, &json) + SchemaDefinition::JsonSchema(json) => convert_json_schema(&name, &json) .map_err(|e| bad_request(format!("Invalid json-schema: {}", e)))?, - SchemaDefinition::ProtobufSchema(_) => { - return Err(bad_request( - "Protobuf schemas are not yet supported".to_string(), - )) - } - SchemaDefinition::AvroSchema(_) => { - return Err(bad_request( - "Avro schemas are not yet supported".to_string(), - )) - } SchemaDefinition::RawSchema(_) => vec![StructField::new( "value".to_string(), None, TypeDef::DataType(DataType::Utf8, false), )], + _ => { + return Err(bad_request( + "Invalid schema type for json format".to_string(), + )) + } }; let fields: Result<_, String> = fields.into_iter().map(|f| f.try_into()).collect(); @@ -447,6 +491,7 @@ pub(crate) fn expand_schema( } Ok(schema) + } /// Test a Connection Schema diff --git a/arroyo-console/src/lib/data_fetching.ts b/arroyo-console/src/lib/data_fetching.ts index 69e32d0f7..b0d2b1da2 100644 --- a/arroyo-console/src/lib/data_fetching.ts +++ b/arroyo-console/src/lib/data_fetching.ts @@ -140,7 +140,7 @@ const pingFetcher = async () => { export const usePing = () => { const { data, error, isLoading } = useSWR('ping', pingFetcher, { - refreshInterval: 1000, + refreshInterval: 3000000, onErrorRetry: (error, key, config, revalidate, {}) => { // explicitly define this function to override the exponential backoff setTimeout(() => revalidate(), 1000); diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs index a4398e426..69576e55a 100644 --- a/arroyo-rpc/src/schema_resolver.rs +++ b/arroyo-rpc/src/schema_resolver.rs @@ -33,10 +33,11 @@ pub enum ConfluentSchemaType { #[derive(Clone, Debug, Serialize, Deserialize)] #[serde(rename_all = "camelCase")] pub struct ConfluentSchemaResponse { - pub name: String, - pub version: String, - pub schema_type: ConfluentSchemaType, + pub id: u32, pub schema: String, + pub schema_type: ConfluentSchemaType, + pub subject: String, + pub version: u32, } pub struct ConfluentSchemaResolver { From c2666ce5fe4a8d3dff46a96800ddc7463b976e3f Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Sat, 21 Oct 2023 18:03:19 -0700 Subject: [PATCH 05/12] handling avro schemas --- Cargo.lock | 15 +- arroyo-api/src/connection_tables.rs | 73 +++++--- arroyo-api/src/jobs.rs | 2 +- arroyo-api/src/pipelines.rs | 12 +- arroyo-api/src/rest.rs | 2 +- arroyo-api/src/rest_utils.rs | 10 +- arroyo-console/src/gen/api-types.ts | 6 +- .../src/routes/connections/DefineSchema.tsx | 163 ++++++++++-------- .../src/routes/connections/JsonForm.tsx | 3 +- ...{JsonSchemaEditor.tsx => SchemaEditor.tsx} | 28 ++- arroyo-rpc/Cargo.toml | 1 + arroyo-rpc/src/formats.rs | 4 +- arroyo-rpc/src/schema_resolver.rs | 12 +- arroyo-sql/Cargo.toml | 3 +- arroyo-sql/src/avro.rs | 115 ++++++++++++ arroyo-sql/src/lib.rs | 1 + 16 files changed, 319 insertions(+), 131 deletions(-) rename arroyo-console/src/routes/connections/{JsonSchemaEditor.tsx => SchemaEditor.tsx} (85%) create mode 100644 arroyo-sql/src/avro.rs diff --git a/Cargo.lock b/Cargo.lock index e7072659d..30760e7fc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -670,6 +670,7 @@ dependencies = [ "tokio", "tonic", "tonic-build", + "tracing", "utoipa", ] @@ -704,6 +705,7 @@ name = "arroyo-sql" version = "0.7.0" dependencies = [ "anyhow", + "apache-avro", "arrow", "arrow-schema", "arroyo-connectors", @@ -7966,11 +7968,10 @@ checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" -version = "0.1.37" +version = "0.1.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" dependencies = [ - "cfg-if", "log", "pin-project-lite", "tracing-attributes", @@ -7990,9 +7991,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", @@ -8001,9 +8002,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" dependencies = [ "once_cell", "valuable", diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index 08ebd2ca6..3880faad2 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -23,9 +23,10 @@ use arroyo_rpc::api_types::connections::{ ConnectionTable, ConnectionTablePost, SchemaDefinition, }; use arroyo_rpc::api_types::{ConnectionTableCollection, PaginationQueryParams}; -use arroyo_rpc::formats::{Format, JsonFormat}; +use arroyo_rpc::formats::{AvroFormat, Format, JsonFormat}; use arroyo_rpc::public_ids::{generate_id, IdTypes}; -use arroyo_rpc::schema_resolver::{ConfluentSchemaResolver, ConfluentSchemaType}; +use arroyo_rpc::schema_resolver::{ConfluentSchemaResolver, ConfluentSchemaResponse, ConfluentSchemaType}; +use arroyo_sql::avro; use arroyo_sql::json_schema::convert_json_schema; use arroyo_sql::types::{StructField, TypeDef}; @@ -134,7 +135,7 @@ pub(crate) async fn delete_connection_table( .map_err(|e| handle_delete("connection_table", "pipelines", e))?; if deleted == 0 { - return Err(not_found("Connection table".to_string())); + return Err(not_found("Connection table")); } Ok(()) @@ -431,34 +432,44 @@ pub(crate) async fn expand_schema( match format { Format::Json(_) => expand_json_schema(name, connector, schema, table_config, profile_config).await, - Format::Avro(_) => todo!(), + Format::Avro(_) => expand_avro_schema(name, connector, schema, table_config, profile_config).await, Format::Parquet(_) => Ok(schema), Format::RawString(_) => Ok(schema), } } -async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionSchema, table_config: &Value, profile_config: &Value) -> Result { - if let Some(Format::Json(JsonFormat { confluent_schema_registry: true, .. })) = &schema.format { - if connector != "kafka" { - return Err(bad_request("confluent schema registry can only be used for Kafka connections".to_string())); +async fn expand_avro_schema(name: &str, connector: &str, mut schema: ConnectionSchema, table_config: &Value, profile_config: &Value) -> Result { + if let Some(Format::Avro(AvroFormat { confluent_schema_registry: true, .. })) = &schema.format { + let schema_response = get_schema(connector, table_config, profile_config).await?; + + if schema_response.schema_type != ConfluentSchemaType::Avro { + return Err(bad_request(format!("Format configured is avro, but confluent schema repository returned a {:?} schema", + schema_response.schema_type))); } - // we unwrap here because this should already have been validated - let profile: KafkaConfig = serde_json::from_value(profile_config.clone()) - .expect("invalid kafka config"); + schema.definition = Some(SchemaDefinition::AvroSchema(schema_response.schema)); + } + + + let Some(SchemaDefinition::AvroSchema(definition)) = schema.definition.as_ref() else { + return Err(bad_request("avro format requires an avro schema be set")); + }; + + let fields: Result<_, String> = avro::convert_avro_schema(&name, &definition) + .map_err(|e| bad_request(format!("Invalid avro schema: {}", e)))? + .into_iter().map(|f| f.try_into()).collect(); - let table: KafkaTable = serde_json::from_value(table_config.clone()) - .expect("invalid kafka table"); - let schema_registry = profile.schema_registry.as_ref().ok_or_else(|| - bad_request("schema registry must be configured on the Kafka connection profile".to_string()))?; + schema.fields = fields + .map_err(|e| bad_request(format!("Failed to convert schema: {}", e)))?; - let resolver = ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic) - .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e)))?; + Ok(schema) +} - let schema_response = resolver.get_schema(None).await - .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e)))?; +async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionSchema, table_config: &Value, profile_config: &Value) -> Result { + if let Some(Format::Json(JsonFormat { confluent_schema_registry: true, .. })) = &schema.format { + let schema_response = get_schema(connector, table_config, profile_config).await?; if schema_response.schema_type != ConfluentSchemaType::Json { return Err(bad_request(format!("Format configured is json, but confluent schema repository returned a {:?} schema", @@ -479,7 +490,7 @@ async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionS )], _ => { return Err(bad_request( - "Invalid schema type for json format".to_string(), + "Invalid schema type for json format", )) } }; @@ -494,6 +505,28 @@ async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionS } +async fn get_schema(connector: &str, table_config: &Value, profile_config: &Value) -> Result { + if connector != "kafka" { + return Err(bad_request("confluent schema registry can only be used for Kafka connections")); + } + + // we unwrap here because this should already have been validated + let profile: KafkaConfig = serde_json::from_value(profile_config.clone()) + .expect("invalid kafka config"); + + let table: KafkaTable = serde_json::from_value(table_config.clone()) + .expect("invalid kafka table"); + + let schema_registry = profile.schema_registry.as_ref().ok_or_else(|| + bad_request("schema registry must be configured on the Kafka connection profile"))?; + + let resolver = ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic) + .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e)))?; + + resolver.get_schema(None).await + .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e))) +} + /// Test a Connection Schema #[utoipa::path( post, diff --git a/arroyo-api/src/jobs.rs b/arroyo-api/src/jobs.rs index f9c627663..ce6feca91 100644 --- a/arroyo-api/src/jobs.rs +++ b/arroyo-api/src/jobs.rs @@ -416,7 +416,7 @@ pub async fn get_checkpoint_details( .await .map_err(log_and_map)? .ok_or_else(|| { - not_found(format!( + not_found(&format!( "Checkpoint with epoch {} for job '{}'", epoch, job_pub_id )) diff --git a/arroyo-api/src/pipelines.rs b/arroyo-api/src/pipelines.rs index 63677a9b2..ccfe6e3c9 100644 --- a/arroyo-api/src/pipelines.rs +++ b/arroyo-api/src/pipelines.rs @@ -600,7 +600,7 @@ pub async fn patch_pipeline( .opt() .await .map_err(log_and_map)? - .ok_or_else(|| not_found("Job".to_string()))?; + .ok_or_else(|| not_found("Job"))?; let program = PipelineProgram::decode(&res.program[..]).map_err(log_and_map)?; let map: HashMap = program @@ -629,7 +629,7 @@ pub async fn patch_pipeline( .map_err(log_and_map)?; if res == 0 { - return Err(not_found("Job".to_string())); + return Err(not_found("Job")); } let pipeline = query_pipeline_by_pub_id(&pipeline_pub_id, &client, &auth_data).await?; @@ -683,7 +683,7 @@ pub async fn restart_pipeline( .map_err(log_and_map)?; if res == 0 { - return Err(not_found("Pipeline".to_string())); + return Err(not_found("Pipeline")); } let pipeline = query_pipeline_by_pub_id(&id, &client, &auth_data).await?; @@ -814,7 +814,7 @@ pub async fn delete_pipeline( .map_err(log_and_map)?; if count != 1 { - return Err(not_found("Pipeline".to_string())); + return Err(not_found("Pipeline")); } Ok(()) @@ -864,7 +864,7 @@ pub async fn query_pipeline_by_pub_id( .await .map_err(log_and_map)?; - let res = pipeline.ok_or_else(|| not_found("Pipeline".to_string()))?; + let res = pipeline.ok_or_else(|| not_found("Pipeline"))?; res.try_into() } @@ -884,7 +884,7 @@ pub async fn query_job_by_pub_id( .await .map_err(log_and_map)?; - let res: DbPipelineJob = job.ok_or_else(|| not_found("Job".to_string()))?; + let res: DbPipelineJob = job.ok_or_else(|| not_found("Job"))?; Ok(res.into()) } diff --git a/arroyo-api/src/rest.rs b/arroyo-api/src/rest.rs index 684f82334..bb713012a 100644 --- a/arroyo-api/src/rest.rs +++ b/arroyo-api/src/rest.rs @@ -55,7 +55,7 @@ pub async fn ping() -> impl IntoResponse { } pub async fn api_fallback() -> impl IntoResponse { - not_found("Route".to_string()) + not_found("Route") } pub fn create_rest_app(pool: Pool, controller_addr: &str) -> Router { diff --git a/arroyo-api/src/rest_utils.rs b/arroyo-api/src/rest_utils.rs index e09f1127f..02822f435 100644 --- a/arroyo-api/src/rest_utils.rs +++ b/arroyo-api/src/rest_utils.rs @@ -76,21 +76,21 @@ pub(crate) async fn authenticate( cloud::authenticate(client, bearer_auth).await } -pub(crate) fn bad_request(message: String) -> ErrorResp { +pub(crate) fn bad_request(message: impl Into) -> ErrorResp { ErrorResp { status_code: StatusCode::BAD_REQUEST, - message: message.to_string(), + message: message.into(), } } -pub(crate) fn unauthorized(message: String) -> ErrorResp { +pub(crate) fn unauthorized(message: impl Into) -> ErrorResp { ErrorResp { status_code: StatusCode::UNAUTHORIZED, - message: message.to_string(), + message: message.into(), } } -pub(crate) fn not_found(object: String) -> ErrorResp { +pub(crate) fn not_found(object: &str) -> ErrorResp { ErrorResp { status_code: StatusCode::NOT_FOUND, message: format!("{} not found", object), diff --git a/arroyo-console/src/gen/api-types.ts b/arroyo-console/src/gen/api-types.ts index d7135192a..71239a628 100644 --- a/arroyo-console/src/gen/api-types.ts +++ b/arroyo-console/src/gen/api-types.ts @@ -177,8 +177,7 @@ export type webhooks = Record; export interface components { schemas: { AvroFormat: { - confluentSchemaRegistry?: components["schemas"]["ConfluentSchemaRegistryConfig"] | null; - writerSchema?: string | null; + confluentSchemaRegistry: boolean; }; Checkpoint: { backend: string; @@ -202,9 +201,6 @@ export interface components { }; /** @enum {string} */ CheckpointSpanType: "alignment" | "sync" | "async" | "committing"; - ConfluentSchemaRegistryConfig: { - endpoint: string; - }; ConnectionProfile: { config: unknown; connector: string; diff --git a/arroyo-console/src/routes/connections/DefineSchema.tsx b/arroyo-console/src/routes/connections/DefineSchema.tsx index 53d734555..fa53058cc 100644 --- a/arroyo-console/src/routes/connections/DefineSchema.tsx +++ b/arroyo-console/src/routes/connections/DefineSchema.tsx @@ -11,30 +11,35 @@ import { } from '@chakra-ui/react'; import React, { ChangeEvent, ChangeEventHandler, Dispatch, ReactElement, useState } from 'react'; import { CreateConnectionState } from './CreateConnection'; -import { JsonSchemaEditor } from './JsonSchemaEditor'; -import {ConnectionProfile, Connector, useConnectionProfiles} from '../../lib/data_fetching'; +import { SchemaEditor } from './SchemaEditor'; +import {ConnectionProfile, ConnectionSchema, Connector, useConnectionProfiles} from '../../lib/data_fetching'; import { ConfluentSchemaEditor } from './ConfluentSchemaEditor'; import { components } from '../../gen/api-types'; -const JsonEditor = ({ +const SchemaFormatEditor = ({ connector, connectionProfiles, state, setState, next, + format, }: { connector: Connector; connectionProfiles: Array, state: CreateConnectionState; setState: Dispatch; next: () => void; + format: 'json' | 'avro', }) => { type SchemaTypeOption = { name: string; value: string }; let schemaTypeOptions: SchemaTypeOption[] = [ - { name: 'JSON Schema', value: 'json' }, - { name: 'Unstructured JSON', value: 'unstructured' }, + { name: format + ' schema', value: 'schema' }, ]; + if (format == 'json') { + schemaTypeOptions.push({ name: 'Unstructured JSON', value: 'unstructured' }); + } + let connectionProfile = null; if (state.connectionProfileId != null) { connectionProfile = connectionProfiles.find(c => c.id == state.connectionProfileId); @@ -44,49 +49,64 @@ const JsonEditor = ({ schemaTypeOptions.push({ name: 'Confluent Schema Registry', value: 'confluent' }); } - const [selectedSchemaType, setSelectedSchemaType] = useState( - undefined - ); - - let editor: JSX.Element | null = null; - switch (selectedSchemaType?.value) { + let def_name: 'json_schema' | 'avro_schema'; + switch (format) { case 'json': - editor = ; - break; - case 'confluent': - editor = ; - break; - case 'unstructured': - editor = ( - - - Connection tables configured with an unstructured JSON schema have a single{' '} - value column with the JSON value, which can be accessed using SQL{' '} - - JSON functions - - . - - - - ); - break; + def_name = 'json_schema'; + break; + case 'avro': + def_name = 'avro_schema'; + break; + default: + throw new Error("unknown format: " + format); + } + + let editor; + let value; + if ((state.schema?.format![format] || {})["confluentSchemaRegistry"]) { + editor = ; + value = 'confluent'; + } else if (state.schema?.format?.json?.unstructured) { + editor = ( + + + Connection tables configured with an unstructured JSON schema have a single{' '} + value column with the JSON value, which can be accessed using SQL{' '} + + JSON functions + + . + + + + ); + value = 'unstructured'; + } else if ((state.schema?.definition || {})[def_name] != undefined) { + editor = ; + value = "schema"; + } else { + editor = null; + value = undefined; } const onChange = (e: React.ChangeEvent) => { - setSelectedSchemaType(schemaTypeOptions.find(o => o.value == e.target.value)); switch (e.target.value) { - case 'json': + case 'schema': { setState({ ...state, schema: { ...state.schema, - definition: { json_schema: '' }, + // @ts-ignore + definition: { + [def_name]: '' + }, fields: [], - format: { json: { unstructured: false, confluentSchemaRegistry: false } }, + // @ts-ignore + format: {[format]: { }}, }, }); break; + } case 'confluent': setState({ ...state, @@ -94,7 +114,8 @@ const JsonEditor = ({ ...state.schema, definition: null, fields: [], - format: { json: { unstructured: false, confluentSchemaRegistry: true } }, + // @ts-ignore + format: { [format]: { confluentSchemaRegistry: true } }, }, }); break; @@ -109,7 +130,18 @@ const JsonEditor = ({ }, }); break; - } + default: + setState({ + ...state, + schema: { + ...state.schema, + definition: undefined, + fields: [], + // @ts-ignore + format: { [format]: { } }, + }, + }); + } }; return ( @@ -119,7 +151,7 @@ const JsonEditor = ({ ({ value: value!.toString(), label: value!.toString(), diff --git a/arroyo-console/src/routes/connections/JsonSchemaEditor.tsx b/arroyo-console/src/routes/connections/SchemaEditor.tsx similarity index 85% rename from arroyo-console/src/routes/connections/JsonSchemaEditor.tsx rename to arroyo-console/src/routes/connections/SchemaEditor.tsx index 11ff44f93..b2bff3897 100644 --- a/arroyo-console/src/routes/connections/JsonSchemaEditor.tsx +++ b/arroyo-console/src/routes/connections/SchemaEditor.tsx @@ -2,17 +2,19 @@ import { Dispatch, useEffect, useRef, useState } from 'react'; import { CreateConnectionState } from './CreateConnection'; import { Alert, AlertIcon, Box, Button, List, ListItem, Stack } from '@chakra-ui/react'; import * as monaco from 'monaco-editor/esm/vs/editor/editor.api'; -import { post } from '../../lib/data_fetching'; +import {ConnectionSchema, post} from '../../lib/data_fetching'; import { formatError } from '../../lib/util'; -export function JsonSchemaEditor({ +export function SchemaEditor({ state, setState, next, + format }: { state: CreateConnectionState; setState: Dispatch; next: () => void; + format: 'avro' | 'json', }) { const [editor, setEditor] = useState(null); const monacoEl = useRef(null); @@ -74,7 +76,6 @@ export function JsonSchemaEditor({ state.schema?.format?.json?.unstructured === false ) { let e = monaco.editor.create(monacoEl.current!, { - // value: state.schema?.definition, language: 'json', theme: 'vs-dark', minimap: { @@ -83,14 +84,23 @@ export function JsonSchemaEditor({ }); e?.getModel()?.onDidChangeContent(_ => { + let schema: ConnectionSchema = { + ...state.schema, + fields: [], + // @ts-ignore + format: {}, + // @ts-ignore + definition: {} + }; + + schema.format![format] = { }; + // @ts-ignore + schema.definition![format + '_schema'] = e.getValue(); + + setState({ ...state, - schema: { - ...state.schema, - definition: { json_schema: e.getValue() }, - fields: [], - format: { json: { confluentSchemaRegistry: false } }, - }, + schema: schema, }); }); diff --git a/arroyo-rpc/Cargo.toml b/arroyo-rpc/Cargo.toml index d7b7df2f9..1e3eedd15 100644 --- a/arroyo-rpc/Cargo.toml +++ b/arroyo-rpc/Cargo.toml @@ -19,6 +19,7 @@ utoipa = "3" anyhow = "1.0.75" reqwest = "0.11.22" log = "0.4.20" +tracing = "0.1.40" [build-dependencies] tonic-build = { workspace = true } diff --git a/arroyo-rpc/src/formats.rs b/arroyo-rpc/src/formats.rs index 0230fc60f..94b0ac451 100644 --- a/arroyo-rpc/src/formats.rs +++ b/arroyo-rpc/src/formats.rs @@ -102,13 +102,13 @@ pub struct ConfluentSchemaRegistryConfig { #[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash, PartialOrd, ToSchema)] #[serde(rename_all = "camelCase")] pub struct AvroFormat { - pub writer_schema: Option, + pub confluent_schema_registry: bool } impl AvroFormat { pub fn from_opts(opts: &mut HashMap) -> Result { Ok(Self { - writer_schema: None, + confluent_schema_registry: false }) } } diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs index 69576e55a..1c4e0414e 100644 --- a/arroyo-rpc/src/schema_resolver.rs +++ b/arroyo-rpc/src/schema_resolver.rs @@ -3,7 +3,7 @@ use anyhow::{anyhow, bail}; use arroyo_types::UserError; use reqwest::{Client, StatusCode, Url}; use serde_json::Value; -use log::warn; +use tracing::warn; use serde::{Deserialize, Serialize}; pub trait SchemaResolver { @@ -22,9 +22,10 @@ impl SchemaResolver for FailingSchemaResolver { } } -#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)] +#[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq, Default)] #[serde(rename_all = "UPPERCASE")] pub enum ConfluentSchemaType { + #[default] Avro, Json, Protobuf @@ -35,6 +36,7 @@ pub enum ConfluentSchemaType { pub struct ConfluentSchemaResponse { pub id: u32, pub schema: String, + #[serde(default)] pub schema_type: ConfluentSchemaType, pub subject: String, pub version: u32, @@ -69,7 +71,7 @@ impl ConfluentSchemaResolver { let url = self.endpoint.join( &version.map(|v| format!("{}", v)).unwrap_or_else(|| "latest".to_string())).unwrap(); - let resp = reqwest::get(url).await.map_err(|e| { + let resp = reqwest::get(url.clone()).await.map_err(|e| { warn!("Got error response from schema registry: {:?}", e); match e.status() { Some(StatusCode::NOT_FOUND) => anyhow!( @@ -101,9 +103,9 @@ impl ConfluentSchemaResolver { } resp.json().await.map_err(|e| { - warn!("Invalid json from schema registry: {:?}", e); + warn!("Invalid json from schema registry: {:?} for request {:?}", e, url); anyhow!( - "Schema registry response could not be deserialied: {}", e + "Schema registry response could not be deserialized: {}", e ) }) } diff --git a/arroyo-sql/Cargo.toml b/arroyo-sql/Cargo.toml index 27e89fac0..df54a7705 100644 --- a/arroyo-sql/Cargo.toml +++ b/arroyo-sql/Cargo.toml @@ -31,4 +31,5 @@ tracing = "0.1.37" typify = "0.0.13" schemars = "0.8" -serde_json_path = "0.6.3" \ No newline at end of file +serde_json_path = "0.6.3" +apache-avro = "0.16.0" \ No newline at end of file diff --git a/arroyo-sql/src/avro.rs b/arroyo-sql/src/avro.rs new file mode 100644 index 000000000..ea0cb5e6b --- /dev/null +++ b/arroyo-sql/src/avro.rs @@ -0,0 +1,115 @@ +use std::sync::Arc; +use anyhow::{anyhow, bail}; +use apache_avro::Schema; +use apache_avro::schema::UnionSchema; +use arrow_schema::{DataType, Field, FieldRef, Fields}; +use tracing::warn; +use crate::types::{StructDef, StructField, TypeDef}; + +pub fn convert_avro_schema(name: &str, schema: &str) -> anyhow::Result> { + let schema = Schema::parse_str(schema) + .map_err(|e| anyhow!("avro schema is not valid: {:?}", e))?; + + let (typedef, _) = to_typedef(name, &schema); + match typedef { + TypeDef::StructDef(sd, _) => { + Ok(sd.fields) + } + TypeDef::DataType(_, _) => { + bail!("top-level schema must be a record") + } + } +} + +fn to_typedef(source_name: &str, schema: &Schema) -> (TypeDef, Option) { + match schema { + Schema::Null => { + (TypeDef::DataType(DataType::Null, false), None) + } + Schema::Boolean => { + (TypeDef::DataType(DataType::Boolean, false), None) + } + Schema::Int => { + (TypeDef::DataType(DataType::Int32, false), None) + } + Schema::Long => { + (TypeDef::DataType(DataType::Int64, false), None) + } + Schema::Float => { + (TypeDef::DataType(DataType::Float32, false), None) + } + Schema::Double => { + (TypeDef::DataType(DataType::Float64, false), None) + } + Schema::Bytes => { + (TypeDef::DataType(DataType::Binary, false), None) + } + Schema::String => { + (TypeDef::DataType(DataType::Utf8, false), None) + } + // Schema::Array(t) => { + // let dt = match to_typedef(source_name, t) { + // (TypeDef::StructDef(sd, _), _) => { + // let fields: Vec = sd.fields.into_iter() + // .map(|f| f.into()) + // .collect(); + // + // DataType::Struct(Fields::from(fields)) + // }, + // (TypeDef::DataType(dt, _), _) => { + // dt + // } + // }; + // + // (TypeDef::DataType(DataType::List(Arc::new(Field::new("", dt,false))), false), None) + // } + Schema::Union(union) => { + // currently just support unions that have [t, null] as variants, which is the + // avro way to represent optional fields + + let (nulls, not_nulls): (Vec<_>, Vec<_>) = union.variants().iter().partition(|v| matches!(v, Schema::Null)); + + if nulls.len() == 1 && not_nulls.len() == 1 { + let (dt, original) = to_typedef(source_name, not_nulls[0]); + (dt.to_optional(), original) + } else { + ( + TypeDef::DataType(DataType::Utf8, false), + Some("json".to_string()), + ) + } + } + Schema::Record(record) => { + let fields = record.fields.iter().map(|f| { + let (ft, original) = to_typedef(source_name, &f.schema); + StructField::with_rename(f.name.clone(), None, ft, None, original) + }).collect(); + + ( + TypeDef::StructDef(StructDef::for_name(Some(format!("{}::{}", source_name, record.name.name)), fields), false), + None + ) + } + _ => { + ( + TypeDef::DataType(DataType::Utf8, false), + Some("json".to_string()), + ) + } + // Schema::Enum(_) => {} + // Schema::Fixed(_) => {} + // Schema::Decimal(_) => {} + // Schema::Uuid => {} + // Schema::Date => {} + // Schema::TimeMillis => {} + // Schema::TimeMicros => {} + // Schema::TimestampMillis => {} + // Schema::TimestampMicros => {} + // Schema::LocalTimestampMillis => {} + // Schema::LocalTimestampMicros => {} + // Schema::Duration => {} + // Schema::Ref { .. } => {} + } +} + + diff --git a/arroyo-sql/src/lib.rs b/arroyo-sql/src/lib.rs index f58a1b4dd..4492b33bb 100644 --- a/arroyo-sql/src/lib.rs +++ b/arroyo-sql/src/lib.rs @@ -12,6 +12,7 @@ pub(crate) mod code_gen; pub mod expressions; pub mod external; pub mod json_schema; +pub mod avro; mod operators; mod optimizations; mod pipeline; From 63bad7e3f02ffa7221222106c19ff768276df691 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Sun, 22 Oct 2023 14:34:14 -0700 Subject: [PATCH 06/12] Code generation for avro --- arroyo-api/src/connection_tables.rs | 3 +-- .../routes/connections/ConnectionTester.tsx | 4 +-- arroyo-sql/src/avro.rs | 27 ++++++++++++++++--- arroyo-sql/src/tables.rs | 6 ++--- 4 files changed, 30 insertions(+), 10 deletions(-) diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index 3880faad2..545aba61c 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -260,7 +260,7 @@ pub async fn create_connection_table( .unwrap() .to_string(); - if let Some(schema) = &req.schema { + if let Some(schema) = &schema { if schema.definition.is_none() { return Err(required_field("schema.definition")); } @@ -436,7 +436,6 @@ pub(crate) async fn expand_schema( Format::Parquet(_) => Ok(schema), Format::RawString(_) => Ok(schema), } - } async fn expand_avro_schema(name: &str, connector: &str, mut schema: ConnectionSchema, table_config: &Value, profile_config: &Value) -> Result { diff --git a/arroyo-console/src/routes/connections/ConnectionTester.tsx b/arroyo-console/src/routes/connections/ConnectionTester.tsx index 1657ddc66..e9c7782ba 100644 --- a/arroyo-console/src/routes/connections/ConnectionTester.tsx +++ b/arroyo-console/src/routes/connections/ConnectionTester.tsx @@ -78,7 +78,6 @@ export function ConnectionTester({ const onClickTest = async () => { setTesting(true); setError(null); - console.log('config', state); if (!testing) { await useConnectionTableTest(sseHandler, createRequest); @@ -89,9 +88,10 @@ export function ConnectionTester({ const submit = async () => { setError(null); const { error } = await post('/v1/connection_tables', { body: createRequest }); - navigate('/connections'); if (error) { setError({ title: 'Failed to create connection', body: formatError(error) }); + } else { + navigate('/connections'); } }; diff --git a/arroyo-sql/src/avro.rs b/arroyo-sql/src/avro.rs index ea0cb5e6b..856109db3 100644 --- a/arroyo-sql/src/avro.rs +++ b/arroyo-sql/src/avro.rs @@ -1,11 +1,13 @@ use std::sync::Arc; use anyhow::{anyhow, bail}; use apache_avro::Schema; -use apache_avro::schema::UnionSchema; -use arrow_schema::{DataType, Field, FieldRef, Fields}; -use tracing::warn; +use arrow_schema::{DataType}; +use proc_macro2::Ident; +use quote::quote; use crate::types::{StructDef, StructField, TypeDef}; +pub const ROOT_NAME: &str = "ArroyoAvroRoot"; + pub fn convert_avro_schema(name: &str, schema: &str) -> anyhow::Result> { let schema = Schema::parse_str(schema) .map_err(|e| anyhow!("avro schema is not valid: {:?}", e))?; @@ -21,6 +23,25 @@ pub fn convert_avro_schema(name: &str, schema: &str) -> anyhow::Result anyhow::Result { + let fields = convert_avro_schema(name, schema)?; + + let sd = StructDef::new(Some(ROOT_NAME.to_string()), true, fields, None); + let defs: Vec<_> = sd.all_structs_including_named().iter() + .map(|p| vec![syn::parse_str(&p.def(false)).unwrap(), p.generate_serializer_items()]) + .flatten() + .collect(); + + let mod_ident: Ident = syn::parse_str(name).unwrap(); + Ok(quote! { + mod #mod_ident { + use crate::*; + #(#defs) + * + } + }.to_string()) +} + fn to_typedef(source_name: &str, schema: &Schema) -> (TypeDef, Option) { match schema { Schema::Null => { diff --git a/arroyo-sql/src/tables.rs b/arroyo-sql/src/tables.rs index 2273a505d..b481419ed 100644 --- a/arroyo-sql/src/tables.rs +++ b/arroyo-sql/src/tables.rs @@ -24,7 +24,7 @@ use datafusion_expr::{ use crate::code_gen::{CodeGenerator, ValuePointerContext}; use crate::expressions::CastExpression; use crate::external::SinkUpdateType; -use crate::DEFAULT_IDLE_TIME; +use crate::{avro, DEFAULT_IDLE_TIME}; use crate::{ expressions::{Column, ColumnExpression, Expression, ExpressionContext}, external::{ProcessingMode, SqlSink, SqlSource}, @@ -89,7 +89,7 @@ fn schema_type(name: &str, schema: &ConnectionSchema) -> Option { Some(format!("{}::{}", name, json_schema::ROOT_NAME)) } SchemaDefinition::ProtobufSchema(_) => todo!(), - SchemaDefinition::AvroSchema(_) => todo!(), + SchemaDefinition::AvroSchema(_) => Some(format!("{}::{}", name, avro::ROOT_NAME)), SchemaDefinition::RawSchema(_) => Some("arroyo_types::RawJson".to_string()), } }) @@ -101,7 +101,7 @@ pub fn schema_defs(name: &str, schema: &ConnectionSchema) -> Option { match def { SchemaDefinition::JsonSchema(s) => Some(json_schema::get_defs(&name, &s).unwrap()), SchemaDefinition::ProtobufSchema(_) => todo!(), - SchemaDefinition::AvroSchema(_) => todo!(), + SchemaDefinition::AvroSchema(s) => Some(avro::get_defs(&name, &s).unwrap()), SchemaDefinition::RawSchema(_) => None, } } From d2d269844c494ace21b6c30d550b6d835412d868 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Wed, 25 Oct 2023 19:47:22 -0700 Subject: [PATCH 07/12] work --- Cargo.lock | 5 +- arroyo-api/src/connection_tables.rs | 123 +++++++++++------- arroyo-api/src/lib.rs | 4 +- arroyo-api/src/rest.rs | 4 +- arroyo-rpc/Cargo.toml | 3 +- arroyo-rpc/src/formats.rs | 10 +- arroyo-rpc/src/lib.rs | 3 +- arroyo-rpc/src/schema_resolver.rs | 90 +++++++++----- arroyo-sql/src/avro.rs | 124 +++++++++---------- arroyo-sql/src/lib.rs | 2 +- arroyo-worker/src/connectors/polling_http.rs | 35 +++--- arroyo-worker/src/formats.rs | 117 ++++++++++++++--- 12 files changed, 338 insertions(+), 182 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 30760e7fc..391e93b42 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -660,6 +660,7 @@ version = "0.7.0" dependencies = [ "anyhow", "arroyo-types", + "async-trait", "bincode 2.0.0-rc.3", "log", "nanoid", @@ -1085,9 +1086,9 @@ checksum = "ecc7ab41815b3c653ccd2978ec3255c81349336702dfdf62ee6f7069b12a3aae" [[package]] name = "async-trait" -version = "0.1.73" +version = "0.1.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index 545aba61c..b0b6c47e6 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -16,16 +16,17 @@ use tokio::sync::mpsc::channel; use tokio_stream::wrappers::ReceiverStream; use tracing::warn; -use arroyo_connectors::{Connector, connector_for_type, ErasedConnector}; use arroyo_connectors::kafka::{KafkaConfig, KafkaConnector, KafkaTable}; +use arroyo_connectors::{connector_for_type, Connector, ErasedConnector}; use arroyo_rpc::api_types::connections::{ - ConnectionProfile, ConnectionSchema, - ConnectionTable, ConnectionTablePost, SchemaDefinition, + ConnectionProfile, ConnectionSchema, ConnectionTable, ConnectionTablePost, SchemaDefinition, }; use arroyo_rpc::api_types::{ConnectionTableCollection, PaginationQueryParams}; use arroyo_rpc::formats::{AvroFormat, Format, JsonFormat}; use arroyo_rpc::public_ids::{generate_id, IdTypes}; -use arroyo_rpc::schema_resolver::{ConfluentSchemaResolver, ConfluentSchemaResponse, ConfluentSchemaType}; +use arroyo_rpc::schema_resolver::{ + ConfluentSchemaResolver, ConfluentSchemaResponse, ConfluentSchemaType, +}; use arroyo_sql::avro; use arroyo_sql::json_schema::convert_json_schema; use arroyo_sql::types::{StructField, TypeDef}; @@ -431,48 +432,75 @@ pub(crate) async fn expand_schema( }; match format { - Format::Json(_) => expand_json_schema(name, connector, schema, table_config, profile_config).await, - Format::Avro(_) => expand_avro_schema(name, connector, schema, table_config, profile_config).await, + Format::Json(_) => { + expand_json_schema(name, connector, schema, table_config, profile_config).await + } + Format::Avro(_) => { + expand_avro_schema(name, connector, schema, table_config, profile_config).await + } Format::Parquet(_) => Ok(schema), Format::RawString(_) => Ok(schema), } } -async fn expand_avro_schema(name: &str, connector: &str, mut schema: ConnectionSchema, table_config: &Value, profile_config: &Value) -> Result { - if let Some(Format::Avro(AvroFormat { confluent_schema_registry: true, .. })) = &schema.format { +async fn expand_avro_schema( + name: &str, + connector: &str, + mut schema: ConnectionSchema, + table_config: &Value, + profile_config: &Value, +) -> Result { + if let Some(Format::Avro(AvroFormat { + confluent_schema_registry: true, + .. + })) = &schema.format + { let schema_response = get_schema(connector, table_config, profile_config).await?; if schema_response.schema_type != ConfluentSchemaType::Avro { - return Err(bad_request(format!("Format configured is avro, but confluent schema repository returned a {:?} schema", - schema_response.schema_type))); + return Err(bad_request(format!( + "Format configured is avro, but confluent schema repository returned a {:?} schema", + schema_response.schema_type + ))); } schema.definition = Some(SchemaDefinition::AvroSchema(schema_response.schema)); } - let Some(SchemaDefinition::AvroSchema(definition)) = schema.definition.as_ref() else { return Err(bad_request("avro format requires an avro schema be set")); }; let fields: Result<_, String> = avro::convert_avro_schema(&name, &definition) .map_err(|e| bad_request(format!("Invalid avro schema: {}", e)))? - .into_iter().map(|f| f.try_into()).collect(); - + .into_iter() + .map(|f| f.try_into()) + .collect(); - schema.fields = fields - .map_err(|e| bad_request(format!("Failed to convert schema: {}", e)))?; + schema.fields = fields.map_err(|e| bad_request(format!("Failed to convert schema: {}", e)))?; Ok(schema) } -async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionSchema, table_config: &Value, profile_config: &Value) -> Result { - if let Some(Format::Json(JsonFormat { confluent_schema_registry: true, .. })) = &schema.format { +async fn expand_json_schema( + name: &str, + connector: &str, + mut schema: ConnectionSchema, + table_config: &Value, + profile_config: &Value, +) -> Result { + if let Some(Format::Json(JsonFormat { + confluent_schema_registry: true, + .. + })) = &schema.format + { let schema_response = get_schema(connector, table_config, profile_config).await?; if schema_response.schema_type != ConfluentSchemaType::Json { - return Err(bad_request(format!("Format configured is json, but confluent schema repository returned a {:?} schema", - schema_response.schema_type))); + return Err(bad_request(format!( + "Format configured is json, but confluent schema repository returned a {:?} schema", + schema_response.schema_type + ))); } schema.definition = Some(SchemaDefinition::JsonSchema(schema_response.schema)); @@ -487,11 +515,7 @@ async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionS None, TypeDef::DataType(DataType::Utf8, false), )], - _ => { - return Err(bad_request( - "Invalid schema type for json format", - )) - } + _ => return Err(bad_request("Invalid schema type for json format")), }; let fields: Result<_, String> = fields.into_iter().map(|f| f.try_into()).collect(); @@ -501,29 +525,44 @@ async fn expand_json_schema(name: &str, connector: &str, mut schema: ConnectionS } Ok(schema) - } -async fn get_schema(connector: &str, table_config: &Value, profile_config: &Value) -> Result { +async fn get_schema( + connector: &str, + table_config: &Value, + profile_config: &Value, +) -> Result { if connector != "kafka" { - return Err(bad_request("confluent schema registry can only be used for Kafka connections")); + return Err(bad_request( + "confluent schema registry can only be used for Kafka connections", + )); } // we unwrap here because this should already have been validated - let profile: KafkaConfig = serde_json::from_value(profile_config.clone()) - .expect("invalid kafka config"); - - let table: KafkaTable = serde_json::from_value(table_config.clone()) - .expect("invalid kafka table"); - - let schema_registry = profile.schema_registry.as_ref().ok_or_else(|| - bad_request("schema registry must be configured on the Kafka connection profile"))?; - - let resolver = ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic) - .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e)))?; - - resolver.get_schema(None).await - .map_err(|e| bad_request(format!("failed to fetch schemas from schema repository: {}", e))) + let profile: KafkaConfig = + serde_json::from_value(profile_config.clone()).expect("invalid kafka config"); + + let table: KafkaTable = + serde_json::from_value(table_config.clone()).expect("invalid kafka table"); + + let schema_registry = profile.schema_registry.as_ref().ok_or_else(|| { + bad_request("schema registry must be configured on the Kafka connection profile") + })?; + + let resolver = + ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic).map_err(|e| { + bad_request(format!( + "failed to fetch schemas from schema repository: {}", + e + )) + })?; + + resolver.get_schema(None).await.map_err(|e| { + bad_request(format!( + "failed to fetch schemas from schema repository: {}", + e + )) + }) } /// Test a Connection Schema @@ -556,4 +595,4 @@ pub(crate) async fn test_schema( Ok(()) } } -} \ No newline at end of file +} diff --git a/arroyo-api/src/lib.rs b/arroyo-api/src/lib.rs index c54ceb395..43d9f4312 100644 --- a/arroyo-api/src/lib.rs +++ b/arroyo-api/src/lib.rs @@ -8,8 +8,8 @@ use crate::connection_profiles::{ __path_create_connection_profile, __path_get_connection_profiles, }; use crate::connection_tables::{ - __path_create_connection_table, __path_delete_connection_table, - __path_get_connection_tables, __path_test_connection_table, __path_test_schema, + __path_create_connection_table, __path_delete_connection_table, __path_get_connection_tables, + __path_test_connection_table, __path_test_schema, }; use crate::connectors::__path_get_connectors; use crate::jobs::{ diff --git a/arroyo-api/src/rest.rs b/arroyo-api/src/rest.rs index bb713012a..8da8d0741 100644 --- a/arroyo-api/src/rest.rs +++ b/arroyo-api/src/rest.rs @@ -19,8 +19,8 @@ use utoipa_swagger_ui::SwaggerUi; use crate::connection_profiles::{create_connection_profile, get_connection_profiles}; use crate::connection_tables::{ - create_connection_table, delete_connection_table, get_connection_tables, - test_connection_table, test_schema, + create_connection_table, delete_connection_table, get_connection_tables, test_connection_table, + test_schema, }; use crate::connectors::get_connectors; use crate::jobs::{ diff --git a/arroyo-rpc/Cargo.toml b/arroyo-rpc/Cargo.toml index 1e3eedd15..475cc714d 100644 --- a/arroyo-rpc/Cargo.toml +++ b/arroyo-rpc/Cargo.toml @@ -17,9 +17,10 @@ serde_json = "1.0" nanoid = "0.4" utoipa = "3" anyhow = "1.0.75" -reqwest = "0.11.22" +reqwest = { version = "0.11.22", features = ["default", "serde_json", "json"] } log = "0.4.20" tracing = "0.1.40" +async-trait = "0.1.74" [build-dependencies] tonic-build = { workspace = true } diff --git a/arroyo-rpc/src/formats.rs b/arroyo-rpc/src/formats.rs index 94b0ac451..a74b18f19 100644 --- a/arroyo-rpc/src/formats.rs +++ b/arroyo-rpc/src/formats.rs @@ -1,10 +1,10 @@ +use arroyo_types::UserError; use serde::{Deserialize, Serialize}; +use serde_json::Value; use std::collections::HashMap; use std::hash::{Hash, Hasher}; use std::str::FromStr; -use serde_json::Value; use utoipa::ToSchema; -use arroyo_types::UserError; #[derive( Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Default, Hash, PartialOrd, ToSchema, @@ -102,13 +102,15 @@ pub struct ConfluentSchemaRegistryConfig { #[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash, PartialOrd, ToSchema)] #[serde(rename_all = "camelCase")] pub struct AvroFormat { - pub confluent_schema_registry: bool + pub confluent_schema_registry: bool, + pub embedded_schema: bool, } impl AvroFormat { pub fn from_opts(opts: &mut HashMap) -> Result { Ok(Self { - confluent_schema_registry: false + confluent_schema_registry: false, + embedded_schema: false, }) } } diff --git a/arroyo-rpc/src/lib.rs b/arroyo-rpc/src/lib.rs index 821e61928..5b744abc5 100644 --- a/arroyo-rpc/src/lib.rs +++ b/arroyo-rpc/src/lib.rs @@ -1,8 +1,7 @@ pub mod api_types; pub mod formats; -pub mod schema_resolver; pub mod public_ids; - +pub mod schema_resolver; use std::{fs, time::SystemTime}; diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs index 1c4e0414e..e352d5b0a 100644 --- a/arroyo-rpc/src/schema_resolver.rs +++ b/arroyo-rpc/src/schema_resolver.rs @@ -1,24 +1,30 @@ -use std::time::Duration; use anyhow::{anyhow, bail}; -use arroyo_types::UserError; +use async_trait::async_trait; use reqwest::{Client, StatusCode, Url}; -use serde_json::Value; -use tracing::warn; use serde::{Deserialize, Serialize}; +use std::time::Duration; +use tracing::warn; -pub trait SchemaResolver { - fn resolve_schema(&self, id: [u8; 4]) -> Result, UserError>; +#[async_trait] +pub trait SchemaResolver: Send { + async fn resolve_schema(&self, id: [u8; 4]) -> Result, String>; } -pub struct FailingSchemaResolver { +pub struct FailingSchemaResolver {} + +impl FailingSchemaResolver { + pub fn new() -> Self { + FailingSchemaResolver {} + } } +#[async_trait] impl SchemaResolver for FailingSchemaResolver { - fn resolve_schema(&self, id: [u8; 4]) -> Result, UserError> { - Err(UserError { - name: "Could not deserialize".to_string(), - details: format!("Schema with id {:?} not available, and no schema registry configured", id), - }) + async fn resolve_schema(&self, id: [u8; 4]) -> Result, String> { + Err(format!( + "Schema with id {:?} not available, and no schema registry configured", + id + )) } } @@ -28,7 +34,7 @@ pub enum ConfluentSchemaType { #[default] Avro, Json, - Protobuf + Protobuf, } #[derive(Clone, Debug, Serialize, Deserialize)] @@ -52,11 +58,12 @@ impl ConfluentSchemaResolver { pub fn new(endpoint: &str, topic: &str) -> anyhow::Result { let client = Client::builder() .timeout(Duration::from_secs(5)) - .build().unwrap(); + .build() + .unwrap(); - - let endpoint: Url = - format!("{}/subjects/{}-value/versions/", endpoint, topic).as_str().try_into() + let endpoint: Url = format!("{}/subjects/{}-value/versions/", endpoint, topic) + .as_str() + .try_into() .map_err(|e| anyhow!("{} is not a valid url", endpoint))?; Ok(Self { @@ -66,21 +73,30 @@ impl ConfluentSchemaResolver { }) } + pub async fn get_schema( + &self, + version: Option, + ) -> anyhow::Result { + let url = self + .endpoint + .join( + &version + .map(|v| format!("{}", v)) + .unwrap_or_else(|| "latest".to_string()), + ) + .unwrap(); - pub async fn get_schema(&self, version: Option) -> anyhow::Result { - let url = self.endpoint.join( - &version.map(|v| format!("{}", v)).unwrap_or_else(|| "latest".to_string())).unwrap(); - - let resp = reqwest::get(url.clone()).await.map_err(|e| { + let resp = self.client.get(url.clone()).send().await.map_err(|e| { warn!("Got error response from schema registry: {:?}", e); match e.status() { - Some(StatusCode::NOT_FOUND) => anyhow!( - "Could not find value schema for topic '{}'", - self.topic), + Some(StatusCode::NOT_FOUND) => { + anyhow!("Could not find value schema for topic '{}'", self.topic) + } Some(code) => anyhow!("Schema registry returned error: {}", code), None => { - warn!("Unknown error connecting to schema registry {}: {:?}", + warn!( + "Unknown error connecting to schema registry {}: {:?}", self.endpoint, e ); anyhow!( @@ -103,10 +119,22 @@ impl ConfluentSchemaResolver { } resp.json().await.map_err(|e| { - warn!("Invalid json from schema registry: {:?} for request {:?}", e, url); - anyhow!( - "Schema registry response could not be deserialized: {}", e - ) + warn!( + "Invalid json from schema registry: {:?} for request {:?}", + e, url + ); + anyhow!("Schema registry response could not be deserialized: {}", e) }) } -} \ No newline at end of file +} + +#[async_trait] +impl SchemaResolver for ConfluentSchemaResolver { + async fn resolve_schema(&self, id: [u8; 4]) -> Result, String> { + let version = u32::from_be_bytes(id); + self.get_schema(Some(version)) + .await + .map(|s| Some(s.schema)) + .map_err(|e| e.to_string()) + } +} diff --git a/arroyo-sql/src/avro.rs b/arroyo-sql/src/avro.rs index 856109db3..1bb6f178b 100644 --- a/arroyo-sql/src/avro.rs +++ b/arroyo-sql/src/avro.rs @@ -1,22 +1,20 @@ -use std::sync::Arc; +use crate::types::{StructDef, StructField, TypeDef}; use anyhow::{anyhow, bail}; use apache_avro::Schema; -use arrow_schema::{DataType}; +use arrow_schema::DataType; use proc_macro2::Ident; use quote::quote; -use crate::types::{StructDef, StructField, TypeDef}; +use std::sync::Arc; pub const ROOT_NAME: &str = "ArroyoAvroRoot"; pub fn convert_avro_schema(name: &str, schema: &str) -> anyhow::Result> { - let schema = Schema::parse_str(schema) - .map_err(|e| anyhow!("avro schema is not valid: {:?}", e))?; + let schema = + Schema::parse_str(schema).map_err(|e| anyhow!("avro schema is not valid: {:?}", e))?; let (typedef, _) = to_typedef(name, &schema); match typedef { - TypeDef::StructDef(sd, _) => { - Ok(sd.fields) - } + TypeDef::StructDef(sd, _) => Ok(sd.fields), TypeDef::DataType(_, _) => { bail!("top-level schema must be a record") } @@ -27,8 +25,15 @@ pub fn get_defs(name: &str, schema: &str) -> anyhow::Result { let fields = convert_avro_schema(name, schema)?; let sd = StructDef::new(Some(ROOT_NAME.to_string()), true, fields, None); - let defs: Vec<_> = sd.all_structs_including_named().iter() - .map(|p| vec![syn::parse_str(&p.def(false)).unwrap(), p.generate_serializer_items()]) + let defs: Vec<_> = sd + .all_structs_including_named() + .iter() + .map(|p| { + vec![ + syn::parse_str(&p.def(false)).unwrap(), + p.generate_serializer_items(), + ] + }) .flatten() .collect(); @@ -39,35 +44,20 @@ pub fn get_defs(name: &str, schema: &str) -> anyhow::Result { #(#defs) * } - }.to_string()) + } + .to_string()) } fn to_typedef(source_name: &str, schema: &Schema) -> (TypeDef, Option) { match schema { - Schema::Null => { - (TypeDef::DataType(DataType::Null, false), None) - } - Schema::Boolean => { - (TypeDef::DataType(DataType::Boolean, false), None) - } - Schema::Int => { - (TypeDef::DataType(DataType::Int32, false), None) - } - Schema::Long => { - (TypeDef::DataType(DataType::Int64, false), None) - } - Schema::Float => { - (TypeDef::DataType(DataType::Float32, false), None) - } - Schema::Double => { - (TypeDef::DataType(DataType::Float64, false), None) - } - Schema::Bytes => { - (TypeDef::DataType(DataType::Binary, false), None) - } - Schema::String => { - (TypeDef::DataType(DataType::Utf8, false), None) - } + Schema::Null => (TypeDef::DataType(DataType::Null, false), None), + Schema::Boolean => (TypeDef::DataType(DataType::Boolean, false), None), + Schema::Int => (TypeDef::DataType(DataType::Int32, false), None), + Schema::Long => (TypeDef::DataType(DataType::Int64, false), None), + Schema::Float => (TypeDef::DataType(DataType::Float32, false), None), + Schema::Double => (TypeDef::DataType(DataType::Float64, false), None), + Schema::Bytes => (TypeDef::DataType(DataType::Binary, false), None), + Schema::String => (TypeDef::DataType(DataType::Utf8, false), None), // Schema::Array(t) => { // let dt = match to_typedef(source_name, t) { // (TypeDef::StructDef(sd, _), _) => { @@ -88,7 +78,10 @@ fn to_typedef(source_name: &str, schema: &Schema) -> (TypeDef, Option) { // currently just support unions that have [t, null] as variants, which is the // avro way to represent optional fields - let (nulls, not_nulls): (Vec<_>, Vec<_>) = union.variants().iter().partition(|v| matches!(v, Schema::Null)); + let (nulls, not_nulls): (Vec<_>, Vec<_>) = union + .variants() + .iter() + .partition(|v| matches!(v, Schema::Null)); if nulls.len() == 1 && not_nulls.len() == 1 { let (dt, original) = to_typedef(source_name, not_nulls[0]); @@ -101,36 +94,41 @@ fn to_typedef(source_name: &str, schema: &Schema) -> (TypeDef, Option) { } } Schema::Record(record) => { - let fields = record.fields.iter().map(|f| { - let (ft, original) = to_typedef(source_name, &f.schema); - StructField::with_rename(f.name.clone(), None, ft, None, original) - }).collect(); + let fields = record + .fields + .iter() + .map(|f| { + let (ft, original) = to_typedef(source_name, &f.schema); + StructField::with_rename(f.name.clone(), None, ft, None, original) + }) + .collect(); ( - TypeDef::StructDef(StructDef::for_name(Some(format!("{}::{}", source_name, record.name.name)), fields), false), - None + TypeDef::StructDef( + StructDef::for_name( + Some(format!("{}::{}", source_name, record.name.name)), + fields, + ), + false, + ), + None, ) } - _ => { - ( - TypeDef::DataType(DataType::Utf8, false), - Some("json".to_string()), - ) - } - // Schema::Enum(_) => {} - // Schema::Fixed(_) => {} - // Schema::Decimal(_) => {} - // Schema::Uuid => {} - // Schema::Date => {} - // Schema::TimeMillis => {} - // Schema::TimeMicros => {} - // Schema::TimestampMillis => {} - // Schema::TimestampMicros => {} - // Schema::LocalTimestampMillis => {} - // Schema::LocalTimestampMicros => {} - // Schema::Duration => {} - // Schema::Ref { .. } => {} + _ => ( + TypeDef::DataType(DataType::Utf8, false), + Some("json".to_string()), + ), // Schema::Enum(_) => {} + // Schema::Fixed(_) => {} + // Schema::Decimal(_) => {} + // Schema::Uuid => {} + // Schema::Date => {} + // Schema::TimeMillis => {} + // Schema::TimeMicros => {} + // Schema::TimestampMillis => {} + // Schema::TimestampMicros => {} + // Schema::LocalTimestampMillis => {} + // Schema::LocalTimestampMicros => {} + // Schema::Duration => {} + // Schema::Ref { .. } => {} } } - - diff --git a/arroyo-sql/src/lib.rs b/arroyo-sql/src/lib.rs index 4492b33bb..488cebf15 100644 --- a/arroyo-sql/src/lib.rs +++ b/arroyo-sql/src/lib.rs @@ -8,11 +8,11 @@ use arroyo_connectors::{Connection, Connector}; use arroyo_datastream::Program; use datafusion::physical_plan::functions::make_scalar_function; +pub mod avro; pub(crate) mod code_gen; pub mod expressions; pub mod external; pub mod json_schema; -pub mod avro; mod operators; mod optimizations; mod pipeline; diff --git a/arroyo-worker/src/connectors/polling_http.rs b/arroyo-worker/src/connectors/polling_http.rs index 591f883b3..0d3fcb1d2 100644 --- a/arroyo-worker/src/connectors/polling_http.rs +++ b/arroyo-worker/src/connectors/polling_http.rs @@ -259,24 +259,29 @@ where } } - let iter = self.deserializer.deserialize_slice(&buf); - - for record in iter { - match record { - Ok(value) => { - ctx.collect(Record { - timestamp: SystemTime::now(), - key: None, - value, - }).await; - } - Err(e) => { - ctx.report_user_error(e).await; + match self.deserializer.deserialize_slice(&buf).await { + Ok(iter) => { + for record in iter { + match record { + Ok(value) => { + ctx.collect(Record { + timestamp: SystemTime::now(), + key: None, + value, + }).await; + } + Err(e) => { + ctx.report_user_error(e).await; + } + } } + + self.state.last_message = Some(buf); + } + Err(e) => { + ctx.report_user_error(e).await; } } - - self.state.last_message = Some(buf); } Err(e) => { ctx.report_user_error(e).await; diff --git a/arroyo-worker/src/formats.rs b/arroyo-worker/src/formats.rs index 443d241e8..0e82c8d35 100644 --- a/arroyo-worker/src/formats.rs +++ b/arroyo-worker/src/formats.rs @@ -1,13 +1,16 @@ +use apache_avro::{Reader, Schema}; use std::sync::Arc; use std::{collections::HashMap, marker::PhantomData}; -use apache_avro::Schema; use arrow::datatypes::{Field, Fields}; use arroyo_rpc::formats::{AvroFormat, Format, Framing, FramingMethod, JsonFormat}; +use arroyo_rpc::schema_resolver::{FailingSchemaResolver, SchemaResolver}; use arroyo_types::UserError; use serde::de::DeserializeOwned; use serde_json::{json, Value}; use tokio::sync::Mutex; +use tracing::log::kv::Source; +use tracing::warn; use crate::SchemaData; @@ -56,12 +59,74 @@ fn deserialize_raw_string(msg: &[u8]) -> Result Ok(serde_json::from_value(json).unwrap()) } -fn deserialize_slice_avro( +async fn deserialize_slice_avro<'a, T: DeserializeOwned>( format: &AvroFormat, schema_registry: Arc>>, - msg: &[u8], -) { + resolver: Arc, + mut msg: &'a [u8], +) -> Result> + 'a, String> { + let id = if format.confluent_schema_registry { + let magic_byte = msg[0]; + if magic_byte != 0 { + return Err(format!("data was not encoded with schema registry wire format; magic byte has unexpected value: {}", magic_byte)); + } + + let id = [msg[1], msg[2], msg[3], msg[4]]; + msg = &msg[5..]; + id + } else { + [0, 0, 0, 0] + }; + + let mut registry = schema_registry.lock().await; + + let mut reader = if format.embedded_schema { + Reader::new(&msg[..]).map_err(|e| format!("invalid Avro schema in message: {:?}", e))? + } else { + let schema = if registry.contains_key(&id) { + registry.get(&id).unwrap() + } else { + let new_schema = resolver.resolve_schema(id).await?.ok_or_else(|| { + format!( + "could not resolve schema for message with id {}", + u32::from_le_bytes(id) + ) + })?; + + let new_schema = Schema::parse_str(&new_schema) + .map_err(|e| format!("invalid avro schema: {:?}", e))?; + + registry.insert(id, new_schema); + + registry.get(&id).unwrap() + }; + + Reader::with_schema(schema, &msg[..]) + .map_err(|e| format!("invalid avro schema: {:?}", e))? + }; + + let messages: Vec<_> = reader.collect(); + Ok(messages.into_iter().map(|record| { + apache_avro::from_value::(&record.map_err(|e| { + UserError::new( + "Deserialization failed", + format!( + "Failed to deserialize from avro: {:?}", + e + ), + ) + })?) + .map_err(|e| { + UserError::new( + "Deserialization failed", + format!("Failed to convert avro message into struct type: {:?}", e), + ) + }) + })) + // let record = reader.next() + // .ok_or_else(|| "avro record did not contain any messages")? + // .map_err(|e| e.to_string())?; } pub struct FramingIterator<'a> { @@ -120,38 +185,56 @@ pub struct DataDeserializer { format: Arc, framing: Option>, schema_registry: Arc>>, - schema_resolver: Arc>, + schema_resolver: Arc, _t: PhantomData, } - impl DataDeserializer { - pub fn new(format: Format, framing: Option,) -> Self { - if let Format::Avro(avro) = &format { - - }; + pub fn new(format: Format, framing: Option) -> Self { + Self::with_schema_resolver(format, framing, Arc::new(FailingSchemaResolver::new())) + } + pub fn with_schema_resolver( + format: Format, + framing: Option, + schema_resolver: Arc, + ) -> Self { Self { format: Arc::new(format), framing: framing.map(|f| Arc::new(f)), + schema_registry: Arc::new(Mutex::new(HashMap::new())), + schema_resolver, _t: PhantomData, } } - pub fn deserialize_slice<'a>( + pub async fn deserialize_slice<'a>( &self, msg: &'a [u8], - ) -> impl Iterator> + 'a { - let new_self = self.clone(); - FramingIterator::new(self.framing.clone(), msg) - .map(move |t| new_self.deserialize_single(t)) + ) -> Result> + 'a, UserError> { + match &*self.format { + Format::Avro(avro) => { + deserialize_slice_avro( + avro, + self.schema_registry.clone(), + self.schema_resolver.clone(), + msg, + ) + .await + } + _ => { + let new_self = self.clone(); + Ok(FramingIterator::new(self.framing.clone(), msg) + .map(move |t| new_self.deserialize_single(t))) + } + } } fn deserialize_single(&self, msg: &[u8]) -> Result { match &*self.format { Format::Json(json) => deserialize_slice_json(json, msg), - Format::Avro(avro) => deserialie_slice_avro(), - Format::Parquet(_) => todo!(), + Format::Avro(avro) => unreachable!("avro should be handled by here"), + Format::Parquet(_) => todo!("parquet is not supported as an input format"), Format::RawString(_) => deserialize_raw_string(msg), } .map_err(|e| { From 4ad578383a3a95bfb7eed3700285a9608f102a00 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Fri, 27 Oct 2023 15:03:20 -0700 Subject: [PATCH 08/12] checkpoint --- arroyo-console/src/gen/api-types.ts | 1 + arroyo-rpc/src/formats.rs | 3 + arroyo-rpc/src/schema_resolver.rs | 11 +- arroyo-sql/src/types.rs | 8 +- arroyo-types/src/lib.rs | 1 + arroyo-worker/src/connectors/fluvio/source.rs | 2 +- .../src/connectors/kafka/source/mod.rs | 14 +- .../src/connectors/kinesis/source/mod.rs | 2 +- arroyo-worker/src/connectors/polling_http.rs | 35 +- arroyo-worker/src/connectors/sse.rs | 2 +- arroyo-worker/src/connectors/websocket.rs | 2 +- arroyo-worker/src/formats/avro.rs | 268 ++++++++++++ .../src/{formats.rs => formats/json.rs} | 393 ++---------------- arroyo-worker/src/formats/mod.rs | 280 +++++++++++++ 14 files changed, 625 insertions(+), 397 deletions(-) create mode 100644 arroyo-worker/src/formats/avro.rs rename arroyo-worker/src/{formats.rs => formats/json.rs} (51%) create mode 100644 arroyo-worker/src/formats/mod.rs diff --git a/arroyo-console/src/gen/api-types.ts b/arroyo-console/src/gen/api-types.ts index 71239a628..9dfd605ae 100644 --- a/arroyo-console/src/gen/api-types.ts +++ b/arroyo-console/src/gen/api-types.ts @@ -178,6 +178,7 @@ export interface components { schemas: { AvroFormat: { confluentSchemaRegistry: boolean; + embeddedSchema: boolean; }; Checkpoint: { backend: string; diff --git a/arroyo-rpc/src/formats.rs b/arroyo-rpc/src/formats.rs index a74b18f19..412e4a928 100644 --- a/arroyo-rpc/src/formats.rs +++ b/arroyo-rpc/src/formats.rs @@ -102,7 +102,10 @@ pub struct ConfluentSchemaRegistryConfig { #[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash, PartialOrd, ToSchema)] #[serde(rename_all = "camelCase")] pub struct AvroFormat { + #[serde(default)] pub confluent_schema_registry: bool, + + #[serde(default)] pub embedded_schema: bool, } diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs index e352d5b0a..8be98ff36 100644 --- a/arroyo-rpc/src/schema_resolver.rs +++ b/arroyo-rpc/src/schema_resolver.rs @@ -7,7 +7,7 @@ use tracing::warn; #[async_trait] pub trait SchemaResolver: Send { - async fn resolve_schema(&self, id: [u8; 4]) -> Result, String>; + async fn resolve_schema(&self, id: u32) -> Result, String>; } pub struct FailingSchemaResolver {} @@ -20,9 +20,9 @@ impl FailingSchemaResolver { #[async_trait] impl SchemaResolver for FailingSchemaResolver { - async fn resolve_schema(&self, id: [u8; 4]) -> Result, String> { + async fn resolve_schema(&self, id: u32) -> Result, String> { Err(format!( - "Schema with id {:?} not available, and no schema registry configured", + "Schema with id {} not available, and no schema registry configured", id )) } @@ -130,9 +130,8 @@ impl ConfluentSchemaResolver { #[async_trait] impl SchemaResolver for ConfluentSchemaResolver { - async fn resolve_schema(&self, id: [u8; 4]) -> Result, String> { - let version = u32::from_be_bytes(id); - self.get_schema(Some(version)) + async fn resolve_schema(&self, id: u32) -> Result, String> { + self.get_schema(Some(id)) .await .map(|s| Some(s.schema)) .map_err(|e| e.to_string()) diff --git a/arroyo-sql/src/types.rs b/arroyo-sql/src/types.rs index 9f29a8f99..058c949a6 100644 --- a/arroyo-sql/src/types.rs +++ b/arroyo-sql/src/types.rs @@ -727,11 +727,11 @@ impl StructField { if nullable { attributes.push(quote! { #[serde(default)] - #[serde(with = "arroyo_worker::formats::opt_timestamp_as_millis")] + #[serde(with = "arroyo_worker::formats::json::opt_timestamp_as_millis")] }); } else { attributes.push(quote! { - #[serde(with = "arroyo_worker::formats::timestamp_as_millis")] + #[serde(with = "arroyo_worker::formats::json::timestamp_as_millis")] }); } } @@ -739,11 +739,11 @@ impl StructField { if nullable { attributes.push(quote! { #[serde(default)] - #[serde(with = "arroyo_worker::formats::opt_timestamp_as_rfc3339")] + #[serde(with = "arroyo_worker::formats::json::opt_timestamp_as_rfc3339")] }); } else { attributes.push(quote!( - #[serde(with = "arroyo_worker::formats::timestamp_as_rfc3339")] + #[serde(with = "arroyo_worker::formats::json::timestamp_as_rfc3339")] )); } } diff --git a/arroyo-types/src/lib.rs b/arroyo-types/src/lib.rs index 3ce33feb3..88a295b90 100644 --- a/arroyo-types/src/lib.rs +++ b/arroyo-types/src/lib.rs @@ -298,6 +298,7 @@ pub struct Record { pub value: T, } +#[derive(Debug, Clone)] pub struct UserError { pub name: String, pub details: String, diff --git a/arroyo-worker/src/connectors/fluvio/source.rs b/arroyo-worker/src/connectors/fluvio/source.rs index d88e84d76..2bf7f80c2 100644 --- a/arroyo-worker/src/connectors/fluvio/source.rs +++ b/arroyo-worker/src/connectors/fluvio/source.rs @@ -200,7 +200,7 @@ where match message { Some((_, Ok(msg))) => { let timestamp = from_millis(msg.timestamp().max(0) as u64); - let iter = self.deserializer.deserialize_slice(msg.value()); + let iter = self.deserializer.deserialize_slice(msg.value()).await; for value in iter { ctx.collector.collect(Record { timestamp, diff --git a/arroyo-worker/src/connectors/kafka/source/mod.rs b/arroyo-worker/src/connectors/kafka/source/mod.rs index 97481753a..d77a0ab02 100644 --- a/arroyo-worker/src/connectors/kafka/source/mod.rs +++ b/arroyo-worker/src/connectors/kafka/source/mod.rs @@ -17,9 +17,11 @@ use serde::de::DeserializeOwned; use std::collections::HashMap; use std::marker::PhantomData; use std::num::NonZeroU32; +use std::sync::Arc; use std::time::Duration; use tokio::select; use tracing::{debug, error, info, warn}; +use arroyo_rpc::schema_resolver::{ConfluentSchemaResolver, FailingSchemaResolver, SchemaResolver}; use super::{client_configs, KafkaConfig, KafkaTable, ReadMode, TableType}; @@ -103,14 +105,22 @@ where client_configs.insert("isolation.level".to_string(), "read_committed".to_string()); } + let schema_resolver: Arc = if let Some(schema_registry) = &connection.schema_registry { + Arc::new(ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic) + .expect("failed to construct confluent schema resolver")) + } else { + Arc::new(FailingSchemaResolver::new()) + }; + Self { topic: table.topic, bootstrap_servers: connection.bootstrap_servers.to_string(), group_id: group_id.clone(), offset_mode: *offset, - deserializer: DataDeserializer::new( + deserializer: DataDeserializer::with_schema_resolver( config.format.expect("Format must be set for Kafka source"), config.framing, + schema_resolver, ), client_configs, messages_per_second: NonZeroU32::new( @@ -247,7 +257,7 @@ where .ok_or_else(|| UserError::new("Failed to read timestamp from Kafka record", "The message read from Kafka did not contain a message timestamp"))?; - let iter = self.deserializer.deserialize_slice(v); + let iter = self.deserializer.deserialize_slice(v).await; for value in iter { ctx.collector.collect(Record { diff --git a/arroyo-worker/src/connectors/kinesis/source/mod.rs b/arroyo-worker/src/connectors/kinesis/source/mod.rs index f6af5118b..baabc9172 100644 --- a/arroyo-worker/src/connectors/kinesis/source/mod.rs +++ b/arroyo-worker/src/connectors/kinesis/source/mod.rs @@ -476,7 +476,7 @@ impl KinesisSourceFunc { let data = record.data.unwrap().into_inner(); let timestamp = record.approximate_arrival_timestamp.unwrap(); - let iter = self.deserializer.deserialize_slice(&data); + let iter = self.deserializer.deserialize_slice(&data).await; for value in iter { let output_record = Record { timestamp: from_nanos(timestamp.as_nanos() as u128), diff --git a/arroyo-worker/src/connectors/polling_http.rs b/arroyo-worker/src/connectors/polling_http.rs index 0d3fcb1d2..e8acda6fa 100644 --- a/arroyo-worker/src/connectors/polling_http.rs +++ b/arroyo-worker/src/connectors/polling_http.rs @@ -259,29 +259,24 @@ where } } - match self.deserializer.deserialize_slice(&buf).await { - Ok(iter) => { - for record in iter { - match record { - Ok(value) => { - ctx.collect(Record { - timestamp: SystemTime::now(), - key: None, - value, - }).await; - } - Err(e) => { - ctx.report_user_error(e).await; - } - } + let iter = self.deserializer.deserialize_slice(&buf).await; + + for record in iter { + match record { + Ok(value) => { + ctx.collect(Record { + timestamp: SystemTime::now(), + key: None, + value, + }).await; + } + Err(e) => { + ctx.report_user_error(e).await; } - - self.state.last_message = Some(buf); - } - Err(e) => { - ctx.report_user_error(e).await; } } + + self.state.last_message = Some(buf); } Err(e) => { ctx.report_user_error(e).await; diff --git a/arroyo-worker/src/connectors/sse.rs b/arroyo-worker/src/connectors/sse.rs index 8099ac29e..1e209d05f 100644 --- a/arroyo-worker/src/connectors/sse.rs +++ b/arroyo-worker/src/connectors/sse.rs @@ -178,7 +178,7 @@ where } if events.is_empty() || events.contains(&event.event_type) { - let iter = self.deserializer.deserialize_slice(&event.data.as_bytes()); + let iter = self.deserializer.deserialize_slice(&event.data.as_bytes()).await; for v in iter { match v { diff --git a/arroyo-worker/src/connectors/websocket.rs b/arroyo-worker/src/connectors/websocket.rs index 02e30a72c..2f15b3ffa 100644 --- a/arroyo-worker/src/connectors/websocket.rs +++ b/arroyo-worker/src/connectors/websocket.rs @@ -149,7 +149,7 @@ where msg: &[u8], ctx: &mut Context<(), T>, ) -> Result<(), UserError> { - let iter = self.deserializer.deserialize_slice(msg); + let iter = self.deserializer.deserialize_slice(msg).await; for value in iter { ctx.collector .collect(Record { diff --git a/arroyo-worker/src/formats/avro.rs b/arroyo-worker/src/formats/avro.rs new file mode 100644 index 000000000..e3d4311bd --- /dev/null +++ b/arroyo-worker/src/formats/avro.rs @@ -0,0 +1,268 @@ +use std::collections::HashMap; +use std::sync::Arc; +use apache_avro::{from_avro_datum, Reader, Schema}; +use serde::de::DeserializeOwned; +use tokio::sync::Mutex; +use arroyo_rpc::formats::AvroFormat; +use arroyo_rpc::schema_resolver::SchemaResolver; +use arroyo_types::UserError; + +pub async fn deserialize_slice_avro<'a, T: DeserializeOwned>( + format: &AvroFormat, + schema_registry: Arc>>, + resolver: Arc, + mut msg: &'a [u8], +) -> Result> + 'a, String> { + let id = if format.confluent_schema_registry { + let magic_byte = msg[0]; + if magic_byte != 0 { + return Err(format!("data was not encoded with schema registry wire format; magic byte has unexpected value: {}", magic_byte)); + } + + let id = u32::from_be_bytes([msg[1], msg[2], msg[3], msg[4]]); + msg = &msg[5..]; + id + } else { + 0 + }; + + let mut registry = schema_registry.lock().await; + + let messages = if format.embedded_schema { + Reader::new(&msg[..]).map_err(|e| format!("invalid Avro schema in message: {:?}", e))? + .collect() + } else { + let schema = if registry.contains_key(&id) { + registry.get(&id).unwrap() + } else { + let new_schema = resolver.resolve_schema(id).await?.ok_or_else(|| { + format!( + "could not resolve schema for message with id {}", + id + ) + })?; + + let new_schema = Schema::parse_str(&new_schema) + .map_err(|e| format!("schema from Confluent Schema registry is not valid: {:?}", e))?; + + registry.insert(id, new_schema); + + registry.get(&id).unwrap() + }; + + if format.confluent_schema_registry { + let mut buf = &msg[..]; + vec![from_avro_datum(schema, &mut buf, None)] + } else { + let reader = Reader::with_schema(schema, &msg[..]) + .map_err(|e| format!("invalid avro data: {:?}", e))?; + + reader.collect() + } + }; + + Ok(messages.into_iter().map(|record| { + apache_avro::from_value::(&record.map_err(|e| { + UserError::new( + "Deserialization failed", + format!( + "Failed to deserialize from avro: {:?}", + e + ), + ) + })?) + .map_err(|e| { + UserError::new( + "Deserialization failed", + format!("Failed to convert avro message into struct type: {:?}", e), + ) + }) + })) + + + // let record = reader.next() + // .ok_or_else(|| "avro record did not contain any messages")? + // .map_err(|e| e.to_string())?; +} + + +#[cfg(test)] +mod tests { + use std::sync::Arc; + use async_trait::async_trait; + use arroyo_rpc::formats::{AvroFormat, Format}; + use arroyo_rpc::schema_resolver::SchemaResolver; + use crate::formats::DataDeserializer; + use crate::SchemaData; + + struct TestSchemaResolver { + schema: String, + id: u32, + } + + #[async_trait] + impl SchemaResolver for TestSchemaResolver { + async fn resolve_schema(&self, id: u32) -> Result, String> { + assert_eq!(id, self.id); + + Ok(Some(self.schema.clone())) + } + } + + #[derive( + Clone, + Debug, + bincode::Encode, + bincode::Decode, + PartialEq, + PartialOrd, + serde::Serialize, + serde::Deserialize + )] + pub struct ArroyoAvroRoot { + pub store_id: i32, + pub store_order_id: i32, + pub coupon_code: i32, + pub date: i64, + pub status: String, + #[serde(deserialize_with = "crate::deserialize_raw_json")] + pub order_lines: String, + } + + #[derive( + Clone, + Debug, + bincode::Encode, + bincode::Decode, + PartialEq, + PartialOrd, + serde::Serialize, + serde::Deserialize + )] + pub struct OrderLine { + pub product_id: i32, + pub category: String, + pub quantity: i32, + pub unit_price: f64, + pub net_price: f64, + } + + impl SchemaData for ArroyoAvroRoot { + fn name() -> &'static str { + "ArroyoAvroRoot" + } + fn schema() -> arrow::datatypes::Schema { + let fields: Vec = vec![ + arrow::datatypes::Field::new("store_id", + arrow::datatypes::DataType::Int32, false), + arrow::datatypes::Field::new("store_order_id", + arrow::datatypes::DataType::Int32, false), + arrow::datatypes::Field::new("coupon_code", + arrow::datatypes::DataType::Int32, false), + arrow::datatypes::Field::new("date", arrow::datatypes::DataType::Utf8, + false), + arrow::datatypes::Field::new("status", + arrow::datatypes::DataType::Utf8, false), + arrow::datatypes::Field::new("order_lines", + arrow::datatypes::DataType::Utf8, false), + ]; + arrow::datatypes::Schema::new(fields) + } + fn to_raw_string(&self) -> Option> { + unimplemented!("to_raw_string is not implemented for this type") + } + } + + #[tokio::test] + async fn test_avro_deserialization() { + let schema = r#" + { + "connect.name": "pizza_orders.pizza_orders", + "fields": [ + { + "name": "store_id", + "type": "int" + }, + { + "name": "store_order_id", + "type": "int" + }, + { + "name": "coupon_code", + "type": "int" + }, + { + "name": "date", + "type": { + "connect.name": "org.apache.kafka.connect.data.Date", + "connect.version": 1, + "logicalType": "date", + "type": "int" + } + }, + { + "name": "status", + "type": "string" + }, + { + "name": "order_lines", + "type": { + "items": { + "connect.name": "pizza_orders.order_line", + "fields": [ + { + "name": "product_id", + "type": "int" + }, + { + "name": "category", + "type": "string" + }, + { + "name": "quantity", + "type": "int" + }, + { + "name": "unit_price", + "type": "double" + }, + { + "name": "net_price", + "type": "double" + } + ], + "name": "order_line", + "type": "record" + }, + "type": "array" + } + } + ], + "name": "pizza_orders", + "namespace": "pizza_orders", + "type": "record" +}"#; + + let message = [0u8, 0, 0, 0, 1, 8, 200, 223, 1, 144, 31, 186, 159, 2, 16, 97, 99, 99, + 101, 112, 116, 101, 100, 4, 156, 1, 10, 112, 105, 122, 122, 97, 4, 102, 102, 102, 102, 102, + 230, 38, 64, 102, 102, 102, 102, 102, 230, 54, 64, 84, 14, 100, 101, 115, 115, 101, 114, 116, + 2, 113, 61, 10, 215, 163, 112, 26, 64, 113, 61, 10, 215, 163, 112, 26, 64, 0, 10]; + + let mut deserializer = DataDeserializer::with_schema_resolver( + Format::Avro(AvroFormat { + confluent_schema_registry: true, + embedded_schema: false, + }), + None, + Arc::new(TestSchemaResolver { + schema: schema.to_string(), + id: 1, + })); + + let v: Vec> = deserializer.deserialize_slice(&message[..]).await.collect(); + + for i in v { + println!("{:?}", i.unwrap()); + } + } +} \ No newline at end of file diff --git a/arroyo-worker/src/formats.rs b/arroyo-worker/src/formats/json.rs similarity index 51% rename from arroyo-worker/src/formats.rs rename to arroyo-worker/src/formats/json.rs index 0e82c8d35..22bcaba8a 100644 --- a/arroyo-worker/src/formats.rs +++ b/arroyo-worker/src/formats/json.rs @@ -1,20 +1,10 @@ -use apache_avro::{Reader, Schema}; -use std::sync::Arc; -use std::{collections::HashMap, marker::PhantomData}; - +use std::collections::HashMap; use arrow::datatypes::{Field, Fields}; -use arroyo_rpc::formats::{AvroFormat, Format, Framing, FramingMethod, JsonFormat}; -use arroyo_rpc::schema_resolver::{FailingSchemaResolver, SchemaResolver}; -use arroyo_types::UserError; use serde::de::DeserializeOwned; use serde_json::{json, Value}; -use tokio::sync::Mutex; -use tracing::log::kv::Source; -use tracing::warn; - -use crate::SchemaData; +use arroyo_rpc::formats::JsonFormat; -fn deserialize_slice_json( +pub fn deserialize_slice_json( format: &JsonFormat, msg: &[u8], ) -> Result { @@ -52,248 +42,6 @@ fn deserialize_slice_json( } } -fn deserialize_raw_string(msg: &[u8]) -> Result { - let json = json! { - { "value": String::from_utf8_lossy(msg) } - }; - Ok(serde_json::from_value(json).unwrap()) -} - -async fn deserialize_slice_avro<'a, T: DeserializeOwned>( - format: &AvroFormat, - schema_registry: Arc>>, - resolver: Arc, - mut msg: &'a [u8], -) -> Result> + 'a, String> { - let id = if format.confluent_schema_registry { - let magic_byte = msg[0]; - if magic_byte != 0 { - return Err(format!("data was not encoded with schema registry wire format; magic byte has unexpected value: {}", magic_byte)); - } - - let id = [msg[1], msg[2], msg[3], msg[4]]; - msg = &msg[5..]; - id - } else { - [0, 0, 0, 0] - }; - - let mut registry = schema_registry.lock().await; - - let mut reader = if format.embedded_schema { - Reader::new(&msg[..]).map_err(|e| format!("invalid Avro schema in message: {:?}", e))? - } else { - let schema = if registry.contains_key(&id) { - registry.get(&id).unwrap() - } else { - let new_schema = resolver.resolve_schema(id).await?.ok_or_else(|| { - format!( - "could not resolve schema for message with id {}", - u32::from_le_bytes(id) - ) - })?; - - let new_schema = Schema::parse_str(&new_schema) - .map_err(|e| format!("invalid avro schema: {:?}", e))?; - - registry.insert(id, new_schema); - - registry.get(&id).unwrap() - }; - - Reader::with_schema(schema, &msg[..]) - .map_err(|e| format!("invalid avro schema: {:?}", e))? - }; - - let messages: Vec<_> = reader.collect(); - Ok(messages.into_iter().map(|record| { - apache_avro::from_value::(&record.map_err(|e| { - UserError::new( - "Deserialization failed", - format!( - "Failed to deserialize from avro: {:?}", - e - ), - ) - })?) - .map_err(|e| { - UserError::new( - "Deserialization failed", - format!("Failed to convert avro message into struct type: {:?}", e), - ) - }) - })) - - // let record = reader.next() - // .ok_or_else(|| "avro record did not contain any messages")? - // .map_err(|e| e.to_string())?; -} - -pub struct FramingIterator<'a> { - framing: Option>, - buf: &'a [u8], - offset: usize, -} - -impl<'a> FramingIterator<'a> { - pub fn new(framing: Option>, buf: &'a [u8]) -> Self { - Self { - framing, - buf, - offset: 0, - } - } -} - -impl<'a> Iterator for FramingIterator<'a> { - type Item = &'a [u8]; - - fn next(&mut self) -> Option { - if self.offset >= self.buf.len() { - return None; - } - - match &self.framing { - Some(framing) => { - match &framing.method { - FramingMethod::Newline(newline) => { - let end = memchr::memchr('\n' as u8, &self.buf[self.offset..]) - .map(|i| self.offset + i) - .unwrap_or(self.buf.len()); - - let prev = self.offset; - self.offset = end + 1; - - // enforce max len if set - let length = - (end - prev).min(newline.max_line_length.unwrap_or(u64::MAX) as usize); - - Some(&self.buf[prev..(prev + length)]) - } - } - } - None => { - self.offset = self.buf.len(); - Some(self.buf) - } - } - } -} - -#[derive(Clone)] -pub struct DataDeserializer { - format: Arc, - framing: Option>, - schema_registry: Arc>>, - schema_resolver: Arc, - _t: PhantomData, -} - -impl DataDeserializer { - pub fn new(format: Format, framing: Option) -> Self { - Self::with_schema_resolver(format, framing, Arc::new(FailingSchemaResolver::new())) - } - - pub fn with_schema_resolver( - format: Format, - framing: Option, - schema_resolver: Arc, - ) -> Self { - Self { - format: Arc::new(format), - framing: framing.map(|f| Arc::new(f)), - schema_registry: Arc::new(Mutex::new(HashMap::new())), - schema_resolver, - _t: PhantomData, - } - } - - pub async fn deserialize_slice<'a>( - &self, - msg: &'a [u8], - ) -> Result> + 'a, UserError> { - match &*self.format { - Format::Avro(avro) => { - deserialize_slice_avro( - avro, - self.schema_registry.clone(), - self.schema_resolver.clone(), - msg, - ) - .await - } - _ => { - let new_self = self.clone(); - Ok(FramingIterator::new(self.framing.clone(), msg) - .map(move |t| new_self.deserialize_single(t))) - } - } - } - - fn deserialize_single(&self, msg: &[u8]) -> Result { - match &*self.format { - Format::Json(json) => deserialize_slice_json(json, msg), - Format::Avro(avro) => unreachable!("avro should be handled by here"), - Format::Parquet(_) => todo!("parquet is not supported as an input format"), - Format::RawString(_) => deserialize_raw_string(msg), - } - .map_err(|e| { - UserError::new( - "Deserialization failed", - format!( - "Failed to deserialize: '{}': {}", - String::from_utf8_lossy(&msg), - e - ), - ) - }) - } -} - -pub struct DataSerializer { - kafka_schema: Value, - #[allow(unused)] - json_schema: Value, - format: Format, - _t: PhantomData, -} - -impl DataSerializer { - pub fn new(format: Format) -> Self { - Self { - kafka_schema: arrow_to_kafka_json(T::name(), T::schema().fields()), - json_schema: arrow_to_json_schema(T::schema().fields()), - format, - _t: PhantomData, - } - } - - pub fn to_vec(&self, record: &T) -> Option> { - match &self.format { - Format::Json(json) => { - let v = if json.include_schema { - let record = json! {{ - "schema": self.kafka_schema, - "payload": record - }}; - - serde_json::to_vec(&record).unwrap() - } else { - serde_json::to_vec(record).unwrap() - }; - - if json.confluent_schema_registry { - todo!("Serializing to confluent schema registry is not yet supported"); - } - - Some(v) - } - Format::Avro(_) => todo!(), - Format::Parquet(_) => todo!(), - Format::RawString(_) => record.to_raw_string(), - } - } -} #[derive(Debug)] pub struct MilliSecondsSystemTimeVisitor; @@ -308,15 +56,15 @@ pub mod timestamp_as_millis { use super::MilliSecondsSystemTimeVisitor; pub fn serialize(t: &SystemTime, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { serializer.serialize_u64(to_millis(*t)) } pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { deserializer.deserialize_i64(MilliSecondsSystemTimeVisitor) } @@ -330,8 +78,8 @@ pub mod timestamp_as_millis { /// Deserialize a timestamp in milliseconds since the epoch fn visit_i64(self, value: i64) -> Result - where - E: de::Error, + where + E: de::Error, { if value >= 0 { Ok(from_millis(value as u64)) @@ -342,8 +90,8 @@ pub mod timestamp_as_millis { /// Deserialize a timestamp in milliseconds since the epoch fn visit_u64(self, value: u64) -> Result - where - E: de::Error, + where + E: de::Error, { Ok(from_millis(value)) } @@ -363,8 +111,8 @@ pub mod opt_timestamp_as_millis { use super::{MilliSecondsSystemTimeVisitor, OptMilliSecondsSystemTimeVisitor}; pub fn serialize(t: &Option, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { if let Some(t) = t { serializer.serialize_some(&to_millis(*t)) @@ -374,8 +122,8 @@ pub mod opt_timestamp_as_millis { } pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { deserializer.deserialize_option(OptMilliSecondsSystemTimeVisitor) } @@ -389,8 +137,8 @@ pub mod opt_timestamp_as_millis { /// Deserialize a timestamp in milliseconds since the epoch fn visit_some(self, deserializer: D) -> Result - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { Ok(Some( deserializer.deserialize_any(MilliSecondsSystemTimeVisitor)?, @@ -398,8 +146,8 @@ pub mod opt_timestamp_as_millis { } fn visit_none(self) -> Result - where - E: de::Error, + where + E: de::Error, { Ok(None) } @@ -417,16 +165,16 @@ pub mod timestamp_as_rfc3339 { use serde::{Deserialize, Deserializer, Serializer}; pub fn serialize(t: &SystemTime, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { let dt: DateTime = (*t).into(); serializer.serialize_str(&dt.to_rfc3339()) } pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { let raw: chrono::DateTime = DateTime::deserialize(deserializer)?; Ok(from_nanos(raw.timestamp_nanos() as u128)) @@ -441,8 +189,8 @@ pub mod opt_timestamp_as_rfc3339 { use serde::{Deserialize, Deserializer, Serializer}; pub fn serialize(t: &Option, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { if let Some(t) = *t { let dt: DateTime = t.into(); @@ -453,15 +201,15 @@ pub mod opt_timestamp_as_rfc3339 { } pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { let raw = Option::>::deserialize(deserializer)?; Ok(raw.map(|raw| from_nanos(raw.timestamp_nanos() as u128))) } } -fn field_to_json_schema(field: &Field) -> Value { +pub fn field_to_json_schema(field: &Field) -> Value { match field.data_type() { arrow::datatypes::DataType::Null => { json! {{ "type": "null" }} @@ -519,7 +267,7 @@ fn field_to_json_schema(field: &Field) -> Value { } } -fn arrow_to_json_schema(fields: &Fields) -> Value { +pub fn arrow_to_json_schema(fields: &Fields) -> Value { let props: HashMap = fields .iter() .map(|f| (f.name().clone(), field_to_json_schema(f))) @@ -537,7 +285,7 @@ fn arrow_to_json_schema(fields: &Fields) -> Value { }} } -fn field_to_kafka_json(field: &Field) -> Value { +pub fn field_to_kafka_json(field: &Field) -> Value { use arrow::datatypes::DataType::*; let typ = match field.data_type() { @@ -605,7 +353,7 @@ fn field_to_kafka_json(field: &Field) -> Value { // For some reason Kafka uses it's own bespoke almost-but-not-quite JSON schema format // https://www.confluent.io/blog/kafka-connect-deep-dive-converters-serialization-explained/#json-schemas -fn arrow_to_kafka_json(name: &str, fields: &Fields) -> Value { +pub fn arrow_to_kafka_json(name: &str, fields: &Fields) -> Value { let fields: Vec<_> = fields.iter().map(|f| field_to_kafka_json(&*f)).collect(); json! {{ "type": "struct", @@ -613,81 +361,4 @@ fn arrow_to_kafka_json(name: &str, fields: &Fields) -> Value { "fields": fields, "optional": false, }} -} - -#[cfg(test)] -mod tests { - use crate::formats::FramingIterator; - use arroyo_rpc::formats::{Framing, FramingMethod, NewlineDelimitedFraming}; - use std::sync::Arc; - - #[test] - fn test_line_framing() { - let framing = Some(Arc::new(Framing { - method: FramingMethod::Newline(NewlineDelimitedFraming { - max_line_length: None, - }), - })); - - let result: Vec<_> = FramingIterator::new(framing.clone(), "one block".as_bytes()) - .map(|t| String::from_utf8(t.to_vec()).unwrap()) - .collect(); - - assert_eq!(vec!["one block".to_string()], result); - - let result: Vec<_> = FramingIterator::new( - framing.clone(), - "one block\ntwo block\nthree block".as_bytes(), - ) - .map(|t| String::from_utf8(t.to_vec()).unwrap()) - .collect(); - - assert_eq!( - vec![ - "one block".to_string(), - "two block".to_string(), - "three block".to_string() - ], - result - ); - - let result: Vec<_> = FramingIterator::new( - framing.clone(), - "one block\ntwo block\nthree block\n".as_bytes(), - ) - .map(|t| String::from_utf8(t.to_vec()).unwrap()) - .collect(); - - assert_eq!( - vec![ - "one block".to_string(), - "two block".to_string(), - "three block".to_string() - ], - result - ); - } - - #[test] - fn test_max_line_length() { - let framing = Some(Arc::new(Framing { - method: FramingMethod::Newline(NewlineDelimitedFraming { - max_line_length: Some(5), - }), - })); - - let result: Vec<_> = - FramingIterator::new(framing, "one block\ntwo block\nwhole".as_bytes()) - .map(|t| String::from_utf8(t.to_vec()).unwrap()) - .collect(); - - assert_eq!( - vec![ - "one b".to_string(), - "two b".to_string(), - "whole".to_string() - ], - result - ); - } -} +} \ No newline at end of file diff --git a/arroyo-worker/src/formats/mod.rs b/arroyo-worker/src/formats/mod.rs new file mode 100644 index 000000000..ce0b84d83 --- /dev/null +++ b/arroyo-worker/src/formats/mod.rs @@ -0,0 +1,280 @@ +mod avro; +pub mod json; + +use apache_avro::{from_avro_datum, Reader, Schema}; +use std::sync::Arc; +use std::{collections::HashMap, marker::PhantomData}; + +use arrow::datatypes::{Field, Fields}; +use arroyo_rpc::formats::{AvroFormat, Format, Framing, FramingMethod, JsonFormat}; +use arroyo_rpc::schema_resolver::{FailingSchemaResolver, SchemaResolver}; +use arroyo_types::UserError; +use serde::de::DeserializeOwned; +use serde_json::{json, Value}; +use tokio::sync::Mutex; + +use crate::SchemaData; + + +fn deserialize_raw_string(msg: &[u8]) -> Result { + let json = json! { + { "value": String::from_utf8_lossy(msg) } + }; + Ok(serde_json::from_value(json).unwrap()) +} + + +pub struct FramingIterator<'a> { + framing: Option>, + buf: &'a [u8], + offset: usize, +} + +impl<'a> FramingIterator<'a> { + pub fn new(framing: Option>, buf: &'a [u8]) -> Self { + Self { + framing, + buf, + offset: 0, + } + } +} + +impl<'a> Iterator for FramingIterator<'a> { + type Item = &'a [u8]; + + fn next(&mut self) -> Option { + if self.offset >= self.buf.len() { + return None; + } + + match &self.framing { + Some(framing) => { + match &framing.method { + FramingMethod::Newline(newline) => { + let end = memchr::memchr('\n' as u8, &self.buf[self.offset..]) + .map(|i| self.offset + i) + .unwrap_or(self.buf.len()); + + let prev = self.offset; + self.offset = end + 1; + + // enforce max len if set + let length = + (end - prev).min(newline.max_line_length.unwrap_or(u64::MAX) as usize); + + Some(&self.buf[prev..(prev + length)]) + } + } + } + None => { + self.offset = self.buf.len(); + Some(self.buf) + } + } + } +} + +#[derive(Clone)] +pub struct DataDeserializer { + format: Arc, + framing: Option>, + schema_registry: Arc>>, + schema_resolver: Arc, + _t: PhantomData, +} + +impl DataDeserializer { + pub fn new(format: Format, framing: Option) -> Self { + Self::with_schema_resolver(format, framing, Arc::new(FailingSchemaResolver::new())) + } + + pub fn with_schema_resolver( + format: Format, + framing: Option, + schema_resolver: Arc, + ) -> Self { + Self { + format: Arc::new(format), + framing: framing.map(|f| Arc::new(f)), + schema_registry: Arc::new(Mutex::new(HashMap::new())), + schema_resolver, + _t: PhantomData, + } + } + + pub async fn deserialize_slice<'a>( + &mut self, + msg: &'a [u8], + ) -> Box> + 'a + Send> { + match &*self.format { + Format::Avro(avro) => { + let schema_registry = self.schema_registry.clone(); + let schema_resolver = self.schema_resolver.clone(); + match avro::deserialize_slice_avro( + avro, + schema_registry, + schema_resolver, + msg, + ).await { + Ok(iter) => Box::new(iter), + Err(e) => Box::new(vec![ + Err(UserError::new("Avro deserialization failed", e)) + ].into_iter()) + } + } + _ => { + let new_self = self.clone(); + Box::new(FramingIterator::new(self.framing.clone(), msg) + .map(move |t| new_self.deserialize_single(t))) + } + } + } + + fn deserialize_single(&self, msg: &[u8]) -> Result { + match &*self.format { + Format::Json(json) => json::deserialize_slice_json(json, msg), + Format::Avro(_) => unreachable!("avro should be handled by here"), + Format::Parquet(_) => todo!("parquet is not supported as an input format"), + Format::RawString(_) => deserialize_raw_string(msg), + } + .map_err(|e| { + UserError::new( + "Deserialization failed", + format!( + "Failed to deserialize: '{}': {}", + String::from_utf8_lossy(&msg), + e + ), + ) + }) + } +} + +pub struct DataSerializer { + kafka_schema: Value, + #[allow(unused)] + json_schema: Value, + format: Format, + _t: PhantomData, +} + +impl DataSerializer { + pub fn new(format: Format) -> Self { + Self { + kafka_schema: json::arrow_to_kafka_json(T::name(), T::schema().fields()), + json_schema: json::arrow_to_json_schema(T::schema().fields()), + format, + _t: PhantomData, + } + } + + pub fn to_vec(&self, record: &T) -> Option> { + match &self.format { + Format::Json(json) => { + let v = if json.include_schema { + let record = json! {{ + "schema": self.kafka_schema, + "payload": record + }}; + + serde_json::to_vec(&record).unwrap() + } else { + serde_json::to_vec(record).unwrap() + }; + + if json.confluent_schema_registry { + todo!("Serializing to confluent schema registry is not yet supported"); + } + + Some(v) + } + Format::Avro(_) => todo!(), + Format::Parquet(_) => todo!(), + Format::RawString(_) => record.to_raw_string(), + } + } +} + + + +#[cfg(test)] +mod tests { + use crate::formats::{DataDeserializer, FramingIterator}; + use arroyo_rpc::formats::{AvroFormat, Format, Framing, FramingMethod, NewlineDelimitedFraming}; + use std::sync::Arc; + use std::time::SystemTime; + use async_trait::async_trait; + use arroyo_rpc::schema_resolver::SchemaResolver; + use crate::SchemaData; + + #[test] + fn test_line_framing() { + let framing = Some(Arc::new(Framing { + method: FramingMethod::Newline(NewlineDelimitedFraming { + max_line_length: None, + }), + })); + + let result: Vec<_> = FramingIterator::new(framing.clone(), "one block".as_bytes()) + .map(|t| String::from_utf8(t.to_vec()).unwrap()) + .collect(); + + assert_eq!(vec!["one block".to_string()], result); + + let result: Vec<_> = FramingIterator::new( + framing.clone(), + "one block\ntwo block\nthree block".as_bytes(), + ) + .map(|t| String::from_utf8(t.to_vec()).unwrap()) + .collect(); + + assert_eq!( + vec![ + "one block".to_string(), + "two block".to_string(), + "three block".to_string() + ], + result + ); + + let result: Vec<_> = FramingIterator::new( + framing.clone(), + "one block\ntwo block\nthree block\n".as_bytes(), + ) + .map(|t| String::from_utf8(t.to_vec()).unwrap()) + .collect(); + + assert_eq!( + vec![ + "one block".to_string(), + "two block".to_string(), + "three block".to_string() + ], + result + ); + } + + #[test] + fn test_max_line_length() { + let framing = Some(Arc::new(Framing { + method: FramingMethod::Newline(NewlineDelimitedFraming { + max_line_length: Some(5), + }), + })); + + let result: Vec<_> = + FramingIterator::new(framing, "one block\ntwo block\nwhole".as_bytes()) + .map(|t| String::from_utf8(t.to_vec()).unwrap()) + .collect(); + + assert_eq!( + vec![ + "one b".to_string(), + "two b".to_string(), + "whole".to_string() + ], + result + ); + } +} From 491ada016c69b2b8eb2d69a87f5c77b4f8f2e633 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Fri, 27 Oct 2023 17:09:03 -0700 Subject: [PATCH 09/12] avro to json conversion --- Cargo.lock | 38 ++-- Cargo.toml | 8 +- arroyo-rpc/src/formats.rs | 17 +- .../src/connectors/kafka/source/mod.rs | 17 +- arroyo-worker/src/formats/avro.rs | 198 ++++++++++++------ arroyo-worker/src/formats/json.rs | 55 +++-- arroyo-worker/src/formats/mod.rs | 35 ++-- 7 files changed, 224 insertions(+), 144 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 391e93b42..ec1508cda 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -442,7 +442,7 @@ dependencies = [ "async-trait", "axum", "axum-extra", - "base64 0.21.4", + "base64 0.21.5", "bincode 2.0.0-rc.3", "chrono", "cornucopia", @@ -1552,9 +1552,9 @@ checksum = "0ea22880d78093b0cbe17c89f64a7d457941e65759157ec6cb31a31d652b05e5" [[package]] name = "base64" -version = "0.21.4" +version = "0.21.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ba43ea6f343b788c8764558649e08df62f86c6ef251fdaeb1ffd010a9ae50a2" +checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" [[package]] name = "base64ct" @@ -1670,7 +1670,7 @@ version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f03db470b3c0213c47e978da93200259a1eb4dae2e5512cba9955e2b540a6fc6" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bollard-stubs", "bytes", "futures-core", @@ -2616,7 +2616,7 @@ dependencies = [ "arrow-array", "arrow-buffer", "arrow-schema", - "base64 0.21.4", + "base64 0.21.5", "blake2", "blake3", "chrono", @@ -3150,7 +3150,7 @@ dependencies = [ "async-lock", "async-rwlock", "async-trait", - "base64 0.21.4", + "base64 0.21.5", "bytes", "cfg-if", "chrono", @@ -3201,7 +3201,7 @@ checksum = "cd0337e771c38978625941b31ebce79b708c9252744f23039b9ff0434e251e02" dependencies = [ "anyhow", "async-trait", - "base64 0.21.4", + "base64 0.21.5", "bytes", "bytesize", "derive_builder", @@ -3734,7 +3734,7 @@ version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "06683b93020a07e3dbcf5f8c0f6d40080d725bea7936fc01ad345c01b97dc270" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bytes", "headers-core", "http", @@ -4322,7 +4322,7 @@ version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cd990069640f9db34b3b0f7a1afc62a05ffaa3be9b66aa3c313f58346df7f788" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bytes", "chrono", "http", @@ -5178,7 +5178,7 @@ version = "0.7.1" source = "git+https://github.com/ArroyoSystems/arrow-rs?branch=object_store/put_part_api#612c9484547d3274571ce932260a75dc6413d6d1" dependencies = [ "async-trait", - "base64 0.21.4", + "base64 0.21.5", "bytes", "chrono", "futures", @@ -5421,7 +5421,7 @@ dependencies = [ "arrow-ipc", "arrow-schema", "arrow-select", - "base64 0.21.4", + "base64 0.21.5", "brotli", "bytes", "chrono", @@ -5684,7 +5684,7 @@ version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "49b6c5ef183cd3ab4ba005f1ca64c21e8bd97ce4699cfea9e8d9a2c4958ca520" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "byteorder", "bytes", "fallible-iterator", @@ -6258,7 +6258,7 @@ version = "0.11.22" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bytes", "encoding_rs", "futures-core", @@ -6602,7 +6602,7 @@ version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d3987094b1d07b653b7dfdc3f70ce9a1da9c51ac18c1b06b662e4f9a0e9f4b2" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", ] [[package]] @@ -6941,7 +6941,7 @@ version = "3.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ca3b16a3d82c4088f343b7480a93550b3eabe1a358569c2dfe38bbcead07237" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "chrono", "hex", "indexmap 1.9.3", @@ -7843,7 +7843,7 @@ checksum = "3082666a3a6433f7f511c7192923fa1fe07c69332d3c6a2e6bb040b569199d5a" dependencies = [ "async-trait", "axum", - "base64 0.21.4", + "base64 0.21.5", "bytes", "futures-core", "futures-util", @@ -7895,7 +7895,7 @@ version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "21b00ec4842256d1fe0a46176e2ef5bc357664c66e7d91aff5a7d43d83a65f47" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bytes", "futures-core", "http", @@ -7935,7 +7935,7 @@ version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "61c5bb1d698276a2443e5ecfabc1008bf15a36c12e6a7176e7bf089ea9131140" dependencies = [ - "base64 0.21.4", + "base64 0.21.5", "bitflags 2.4.0", "bytes", "futures-core", @@ -8543,7 +8543,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f58ddfe801df3886feaf466d883ea37e941bcc6d841b9f644a08c7acabfe7f8" dependencies = [ "anyhow", - "base64 0.21.4", + "base64 0.21.5", "bincode 1.3.3", "directories-next", "file-per-thread-logger", diff --git a/Cargo.toml b/Cargo.toml index 76076f44e..be113f657 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -35,7 +35,6 @@ tonic-build = { version = "0.9" } tonic-web = { version = "0.9" } tonic-reflection = { version = "0.9" } arrow = { version = "46.0.0" } -arrow-buffer = { version = "46.0.0" } arrow-array = { version = "46.0.0" } arrow-schema = { version = "46.0.0" } object_store = { version = "0.7.1" } @@ -44,11 +43,6 @@ parquet = { version = "46.0.0" } [profile.release] debug = 1 -[profile.release.package.copy-artifacts] -# optimize for small binary size -strip = true -opt-level = "z" - [patch.crates-io] typify = { git = 'https://github.com/ArroyoSystems/typify.git', branch = 'arroyo' } parquet = {git = 'https://github.com/ArroyoSystems/arrow-rs', branch = '46.0.0/parquet_bytes'} @@ -56,4 +50,4 @@ arrow = {git = 'https://github.com/ArroyoSystems/arrow-rs', branch = '46.0.0/par arrow-buffer = {git = 'https://github.com/ArroyoSystems/arrow-rs', branch = '46.0.0/parquet_bytes'} arrow-array = {git = 'https://github.com/ArroyoSystems/arrow-rs', branch = '46.0.0/parquet_bytes'} arrow-schema = {git = 'https://github.com/ArroyoSystems/arrow-rs', branch = '46.0.0/parquet_bytes'} -object_store = {git = 'https://github.com/ArroyoSystems/arrow-rs', branch = 'object_store/put_part_api'} \ No newline at end of file +object_store = {git = 'https://github.com/ArroyoSystems/arrow-rs', branch = 'object_store/put_part_api'} diff --git a/arroyo-rpc/src/formats.rs b/arroyo-rpc/src/formats.rs index 412e4a928..0d7036c8d 100644 --- a/arroyo-rpc/src/formats.rs +++ b/arroyo-rpc/src/formats.rs @@ -107,13 +107,26 @@ pub struct AvroFormat { #[serde(default)] pub embedded_schema: bool, + + #[serde(default)] + pub into_json: bool, } impl AvroFormat { pub fn from_opts(opts: &mut HashMap) -> Result { Ok(Self { - confluent_schema_registry: false, - embedded_schema: false, + confluent_schema_registry: opts + .remove("avro.confluent_schema_registry") + .filter(|t| t == "true") + .is_some(), + embedded_schema: opts + .remove("avro.include_schema") + .filter(|t| t == "true") + .is_some(), + into_json: opts + .remove("avro.into_json") + .filter(|t| t == "true") + .is_some(), }) } } diff --git a/arroyo-worker/src/connectors/kafka/source/mod.rs b/arroyo-worker/src/connectors/kafka/source/mod.rs index d77a0ab02..0de53ebcc 100644 --- a/arroyo-worker/src/connectors/kafka/source/mod.rs +++ b/arroyo-worker/src/connectors/kafka/source/mod.rs @@ -5,6 +5,7 @@ use crate::SourceFinishType; use arroyo_macro::source_fn; use arroyo_rpc::formats::{Format, Framing}; use arroyo_rpc::grpc::TableDescriptor; +use arroyo_rpc::schema_resolver::{ConfluentSchemaResolver, FailingSchemaResolver, SchemaResolver}; use arroyo_rpc::OperatorConfig; use arroyo_rpc::{grpc::StopMode, ControlMessage, ControlResp}; use arroyo_state::tables::global_keyed_map::GlobalKeyedState; @@ -21,7 +22,6 @@ use std::sync::Arc; use std::time::Duration; use tokio::select; use tracing::{debug, error, info, warn}; -use arroyo_rpc::schema_resolver::{ConfluentSchemaResolver, FailingSchemaResolver, SchemaResolver}; use super::{client_configs, KafkaConfig, KafkaTable, ReadMode, TableType}; @@ -105,12 +105,15 @@ where client_configs.insert("isolation.level".to_string(), "read_committed".to_string()); } - let schema_resolver: Arc = if let Some(schema_registry) = &connection.schema_registry { - Arc::new(ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic) - .expect("failed to construct confluent schema resolver")) - } else { - Arc::new(FailingSchemaResolver::new()) - }; + let schema_resolver: Arc = + if let Some(schema_registry) = &connection.schema_registry { + Arc::new( + ConfluentSchemaResolver::new(&schema_registry.endpoint, &table.topic) + .expect("failed to construct confluent schema resolver"), + ) + } else { + Arc::new(FailingSchemaResolver::new()) + }; Self { topic: table.topic, diff --git a/arroyo-worker/src/formats/avro.rs b/arroyo-worker/src/formats/avro.rs index e3d4311bd..3880a9b93 100644 --- a/arroyo-worker/src/formats/avro.rs +++ b/arroyo-worker/src/formats/avro.rs @@ -1,11 +1,13 @@ -use std::collections::HashMap; -use std::sync::Arc; -use apache_avro::{from_avro_datum, Reader, Schema}; -use serde::de::DeserializeOwned; -use tokio::sync::Mutex; +use apache_avro::types::{Value as AvroValue, Value}; +use apache_avro::{from_avro_datum, Decimal, Reader, Schema}; use arroyo_rpc::formats::AvroFormat; use arroyo_rpc::schema_resolver::SchemaResolver; use arroyo_types::UserError; +use serde::de::DeserializeOwned; +use serde_json::{json, Value as JsonValue}; +use std::collections::HashMap; +use std::sync::Arc; +use tokio::sync::Mutex; pub async fn deserialize_slice_avro<'a, T: DeserializeOwned>( format: &AvroFormat, @@ -29,22 +31,25 @@ pub async fn deserialize_slice_avro<'a, T: DeserializeOwned>( let mut registry = schema_registry.lock().await; let messages = if format.embedded_schema { - Reader::new(&msg[..]).map_err(|e| format!("invalid Avro schema in message: {:?}", e))? + Reader::new(&msg[..]) + .map_err(|e| format!("invalid Avro schema in message: {:?}", e))? .collect() } else { let schema = if registry.contains_key(&id) { registry.get(&id).unwrap() } else { - let new_schema = resolver.resolve_schema(id).await?.ok_or_else(|| { + let new_schema = resolver + .resolve_schema(id) + .await? + .ok_or_else(|| format!("could not resolve schema for message with id {}", id))?; + + let new_schema = Schema::parse_str(&new_schema).map_err(|e| { format!( - "could not resolve schema for message with id {}", - id + "schema from Confluent Schema registry is not valid: {:?}", + e ) })?; - let new_schema = Schema::parse_str(&new_schema) - .map_err(|e| format!("schema from Confluent Schema registry is not valid: {:?}", e))?; - registry.insert(id, new_schema); registry.get(&id).unwrap() @@ -61,39 +66,98 @@ pub async fn deserialize_slice_avro<'a, T: DeserializeOwned>( } }; - Ok(messages.into_iter().map(|record| { - apache_avro::from_value::(&record.map_err(|e| { + let into_json = format.into_json; + Ok(messages.into_iter().map(move |record| { + let value = record.map_err(|e| { UserError::new( "Deserialization failed", - format!( - "Failed to deserialize from avro: {:?}", - e - ), + format!("Failed to deserialize from avro: {:?}", e), ) - })?) - .map_err(|e| { + })?; + + if into_json { + Ok(serde_json::from_value(json!({"value": avro_to_json(value).to_string()})).unwrap()) + } else { + apache_avro::from_value::(&value).map_err(|e| { UserError::new( "Deserialization failed", format!("Failed to convert avro message into struct type: {:?}", e), ) }) + } })) +} +fn convert_float(f: f64) -> JsonValue { + match serde_json::Number::from_f64(f) { + Some(n) => JsonValue::Number(n), + None => JsonValue::String( + (if f.is_infinite() && f.is_sign_positive() { + "+Inf" + } else if f.is_infinite() { + "-Inf" + } else { + "NaN" + }) + .to_string(), + ), + } +} - // let record = reader.next() - // .ok_or_else(|| "avro record did not contain any messages")? - // .map_err(|e| e.to_string())?; +fn encode_vec(v: Vec) -> JsonValue { + JsonValue::String(v.into_iter().map(char::from).collect()) } +fn avro_to_json(value: AvroValue) -> JsonValue { + match value { + Value::Null => JsonValue::Null, + Value::Boolean(b) => JsonValue::Bool(b), + Value::Int(i) | Value::Date(i) | Value::TimeMillis(i) => { + JsonValue::Number(serde_json::Number::from(i)) + } + Value::Long(i) + | Value::TimeMicros(i) + | Value::TimestampMillis(i) + | Value::TimestampMicros(i) + | Value::LocalTimestampMillis(i) + | Value::LocalTimestampMicros(i) => JsonValue::Number(serde_json::Number::from(i)), + Value::Float(f) => convert_float(f as f64), + Value::Double(f) => convert_float(f), + Value::String(s) | Value::Enum(_, s) => JsonValue::String(s), + // this isn't the standard Avro json encoding, which just + Value::Bytes(b) | Value::Fixed(_, b) => encode_vec(b), + Value::Union(_, b) => avro_to_json(*b), + Value::Array(a) => JsonValue::Array(a.into_iter().map(|v| avro_to_json(v)).collect()), + Value::Map(m) => { + JsonValue::Object(m.into_iter().map(|(k, v)| (k, avro_to_json(v))).collect()) + } + Value::Record(rec) => { + JsonValue::Object(rec.into_iter().map(|(k, v)| (k, avro_to_json(v))).collect()) + } + + Value::Decimal(d) => { + let b: Vec = d.try_into().unwrap_or_else(|_| vec![]); + encode_vec(b) + } + Value::Duration(d) => { + json!({ + "months": u32::from(d.months()), + "days": u32::from(d.days()), + "milliseconds": u32::from(d.millis()) + }) + } + Value::Uuid(u) => JsonValue::String(u.to_string()), + } +} #[cfg(test)] mod tests { - use std::sync::Arc; - use async_trait::async_trait; - use arroyo_rpc::formats::{AvroFormat, Format}; - use arroyo_rpc::schema_resolver::SchemaResolver; use crate::formats::DataDeserializer; use crate::SchemaData; + use arroyo_rpc::formats::{AvroFormat, Format}; + use arroyo_rpc::schema_resolver::SchemaResolver; + use async_trait::async_trait; + use std::sync::Arc; struct TestSchemaResolver { schema: String, @@ -110,14 +174,14 @@ mod tests { } #[derive( - Clone, - Debug, - bincode::Encode, - bincode::Decode, - PartialEq, - PartialOrd, - serde::Serialize, - serde::Deserialize + Clone, + Debug, + bincode::Encode, + bincode::Decode, + PartialEq, + PartialOrd, + serde::Serialize, + serde::Deserialize, )] pub struct ArroyoAvroRoot { pub store_id: i32, @@ -130,14 +194,14 @@ mod tests { } #[derive( - Clone, - Debug, - bincode::Encode, - bincode::Decode, - PartialEq, - PartialOrd, - serde::Serialize, - serde::Deserialize + Clone, + Debug, + bincode::Encode, + bincode::Decode, + PartialEq, + PartialOrd, + serde::Serialize, + serde::Deserialize, )] pub struct OrderLine { pub product_id: i32, @@ -153,18 +217,24 @@ mod tests { } fn schema() -> arrow::datatypes::Schema { let fields: Vec = vec![ - arrow::datatypes::Field::new("store_id", - arrow::datatypes::DataType::Int32, false), - arrow::datatypes::Field::new("store_order_id", - arrow::datatypes::DataType::Int32, false), - arrow::datatypes::Field::new("coupon_code", - arrow::datatypes::DataType::Int32, false), - arrow::datatypes::Field::new("date", arrow::datatypes::DataType::Utf8, - false), - arrow::datatypes::Field::new("status", - arrow::datatypes::DataType::Utf8, false), - arrow::datatypes::Field::new("order_lines", - arrow::datatypes::DataType::Utf8, false), + arrow::datatypes::Field::new("store_id", arrow::datatypes::DataType::Int32, false), + arrow::datatypes::Field::new( + "store_order_id", + arrow::datatypes::DataType::Int32, + false, + ), + arrow::datatypes::Field::new( + "coupon_code", + arrow::datatypes::DataType::Int32, + false, + ), + arrow::datatypes::Field::new("date", arrow::datatypes::DataType::Utf8, false), + arrow::datatypes::Field::new("status", arrow::datatypes::DataType::Utf8, false), + arrow::datatypes::Field::new( + "order_lines", + arrow::datatypes::DataType::Utf8, + false, + ), ]; arrow::datatypes::Schema::new(fields) } @@ -243,10 +313,12 @@ mod tests { "type": "record" }"#; - let message = [0u8, 0, 0, 0, 1, 8, 200, 223, 1, 144, 31, 186, 159, 2, 16, 97, 99, 99, - 101, 112, 116, 101, 100, 4, 156, 1, 10, 112, 105, 122, 122, 97, 4, 102, 102, 102, 102, 102, - 230, 38, 64, 102, 102, 102, 102, 102, 230, 54, 64, 84, 14, 100, 101, 115, 115, 101, 114, 116, - 2, 113, 61, 10, 215, 163, 112, 26, 64, 113, 61, 10, 215, 163, 112, 26, 64, 0, 10]; + let message = [ + 0u8, 0, 0, 0, 1, 8, 200, 223, 1, 144, 31, 186, 159, 2, 16, 97, 99, 99, 101, 112, 116, + 101, 100, 4, 156, 1, 10, 112, 105, 122, 122, 97, 4, 102, 102, 102, 102, 102, 230, 38, + 64, 102, 102, 102, 102, 102, 230, 54, 64, 84, 14, 100, 101, 115, 115, 101, 114, 116, 2, + 113, 61, 10, 215, 163, 112, 26, 64, 113, 61, 10, 215, 163, 112, 26, 64, 0, 10, + ]; let mut deserializer = DataDeserializer::with_schema_resolver( Format::Avro(AvroFormat { @@ -257,12 +329,14 @@ mod tests { Arc::new(TestSchemaResolver { schema: schema.to_string(), id: 1, - })); + }), + ); - let v: Vec> = deserializer.deserialize_slice(&message[..]).await.collect(); + let v: Vec> = + deserializer.deserialize_slice(&message[..]).await.collect(); for i in v { println!("{:?}", i.unwrap()); } } -} \ No newline at end of file +} diff --git a/arroyo-worker/src/formats/json.rs b/arroyo-worker/src/formats/json.rs index 22bcaba8a..4a4706b8d 100644 --- a/arroyo-worker/src/formats/json.rs +++ b/arroyo-worker/src/formats/json.rs @@ -1,8 +1,8 @@ -use std::collections::HashMap; use arrow::datatypes::{Field, Fields}; +use arroyo_rpc::formats::JsonFormat; use serde::de::DeserializeOwned; use serde_json::{json, Value}; -use arroyo_rpc::formats::JsonFormat; +use std::collections::HashMap; pub fn deserialize_slice_json( format: &JsonFormat, @@ -42,7 +42,6 @@ pub fn deserialize_slice_json( } } - #[derive(Debug)] pub struct MilliSecondsSystemTimeVisitor; @@ -56,15 +55,15 @@ pub mod timestamp_as_millis { use super::MilliSecondsSystemTimeVisitor; pub fn serialize(t: &SystemTime, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { serializer.serialize_u64(to_millis(*t)) } pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { deserializer.deserialize_i64(MilliSecondsSystemTimeVisitor) } @@ -78,8 +77,8 @@ pub mod timestamp_as_millis { /// Deserialize a timestamp in milliseconds since the epoch fn visit_i64(self, value: i64) -> Result - where - E: de::Error, + where + E: de::Error, { if value >= 0 { Ok(from_millis(value as u64)) @@ -90,8 +89,8 @@ pub mod timestamp_as_millis { /// Deserialize a timestamp in milliseconds since the epoch fn visit_u64(self, value: u64) -> Result - where - E: de::Error, + where + E: de::Error, { Ok(from_millis(value)) } @@ -111,8 +110,8 @@ pub mod opt_timestamp_as_millis { use super::{MilliSecondsSystemTimeVisitor, OptMilliSecondsSystemTimeVisitor}; pub fn serialize(t: &Option, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { if let Some(t) = t { serializer.serialize_some(&to_millis(*t)) @@ -122,8 +121,8 @@ pub mod opt_timestamp_as_millis { } pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { deserializer.deserialize_option(OptMilliSecondsSystemTimeVisitor) } @@ -137,8 +136,8 @@ pub mod opt_timestamp_as_millis { /// Deserialize a timestamp in milliseconds since the epoch fn visit_some(self, deserializer: D) -> Result - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { Ok(Some( deserializer.deserialize_any(MilliSecondsSystemTimeVisitor)?, @@ -146,8 +145,8 @@ pub mod opt_timestamp_as_millis { } fn visit_none(self) -> Result - where - E: de::Error, + where + E: de::Error, { Ok(None) } @@ -165,16 +164,16 @@ pub mod timestamp_as_rfc3339 { use serde::{Deserialize, Deserializer, Serializer}; pub fn serialize(t: &SystemTime, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { let dt: DateTime = (*t).into(); serializer.serialize_str(&dt.to_rfc3339()) } pub fn deserialize<'de, D>(deserializer: D) -> Result - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { let raw: chrono::DateTime = DateTime::deserialize(deserializer)?; Ok(from_nanos(raw.timestamp_nanos() as u128)) @@ -189,8 +188,8 @@ pub mod opt_timestamp_as_rfc3339 { use serde::{Deserialize, Deserializer, Serializer}; pub fn serialize(t: &Option, serializer: S) -> Result - where - S: Serializer, + where + S: Serializer, { if let Some(t) = *t { let dt: DateTime = t.into(); @@ -201,8 +200,8 @@ pub mod opt_timestamp_as_rfc3339 { } pub fn deserialize<'de, D>(deserializer: D) -> Result, D::Error> - where - D: Deserializer<'de>, + where + D: Deserializer<'de>, { let raw = Option::>::deserialize(deserializer)?; Ok(raw.map(|raw| from_nanos(raw.timestamp_nanos() as u128))) @@ -361,4 +360,4 @@ pub fn arrow_to_kafka_json(name: &str, fields: &Fields) -> Value { "fields": fields, "optional": false, }} -} \ No newline at end of file +} diff --git a/arroyo-worker/src/formats/mod.rs b/arroyo-worker/src/formats/mod.rs index ce0b84d83..08005452a 100644 --- a/arroyo-worker/src/formats/mod.rs +++ b/arroyo-worker/src/formats/mod.rs @@ -15,7 +15,6 @@ use tokio::sync::Mutex; use crate::SchemaData; - fn deserialize_raw_string(msg: &[u8]) -> Result { let json = json! { { "value": String::from_utf8_lossy(msg) } @@ -23,7 +22,6 @@ fn deserialize_raw_string(msg: &[u8]) -> Result Ok(serde_json::from_value(json).unwrap()) } - pub struct FramingIterator<'a> { framing: Option>, buf: &'a [u8], @@ -111,22 +109,21 @@ impl DataDeserializer { Format::Avro(avro) => { let schema_registry = self.schema_registry.clone(); let schema_resolver = self.schema_resolver.clone(); - match avro::deserialize_slice_avro( - avro, - schema_registry, - schema_resolver, - msg, - ).await { + match avro::deserialize_slice_avro(avro, schema_registry, schema_resolver, msg) + .await + { Ok(iter) => Box::new(iter), - Err(e) => Box::new(vec![ - Err(UserError::new("Avro deserialization failed", e)) - ].into_iter()) + Err(e) => Box::new( + vec![Err(UserError::new("Avro deserialization failed", e))].into_iter(), + ), } } _ => { let new_self = self.clone(); - Box::new(FramingIterator::new(self.framing.clone(), msg) - .map(move |t| new_self.deserialize_single(t))) + Box::new( + FramingIterator::new(self.framing.clone(), msg) + .map(move |t| new_self.deserialize_single(t)), + ) } } } @@ -196,17 +193,17 @@ impl DataSerializer { } } - - #[cfg(test)] mod tests { use crate::formats::{DataDeserializer, FramingIterator}; - use arroyo_rpc::formats::{AvroFormat, Format, Framing, FramingMethod, NewlineDelimitedFraming}; + use crate::SchemaData; + use arroyo_rpc::formats::{ + AvroFormat, Format, Framing, FramingMethod, NewlineDelimitedFraming, + }; + use arroyo_rpc::schema_resolver::SchemaResolver; + use async_trait::async_trait; use std::sync::Arc; use std::time::SystemTime; - use async_trait::async_trait; - use arroyo_rpc::schema_resolver::SchemaResolver; - use crate::SchemaData; #[test] fn test_line_framing() { From d5c1771c7ce86cdd8b0e9ef81db8fcfbca4302fd Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Sat, 28 Oct 2023 14:41:51 -0700 Subject: [PATCH 10/12] cleanups --- arroyo-api/src/connection_tables.rs | 5 ++- arroyo-connectors/src/kafka.rs | 6 +++- arroyo-rpc/src/formats.rs | 10 +++--- arroyo-rpc/src/schema_resolver.rs | 2 +- arroyo-sql/src/avro.rs | 47 ++++++++--------------------- arroyo-sql/src/types.rs | 12 ++++++++ arroyo-worker/src/formats/avro.rs | 8 +++-- arroyo-worker/src/formats/json.rs | 13 ++++++-- arroyo-worker/src/formats/mod.rs | 22 +++++--------- 9 files changed, 61 insertions(+), 64 deletions(-) diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index b0b6c47e6..c6d43f95e 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -9,15 +9,14 @@ use axum_extra::extract::WithRejection; use cornucopia_async::GenericClient; use cornucopia_async::Params; use futures_util::stream::Stream; -use http::StatusCode; use serde_json::{json, Value}; use std::convert::Infallible; use tokio::sync::mpsc::channel; use tokio_stream::wrappers::ReceiverStream; use tracing::warn; -use arroyo_connectors::kafka::{KafkaConfig, KafkaConnector, KafkaTable}; -use arroyo_connectors::{connector_for_type, Connector, ErasedConnector}; +use arroyo_connectors::kafka::{KafkaConfig, KafkaTable}; +use arroyo_connectors::{connector_for_type, ErasedConnector}; use arroyo_rpc::api_types::connections::{ ConnectionProfile, ConnectionSchema, ConnectionTable, ConnectionTablePost, SchemaDefinition, }; diff --git a/arroyo-connectors/src/kafka.rs b/arroyo-connectors/src/kafka.rs index dea6e3014..a5e5262fa 100644 --- a/arroyo-connectors/src/kafka.rs +++ b/arroyo-connectors/src/kafka.rs @@ -151,10 +151,14 @@ impl Connector for KafkaConnector { Some(other) => bail!("unknown auth type '{}'", other), }; + let schema_registry = opts + .remove("schema_registry.endpoint") + .map(|endpoint| SchemaRegistry { endpoint }); + let connection = KafkaConfig { authentication: auth, bootstrap_servers: BootstrapServers(pull_opt("bootstrap_servers", opts)?), - schema_registry: None, + schema_registry, }; let typ = pull_opt("type", opts)?; diff --git a/arroyo-rpc/src/formats.rs b/arroyo-rpc/src/formats.rs index 0d7036c8d..f40df8d89 100644 --- a/arroyo-rpc/src/formats.rs +++ b/arroyo-rpc/src/formats.rs @@ -1,8 +1,6 @@ -use arroyo_types::UserError; use serde::{Deserialize, Serialize}; -use serde_json::Value; use std::collections::HashMap; -use std::hash::{Hash, Hasher}; +use std::hash::Hash; use std::str::FromStr; use utoipa::ToSchema; @@ -109,7 +107,7 @@ pub struct AvroFormat { pub embedded_schema: bool, #[serde(default)] - pub into_json: bool, + pub into_unstructured_json: bool, } impl AvroFormat { @@ -123,8 +121,8 @@ impl AvroFormat { .remove("avro.include_schema") .filter(|t| t == "true") .is_some(), - into_json: opts - .remove("avro.into_json") + into_unstructured_json: opts + .remove("avro.into_unstructured_json") .filter(|t| t == "true") .is_some(), }) diff --git a/arroyo-rpc/src/schema_resolver.rs b/arroyo-rpc/src/schema_resolver.rs index 8be98ff36..4d8b7ef00 100644 --- a/arroyo-rpc/src/schema_resolver.rs +++ b/arroyo-rpc/src/schema_resolver.rs @@ -64,7 +64,7 @@ impl ConfluentSchemaResolver { let endpoint: Url = format!("{}/subjects/{}-value/versions/", endpoint, topic) .as_str() .try_into() - .map_err(|e| anyhow!("{} is not a valid url", endpoint))?; + .map_err(|_| anyhow!("{} is not a valid url", endpoint))?; Ok(Self { client, diff --git a/arroyo-sql/src/avro.rs b/arroyo-sql/src/avro.rs index 1bb6f178b..7e04fb3a4 100644 --- a/arroyo-sql/src/avro.rs +++ b/arroyo-sql/src/avro.rs @@ -4,7 +4,6 @@ use apache_avro::Schema; use arrow_schema::DataType; use proc_macro2::Ident; use quote::quote; -use std::sync::Arc; pub const ROOT_NAME: &str = "ArroyoAvroRoot"; @@ -52,28 +51,20 @@ fn to_typedef(source_name: &str, schema: &Schema) -> (TypeDef, Option) { match schema { Schema::Null => (TypeDef::DataType(DataType::Null, false), None), Schema::Boolean => (TypeDef::DataType(DataType::Boolean, false), None), - Schema::Int => (TypeDef::DataType(DataType::Int32, false), None), - Schema::Long => (TypeDef::DataType(DataType::Int64, false), None), + Schema::Int | Schema::TimeMillis => (TypeDef::DataType(DataType::Int32, false), None), + Schema::Long + | Schema::TimeMicros + | Schema::TimestampMillis + | Schema::LocalTimestampMillis + | Schema::LocalTimestampMicros => (TypeDef::DataType(DataType::Int64, false), None), Schema::Float => (TypeDef::DataType(DataType::Float32, false), None), Schema::Double => (TypeDef::DataType(DataType::Float64, false), None), - Schema::Bytes => (TypeDef::DataType(DataType::Binary, false), None), - Schema::String => (TypeDef::DataType(DataType::Utf8, false), None), - // Schema::Array(t) => { - // let dt = match to_typedef(source_name, t) { - // (TypeDef::StructDef(sd, _), _) => { - // let fields: Vec = sd.fields.into_iter() - // .map(|f| f.into()) - // .collect(); - // - // DataType::Struct(Fields::from(fields)) - // }, - // (TypeDef::DataType(dt, _), _) => { - // dt - // } - // }; - // - // (TypeDef::DataType(DataType::List(Arc::new(Field::new("", dt,false))), false), None) - // } + Schema::Bytes | Schema::Fixed(_) | Schema::Decimal(_) => { + (TypeDef::DataType(DataType::Binary, false), None) + } + Schema::String | Schema::Enum(_) | Schema::Uuid => { + (TypeDef::DataType(DataType::Utf8, false), None) + } Schema::Union(union) => { // currently just support unions that have [t, null] as variants, which is the // avro way to represent optional fields @@ -117,18 +108,6 @@ fn to_typedef(source_name: &str, schema: &Schema) -> (TypeDef, Option) { _ => ( TypeDef::DataType(DataType::Utf8, false), Some("json".to_string()), - ), // Schema::Enum(_) => {} - // Schema::Fixed(_) => {} - // Schema::Decimal(_) => {} - // Schema::Uuid => {} - // Schema::Date => {} - // Schema::TimeMillis => {} - // Schema::TimeMicros => {} - // Schema::TimestampMillis => {} - // Schema::TimestampMicros => {} - // Schema::LocalTimestampMillis => {} - // Schema::LocalTimestampMicros => {} - // Schema::Duration => {} - // Schema::Ref { .. } => {} + ), } } diff --git a/arroyo-sql/src/types.rs b/arroyo-sql/src/types.rs index 058c949a6..af92321e6 100644 --- a/arroyo-sql/src/types.rs +++ b/arroyo-sql/src/types.rs @@ -748,7 +748,19 @@ impl StructField { } } } + } else if let Some("json") = self.original_type.as_ref().map(|i| i.as_str()) { + if self.nullable() { + attributes.push(quote!( + #[serde(default)] + #[serde(deserialize_with = "arroyo_worker::deserialize_raw_json_opt")] + )) + } else { + attributes.push(quote! { + #[serde(deserialize_with = "arroyo_worker::deserialize_raw_json")] + }) + } } + quote! { #(#attributes )* pub #name: #type_string diff --git a/arroyo-worker/src/formats/avro.rs b/arroyo-worker/src/formats/avro.rs index 3880a9b93..8d37b6cf7 100644 --- a/arroyo-worker/src/formats/avro.rs +++ b/arroyo-worker/src/formats/avro.rs @@ -1,5 +1,5 @@ use apache_avro::types::{Value as AvroValue, Value}; -use apache_avro::{from_avro_datum, Decimal, Reader, Schema}; +use apache_avro::{from_avro_datum, Reader, Schema}; use arroyo_rpc::formats::AvroFormat; use arroyo_rpc::schema_resolver::SchemaResolver; use arroyo_types::UserError; @@ -66,7 +66,7 @@ pub async fn deserialize_slice_avro<'a, T: DeserializeOwned>( } }; - let into_json = format.into_json; + let into_json = format.into_unstructured_json; Ok(messages.into_iter().map(move |record| { let value = record.map_err(|e| { UserError::new( @@ -78,7 +78,9 @@ pub async fn deserialize_slice_avro<'a, T: DeserializeOwned>( if into_json { Ok(serde_json::from_value(json!({"value": avro_to_json(value).to_string()})).unwrap()) } else { - apache_avro::from_value::(&value).map_err(|e| { + // for now round-trip through json in order to handle unsupported avro features + // as that allows us to rely on raw json deserialization + serde_json::from_value(avro_to_json(value)).map_err(|e| { UserError::new( "Deserialization failed", format!("Failed to convert avro message into struct type: {:?}", e), diff --git a/arroyo-worker/src/formats/json.rs b/arroyo-worker/src/formats/json.rs index 4a4706b8d..47bcfd511 100644 --- a/arroyo-worker/src/formats/json.rs +++ b/arroyo-worker/src/formats/json.rs @@ -176,7 +176,10 @@ pub mod timestamp_as_rfc3339 { D: Deserializer<'de>, { let raw: chrono::DateTime = DateTime::deserialize(deserializer)?; - Ok(from_nanos(raw.timestamp_nanos() as u128)) + Ok(from_nanos( + raw.timestamp_nanos_opt() + .expect("could not represent time as a number of nanoseconds") as u128, + )) } } @@ -204,7 +207,13 @@ pub mod opt_timestamp_as_rfc3339 { D: Deserializer<'de>, { let raw = Option::>::deserialize(deserializer)?; - Ok(raw.map(|raw| from_nanos(raw.timestamp_nanos() as u128))) + Ok(raw.map(|raw| { + from_nanos( + raw.timestamp_nanos_opt() + .expect("could not represent time as a number of nanoseconds") + as u128, + ) + })) } } diff --git a/arroyo-worker/src/formats/mod.rs b/arroyo-worker/src/formats/mod.rs index 08005452a..a858c8ef8 100644 --- a/arroyo-worker/src/formats/mod.rs +++ b/arroyo-worker/src/formats/mod.rs @@ -1,12 +1,11 @@ mod avro; pub mod json; -use apache_avro::{from_avro_datum, Reader, Schema}; +use apache_avro::Schema; use std::sync::Arc; use std::{collections::HashMap, marker::PhantomData}; -use arrow::datatypes::{Field, Fields}; -use arroyo_rpc::formats::{AvroFormat, Format, Framing, FramingMethod, JsonFormat}; +use arroyo_rpc::formats::{Format, Framing, FramingMethod}; use arroyo_rpc::schema_resolver::{FailingSchemaResolver, SchemaResolver}; use arroyo_types::UserError; use serde::de::DeserializeOwned; @@ -104,7 +103,7 @@ impl DataDeserializer { pub async fn deserialize_slice<'a>( &mut self, msg: &'a [u8], - ) -> Box> + 'a + Send> { + ) -> impl Iterator> + 'a + Send { match &*self.format { Format::Avro(avro) => { let schema_registry = self.schema_registry.clone(); @@ -115,7 +114,8 @@ impl DataDeserializer { Ok(iter) => Box::new(iter), Err(e) => Box::new( vec![Err(UserError::new("Avro deserialization failed", e))].into_iter(), - ), + ) + as Box> + Send>, } } _ => { @@ -123,7 +123,7 @@ impl DataDeserializer { Box::new( FramingIterator::new(self.framing.clone(), msg) .map(move |t| new_self.deserialize_single(t)), - ) + ) as Box> + Send> } } } @@ -195,15 +195,9 @@ impl DataSerializer { #[cfg(test)] mod tests { - use crate::formats::{DataDeserializer, FramingIterator}; - use crate::SchemaData; - use arroyo_rpc::formats::{ - AvroFormat, Format, Framing, FramingMethod, NewlineDelimitedFraming, - }; - use arroyo_rpc::schema_resolver::SchemaResolver; - use async_trait::async_trait; + use crate::formats::FramingIterator; + use arroyo_rpc::formats::{Framing, FramingMethod, NewlineDelimitedFraming}; use std::sync::Arc; - use std::time::SystemTime; #[test] fn test_line_framing() { From 3859c9148487a3d32fd6979af6eed1946ab18024 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Sun, 29 Oct 2023 18:31:04 -0700 Subject: [PATCH 11/12] formatted --- Cargo.lock | 2 + arroyo-api/Cargo.toml | 1 + arroyo-api/src/connection_tables.rs | 7 + arroyo-console/src/lib/data_fetching.ts | 2 +- .../connections/ConfluentSchemaEditor.tsx | 9 +- .../src/routes/connections/DefineSchema.tsx | 86 +++-- .../src/routes/connections/SchemaEditor.tsx | 12 +- arroyo-rpc/Cargo.toml | 1 + arroyo-rpc/src/formats.rs | 91 ++++- arroyo-rpc/src/schema_resolver.rs | 28 ++ arroyo-worker/src/formats/avro.rs | 343 ++++++++++++------ arroyo-worker/src/formats/mod.rs | 17 +- 12 files changed, 435 insertions(+), 164 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ec1508cda..1dc45307b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -429,6 +429,7 @@ name = "arroyo-api" version = "0.7.0" dependencies = [ "anyhow", + "apache-avro", "argon2", "arrow", "arrow-schema", @@ -659,6 +660,7 @@ name = "arroyo-rpc" version = "0.7.0" dependencies = [ "anyhow", + "apache-avro", "arroyo-types", "async-trait", "bincode 2.0.0-rc.3", diff --git a/arroyo-api/Cargo.toml b/arroyo-api/Cargo.toml index 9ce5e6361..3eaab4871 100644 --- a/arroyo-api/Cargo.toml +++ b/arroyo-api/Cargo.toml @@ -77,6 +77,7 @@ cornucopia_async = { version = "0.4", features = ["with-serde_json-1"] } jwt-simple = "0.11.4" uuid = "1.3.3" regress = "0.6.0" +apache-avro = "0.16.0" [build-dependencies] cornucopia = { version = "0.9" } diff --git a/arroyo-api/src/connection_tables.rs b/arroyo-api/src/connection_tables.rs index c6d43f95e..b13788f31 100644 --- a/arroyo-api/src/connection_tables.rs +++ b/arroyo-api/src/connection_tables.rs @@ -470,6 +470,13 @@ async fn expand_avro_schema( return Err(bad_request("avro format requires an avro schema be set")); }; + if let Some(Format::Avro(format)) = &mut schema.format { + format.add_reader_schema( + apache_avro::Schema::parse_str(&definition) + .map_err(|e| bad_request(format!("Avro schema is invalid: {:?}", e)))?, + ); + } + let fields: Result<_, String> = avro::convert_avro_schema(&name, &definition) .map_err(|e| bad_request(format!("Invalid avro schema: {}", e)))? .into_iter() diff --git a/arroyo-console/src/lib/data_fetching.ts b/arroyo-console/src/lib/data_fetching.ts index b0d2b1da2..69e32d0f7 100644 --- a/arroyo-console/src/lib/data_fetching.ts +++ b/arroyo-console/src/lib/data_fetching.ts @@ -140,7 +140,7 @@ const pingFetcher = async () => { export const usePing = () => { const { data, error, isLoading } = useSWR('ping', pingFetcher, { - refreshInterval: 3000000, + refreshInterval: 1000, onErrorRetry: (error, key, config, revalidate, {}) => { // explicitly define this function to override the exponential backoff setTimeout(() => revalidate(), 1000); diff --git a/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx b/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx index 49d465314..9e603dec2 100644 --- a/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx +++ b/arroyo-console/src/routes/connections/ConfluentSchemaEditor.tsx @@ -1,9 +1,6 @@ import { Dispatch } from 'react'; import { CreateConnectionState } from './CreateConnection'; -import { - Button, - Stack, Text, -} from '@chakra-ui/react'; +import { Button, Stack, Text } from '@chakra-ui/react'; export function ConfluentSchemaEditor({ state, @@ -16,9 +13,7 @@ export function ConfluentSchemaEditor({ }) { return ( - - Schemas will be loaded from the configured Confluent Schema Registry - + Schemas will be loaded from the configured Confluent Schema Registry