From af9b83c7126c177bd7ef9710999fddcde22e7d4d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 5 May 2022 13:32:01 +0800 Subject: [PATCH 01/15] feat(format): add sequential format --- common/exception/src/exception_code.rs | 2 + common/streams/src/lib.rs | 3 + common/streams/src/sources/cutter.rs | 13 ++ common/streams/src/sources/mod.rs | 4 + query/src/format/format.rs | 21 +++ query/src/format/format_factory.rs | 49 ++++++ query/src/format/mod.rs | 6 + query/src/lib.rs | 1 + query/src/servers/http/v1/load.rs | 25 ++- query/src/servers/http/v1/mod.rs | 1 + query/src/servers/http/v1/multipart_format.rs | 146 ++++++++++++++++++ 11 files changed, 256 insertions(+), 15 deletions(-) create mode 100644 common/streams/src/sources/cutter.rs create mode 100644 query/src/format/format.rs create mode 100644 query/src/format/format_factory.rs create mode 100644 query/src/format/mod.rs create mode 100644 query/src/servers/http/v1/multipart_format.rs diff --git a/common/exception/src/exception_code.rs b/common/exception/src/exception_code.rs index a07518770d087..bdb0de75ba847 100644 --- a/common/exception/src/exception_code.rs +++ b/common/exception/src/exception_code.rs @@ -139,6 +139,8 @@ build_exceptions! { // Network error codes. NetworkRequestError(1073), + UnknownFormat(1074), + // Tenant error codes. TenantIsEmpty(1101), IndexOutOfBounds(1102), diff --git a/common/streams/src/lib.rs b/common/streams/src/lib.rs index c5bd207c2fd54..5596dfb740819 100644 --- a/common/streams/src/lib.rs +++ b/common/streams/src/lib.rs @@ -41,3 +41,6 @@ pub use stream_sort::SortStream; pub use stream_source::SourceStream; pub use stream_sub_queries::SubQueriesStream; pub use stream_take::TakeStream; +pub use sources::Cutter; +pub use sources::InputState; +pub use sources::InputFormat; diff --git a/common/streams/src/sources/cutter.rs b/common/streams/src/sources/cutter.rs new file mode 100644 index 0000000000000..2c40a5681b70c --- /dev/null +++ b/common/streams/src/sources/cutter.rs @@ -0,0 +1,13 @@ +use common_datablocks::DataBlock; +use common_exception::Result; + +#[async_trait::async_trait] +pub trait Cutter { + async fn get_chunk(&mut self) -> Result>>; +} + +trait NewSource { + fn deserialize(&self, data: &[u8]) -> Result; +} + + diff --git a/common/streams/src/sources/mod.rs b/common/streams/src/sources/mod.rs index a87a44667be29..97d6d801c740f 100644 --- a/common/streams/src/sources/mod.rs +++ b/common/streams/src/sources/mod.rs @@ -16,6 +16,7 @@ mod source; mod source_csv; mod source_ndjson; mod source_parquet; +mod cutter; pub use source::Source; pub use source_csv::CsvSource; @@ -24,3 +25,6 @@ pub use source_ndjson::NDJsonSource; pub use source_ndjson::NDJsonSourceBuilder; pub use source_parquet::ParquetSource; pub use source_parquet::ParquetSourceBuilder; +pub use cutter::Cutter; +pub use databend_query::format::format::InputFormat; +pub use databend_query::format::format::InputState; diff --git a/query/src/format/format.rs b/query/src/format/format.rs new file mode 100644 index 0000000000000..084e2bbbd323f --- /dev/null +++ b/query/src/format/format.rs @@ -0,0 +1,21 @@ +use common_datablocks::DataBlock; +use common_datavalues::ColumnRef; +use common_exception::Result; + +pub trait InputState: Sized {} + +pub trait InputFormat { + fn support_parallel(&self) -> bool { + false + } + + fn create_state(&self) -> Box; + + fn deserialize_data(&self, state: &mut Box) -> Result; + + fn read_buf(&self, buf: &[u8], state: &mut Box) -> Result; +} + + + + diff --git a/query/src/format/format_factory.rs b/query/src/format/format_factory.rs new file mode 100644 index 0000000000000..0515170401a2b --- /dev/null +++ b/query/src/format/format_factory.rs @@ -0,0 +1,49 @@ +use std::collections::HashMap; +use std::sync::Arc; +use once_cell::sync::Lazy; +use crate::InputFormat; +use common_exception::{ErrorCode, Result}; +use crate::format::format::InputFormat; + +pub type InputFormatFactoryCreator = Box Result> + Send + Sync>; + +pub struct FormatFactory { + case_insensitive_desc: HashMap, +} + +static FORMAT_FACTORY: Lazy> = Lazy::new(|| { + let mut format_factory = FormatFactory::create(); + + Arc::new(format_factory) +}); + +impl FormatFactory { + pub(in crate::format::format_factory) fn create() -> FormatFactory { + FormatFactory { + case_insensitive_desc: Default::default(), + } + } + + pub fn instance() -> &'static FormatFactory { + FORMAT_FACTORY.as_ref() + } + + pub fn register_input(&mut self, name: &str, creator: InputFormatFactoryCreator) { + let case_insensitive_desc = &mut self.case_insensitive_desc; + case_insensitive_desc.insert(name.to_lowercase(), creator); + } + + pub fn get_input(&self, name: impl AsRef) -> Result> { + let origin_name = name.as_ref(); + let lowercase_name = origin_name.to_lowercase(); + + let creator = self + .case_insensitive_desc + .get(&lowercase_name) + .ok_or_else(|| ErrorCode::UnknownFormat(format!("Unsupported format: {}", origin_name)))?; + + + creator(&origin_name) + } +} + diff --git a/query/src/format/mod.rs b/query/src/format/mod.rs new file mode 100644 index 0000000000000..d1886b3cfcbba --- /dev/null +++ b/query/src/format/mod.rs @@ -0,0 +1,6 @@ +pub mod format; +mod format_factory; + +pub use format::InputState; +pub use format::InputFormat; +pub use format_factory::FormatFactory; diff --git a/query/src/lib.rs b/query/src/lib.rs index c48ee7aca3e2a..3c5d579d6354d 100644 --- a/query/src/lib.rs +++ b/query/src/lib.rs @@ -36,3 +36,4 @@ pub mod sql; pub mod storages; pub mod table_functions; pub mod users; +pub mod format; diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index 2bc435d30a434..107dbf6a5ab35 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -45,6 +45,7 @@ use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::StreamSourceV2; use crate::pipelines::new::SourcePipeBuilder; +use crate::servers::http::v1::multipart_format::MultipartFormat; use crate::sessions::QueryContext; use crate::sessions::SessionType; use crate::sql::PlanParser; @@ -121,7 +122,7 @@ pub async fn streaming_load( multipart, max_block_size, ) - .await + .await } else if format.to_lowercase().as_str() == "parquet" { parquet_source_pipe_builder(context.clone(), &plan, multipart).await } else if format.to_lowercase().as_str() == "ndjson" @@ -335,22 +336,16 @@ async fn csv_source_pipe_builder( mut multipart: Multipart, block_size: usize, ) -> PoemResult { - let mut builder = CsvSourceBuilder::create(plan.schema(), format_settings.clone()); - builder.block_size(block_size); + // let mut builder = CsvSourceBuilder::create(plan.schema(), format_settings.clone()); + // builder.block_size(block_size); + let ports = vec![OutputPort::create()]; let mut source_pipe_builder = SourcePipeBuilder::create(); - while let Ok(Some(field)) = multipart.next_field().await { - let bytes = field - .bytes() - .await - .map_err_to_code(ErrorCode::BadBytes, || "Read part to field bytes error") - .unwrap(); - let cursor = Cursor::new(bytes); - let csv_source = builder.build(cursor).unwrap(); - let output_port = OutputPort::create(); - let source = - StreamSourceV2::create(ctx.clone(), Box::new(csv_source), output_port.clone()).unwrap(); - source_pipe_builder.add_source(output_port, source); + let sources = MultipartFormat::input_sources("csv", multipart, ports.clone())?; + + for (index, source) in sources.into_iter().enumerate() { + source_pipe_builder.add_source(ports[index].clone(), source); } + Ok(source_pipe_builder) } diff --git a/query/src/servers/http/v1/mod.rs b/query/src/servers/http/v1/mod.rs index e9e438aea9ff0..e728b10ea7fe6 100644 --- a/query/src/servers/http/v1/mod.rs +++ b/query/src/servers/http/v1/mod.rs @@ -18,6 +18,7 @@ mod load; mod query; mod stage; mod statement; +mod multipart_format; pub use http_query_handlers::make_final_uri; pub use http_query_handlers::make_page_uri; diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs new file mode 100644 index 0000000000000..e591eb82b556e --- /dev/null +++ b/query/src/servers/http/v1/multipart_format.rs @@ -0,0 +1,146 @@ +use std::mem::replace; +use std::sync::Arc; +use poem::web::Multipart; +use sqlparser::ast::ShowCreateObject::Event; +use common_base::tokio::io::AsyncReadExt; +use common_base::tokio::sync::mpsc::{Receiver, Sender}; +use common_datablocks::DataBlock; +use common_exception::{ErrorCode, Result}; +use crate::format::{FormatFactory, InputFormat, InputState}; +use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::Processor; +use crate::pipelines::new::processors::processor::{Event, ProcessorPtr}; + + +pub struct MultipartFormat; + +impl MultipartFormat { + pub fn input_sources(name: &str, mut multipart: Multipart, ports: Vec>) -> Result> { + let input_format = FormatFactory::instance().get_input(name)?; + + if ports.len() != 1 || input_format.support_parallel() { + return Err(ErrorCode::UnImplement("Unimplemented parallel input format.")); + } + + let (tx, rx) = common_base::tokio::sync::mpsc::channel(2); + + common_base::tokio::spawn(async move { + while let Ok(Some(field)) = multipart.next_field().await { + let mut buf = vec![0; 2048]; + let mut async_reader = field.into_async_read(); + match async_reader.read(&mut buf[..]).await { + Ok(0) => { break; } + Ok(read_size) => { + if read_size != buf.len() { + buf = buf[0..read_size].to_vec(); + } + + if let Err(cause) = tx.send(Ok(buf)).await { + common_tracing::tracing::warn!("Multipart channel disconnect. {}", cause); + } + } + Err(cause) => { + if let Err(cause) = tx.send(Err(ErrorCode::BadBytes(format!("Read part to field bytes error, cause {:?}", cause)))) { + common_tracing::tracing::warn!("Multipart channel disconnect. {}", cause); + } + } + } + } + }); + + Ok(vec![SequentialInputFormatSource::create(ports[0].clone(), input_format, rx)?]) + } +} + +enum State { + NeedReceiveData, + ReceivedData(Vec), + NeedDeserialize, +} + +pub struct SequentialInputFormatSource { + state: State, + finished: bool, + output: Arc, + data_block: Option, + input_state: Box, + input_format: Box, + data_receiver: Receiver>>, +} + +impl SequentialInputFormatSource { + pub fn create(output: Arc, input_format: Box, data_receiver: Receiver>>) -> Result { + let input_state = input_format.create_state(); + Ok(ProcessorPtr::create(Box::new(SequentialInputFormatSource { + output, + input_state, + input_format, + data_receiver, + finished: false, + state: State::NeedReceiveData, + data_block: None, + }))) + } +} + +#[async_trait::async_trait] +impl Processor for SequentialInputFormatSource { + fn name(&self) -> &'static str { + "SequentialInputFormatSource" + } + + fn event(&mut self) -> Result { + if self.output.is_finished() { + return Ok(Event::Finished); + } + + if !self.output.can_push() { + return Ok(Event::NeedConsume); + } + + if let Some(data_block) = self.data_block.take() { + self.output.push_data(Ok(data_block)); + return Ok(Event::NeedConsume); + } + + if self.finished { + self.output.finish(); + return Ok(Event::Finished); + } + + match &self.state { + State::NeedReceiveData => Ok(Event::Async), + State::ReceivedData(_data) => Ok(Event::Sync), + State::NeedDeserialize => Ok(Event::Sync), + } + } + + fn process(&mut self) -> Result<()> { + if let State::ReceivedData(data) = replace(&mut self.state, State::NeedReceiveData) { + let read_size = self.input_format.read_buf(&data, &mut self.input_state)?; + + if read_size < data.len() { + self.state = State::NeedDeserialize; + } + } + + if let State::NeedDeserialize = replace(&mut self.state, State::NeedReceiveData) { + self.data_block = Some(self.input_format.deserialize_data(&mut self.input_state)?); + } + + Ok(()) + } + + async fn async_process(&mut self) -> Result<()> { + if let State::NeedReceiveData = std::mem::replace(&mut self.state, State::NeedReceiveData) { + if let Some(receive_res) = self.data_receiver.recv().await { + self.state = State::ReceivedData(receive_res?); + return Ok(()); + } + } + + self.finished = true; + self.state = State::NeedDeserialize; + Ok(()) + } +} From fc17b14d24046fee8b210c1c1d34e1da8fb0719d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 6 May 2022 00:02:48 +0800 Subject: [PATCH 02/15] feat(format): implement csv input format read_buf --- common/streams/src/lib.rs | 3 - common/streams/src/sources/cutter.rs | 13 -- common/streams/src/sources/mod.rs | 4 - query/src/format/format.rs | 7 +- query/src/format/format_csv.rs | 149 ++++++++++++++++++ query/src/format/format_factory.rs | 13 +- query/src/format/mod.rs | 1 + query/src/servers/http/v1/load.rs | 3 +- query/src/servers/http/v1/multipart_format.rs | 42 +++-- query/tests/it/format/format_csv.rs | 79 ++++++++++ query/tests/it/format/mod.rs | 1 + query/tests/it/main.rs | 1 + 12 files changed, 274 insertions(+), 42 deletions(-) delete mode 100644 common/streams/src/sources/cutter.rs create mode 100644 query/src/format/format_csv.rs create mode 100644 query/tests/it/format/format_csv.rs create mode 100644 query/tests/it/format/mod.rs diff --git a/common/streams/src/lib.rs b/common/streams/src/lib.rs index 5596dfb740819..c5bd207c2fd54 100644 --- a/common/streams/src/lib.rs +++ b/common/streams/src/lib.rs @@ -41,6 +41,3 @@ pub use stream_sort::SortStream; pub use stream_source::SourceStream; pub use stream_sub_queries::SubQueriesStream; pub use stream_take::TakeStream; -pub use sources::Cutter; -pub use sources::InputState; -pub use sources::InputFormat; diff --git a/common/streams/src/sources/cutter.rs b/common/streams/src/sources/cutter.rs deleted file mode 100644 index 2c40a5681b70c..0000000000000 --- a/common/streams/src/sources/cutter.rs +++ /dev/null @@ -1,13 +0,0 @@ -use common_datablocks::DataBlock; -use common_exception::Result; - -#[async_trait::async_trait] -pub trait Cutter { - async fn get_chunk(&mut self) -> Result>>; -} - -trait NewSource { - fn deserialize(&self, data: &[u8]) -> Result; -} - - diff --git a/common/streams/src/sources/mod.rs b/common/streams/src/sources/mod.rs index 97d6d801c740f..a87a44667be29 100644 --- a/common/streams/src/sources/mod.rs +++ b/common/streams/src/sources/mod.rs @@ -16,7 +16,6 @@ mod source; mod source_csv; mod source_ndjson; mod source_parquet; -mod cutter; pub use source::Source; pub use source_csv::CsvSource; @@ -25,6 +24,3 @@ pub use source_ndjson::NDJsonSource; pub use source_ndjson::NDJsonSourceBuilder; pub use source_parquet::ParquetSource; pub use source_parquet::ParquetSourceBuilder; -pub use cutter::Cutter; -pub use databend_query::format::format::InputFormat; -pub use databend_query::format::format::InputState; diff --git a/query/src/format/format.rs b/query/src/format/format.rs index 084e2bbbd323f..0e322130676a5 100644 --- a/query/src/format/format.rs +++ b/query/src/format/format.rs @@ -1,10 +1,13 @@ +use std::any::Any; use common_datablocks::DataBlock; use common_datavalues::ColumnRef; use common_exception::Result; -pub trait InputState: Sized {} +pub trait InputState: Send { + fn as_any(&mut self) -> &mut dyn Any; +} -pub trait InputFormat { +pub trait InputFormat: Send { fn support_parallel(&self) -> bool { false } diff --git a/query/src/format/format_csv.rs b/query/src/format/format_csv.rs new file mode 100644 index 0000000000000..036315481e89f --- /dev/null +++ b/query/src/format/format_csv.rs @@ -0,0 +1,149 @@ +use std::any::Any; +use std::io::Cursor; +use std::ops::Deref; +use common_arrow::arrow::io::csv; +use common_arrow::arrow::io::csv::read::ByteRecord; +use common_datablocks::DataBlock; +use common_datavalues::{DataSchemaRef, TypeDeserializerImpl}; +use crate::format::{FormatFactory, InputFormat, InputState}; +use common_exception::Result; +use common_io::prelude::FormatSettings; +use common_datavalues::DataType; + +pub struct CsvInputState { + pub quotes: bool, + pub memory: Vec, + pub accepted_rows: usize, + pub accepted_bytes: usize, + pub ignore_if_first_is_r: bool, + pub ignore_if_first_is_n: bool, +} + +impl InputState for CsvInputState { + fn as_any(&mut self) -> &mut dyn Any { + self + } +} + +pub struct CsvInputFormat { + schema: DataSchemaRef, + min_accepted_rows: usize, + min_accepted_bytes: usize, +} + +impl CsvInputFormat { + pub fn register(factory: &mut FormatFactory) { + factory.register_input("csv", Box::new(|name: &str, schema: DataSchemaRef, settings: FormatSettings| { + CsvInputFormat::try_create(name, schema, settings, 8192, 10 * 1024 * 1024) + })) + } + + pub fn try_create(name: &str, schema: DataSchemaRef, settings: FormatSettings, min_accepted_rows: usize, min_accepted_bytes: usize) -> Result> { + Ok(Box::new(CsvInputFormat { + schema, + min_accepted_rows, + min_accepted_bytes: min_accepted_bytes, + })) + } + + fn find_quotes(buf: &[u8], pos: usize, state: &mut CsvInputState) -> usize { + for index in pos..buf.len() { + if buf[index] == b'"' { + state.quotes = false; + return index + 1; + } + } + + buf.len() + } + + fn find_delimiter(&self, buf: &[u8], pos: usize, state: &mut CsvInputState, more_data: &mut bool) -> usize { + for index in pos..buf.len() { + match buf[index] { + b'"' => { + state.quotes = true; + return index + 1; + } + b'\r' => { + state.accepted_rows += 1; + if state.accepted_rows >= self.min_accepted_rows || (state.accepted_bytes + index) >= self.min_accepted_bytes { + *more_data = false; + } + + if buf.len() <= index + 1 { + state.ignore_if_first_is_n = true; + } else if buf[index + 1] == b'\n' { + return index + 2; + } + + return index + 1; + } + b'\n' => { + state.accepted_rows += 1; + if state.accepted_rows >= self.min_accepted_rows || (state.accepted_bytes + index) >= self.min_accepted_bytes { + *more_data = false; + } + + if buf.len() <= index + 1 { + state.ignore_if_first_is_r = true; + } else if buf[index + 1] == b'\r' { + return index + 2; + } + + return index + 1; + } + _ => { /*do nothing*/ } + } + } + + buf.len() + } +} + +impl InputFormat for CsvInputFormat { + fn create_state(&self) -> Box { + Box::new(CsvInputState { + quotes: false, + memory: vec![], + accepted_rows: 0, + accepted_bytes: 0, + ignore_if_first_is_r: false, + ignore_if_first_is_n: false, + }) + } + + fn deserialize_data(&self, state: &mut Box) -> Result { + for field in self.schema.fields() { + let data_type = field.data_type(); + let deserializer = data_type.create_deserializer(self.min_accepted_rows); + // deserializer + } + todo!() + } + + fn read_buf(&self, buf: &[u8], state: &mut Box) -> Result { + let mut index = 0; + let mut need_more_data = true; + let state = state.as_any().downcast_mut::().unwrap(); + + if state.ignore_if_first_is_r { + if buf[0] == b'\r' { + index += 1; + } + } else if state.ignore_if_first_is_n { + if buf[0] == b'\n' { + index += 1; + } + } + + while index < buf.len() && need_more_data { + index = match state.quotes { + true => Self::find_quotes(buf, index, state), + false => self.find_delimiter(buf, index, state, &mut need_more_data), + } + } + + state.memory.extend_from_slice(&buf[0..index]); + Ok(index) + } +} diff --git a/query/src/format/format_factory.rs b/query/src/format/format_factory.rs index 0515170401a2b..8d12a313d4879 100644 --- a/query/src/format/format_factory.rs +++ b/query/src/format/format_factory.rs @@ -1,11 +1,14 @@ use std::collections::HashMap; use std::sync::Arc; use once_cell::sync::Lazy; -use crate::InputFormat; +use common_datavalues::DataSchemaRef; use common_exception::{ErrorCode, Result}; +use common_io::prelude::FormatSettings; use crate::format::format::InputFormat; +use crate::format::format_csv::CsvInputFormat; +use crate::pipelines::processors::FormatterSettings; -pub type InputFormatFactoryCreator = Box Result> + Send + Sync>; +pub type InputFormatFactoryCreator = Box Result> + Send + Sync>; pub struct FormatFactory { case_insensitive_desc: HashMap, @@ -14,6 +17,8 @@ pub struct FormatFactory { static FORMAT_FACTORY: Lazy> = Lazy::new(|| { let mut format_factory = FormatFactory::create(); + CsvInputFormat::register(&mut format_factory); + Arc::new(format_factory) }); @@ -33,7 +38,7 @@ impl FormatFactory { case_insensitive_desc.insert(name.to_lowercase(), creator); } - pub fn get_input(&self, name: impl AsRef) -> Result> { + pub fn get_input(&self, name: impl AsRef, schema: DataSchemaRef, settings: FormatSettings) -> Result> { let origin_name = name.as_ref(); let lowercase_name = origin_name.to_lowercase(); @@ -43,7 +48,7 @@ impl FormatFactory { .ok_or_else(|| ErrorCode::UnknownFormat(format!("Unsupported format: {}", origin_name)))?; - creator(&origin_name) + creator(&origin_name, schema, settings) } } diff --git a/query/src/format/mod.rs b/query/src/format/mod.rs index d1886b3cfcbba..4fbdae213614b 100644 --- a/query/src/format/mod.rs +++ b/query/src/format/mod.rs @@ -1,5 +1,6 @@ pub mod format; mod format_factory; +pub mod format_csv; pub use format::InputState; pub use format::InputFormat; diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index 107dbf6a5ab35..41e4750d5ef55 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -336,11 +336,12 @@ async fn csv_source_pipe_builder( mut multipart: Multipart, block_size: usize, ) -> PoemResult { + /// CREATE FORMAT sk // let mut builder = CsvSourceBuilder::create(plan.schema(), format_settings.clone()); // builder.block_size(block_size); let ports = vec![OutputPort::create()]; let mut source_pipe_builder = SourcePipeBuilder::create(); - let sources = MultipartFormat::input_sources("csv", multipart, ports.clone())?; + let sources = MultipartFormat::input_sources("csv", multipart, plan.schema(), format_settings.clone(), ports.clone()).unwrap(); for (index, source) in sources.into_iter().enumerate() { source_pipe_builder.add_source(ports[index].clone(), source); diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index e591eb82b556e..f991fc49230ac 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -1,11 +1,12 @@ use std::mem::replace; use std::sync::Arc; use poem::web::Multipart; -use sqlparser::ast::ShowCreateObject::Event; use common_base::tokio::io::AsyncReadExt; use common_base::tokio::sync::mpsc::{Receiver, Sender}; use common_datablocks::DataBlock; +use common_datavalues::DataSchemaRef; use common_exception::{ErrorCode, Result}; +use common_io::prelude::FormatSettings; use crate::format::{FormatFactory, InputFormat, InputState}; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::Processor; @@ -15,8 +16,8 @@ use crate::pipelines::new::processors::processor::{Event, ProcessorPtr}; pub struct MultipartFormat; impl MultipartFormat { - pub fn input_sources(name: &str, mut multipart: Multipart, ports: Vec>) -> Result> { - let input_format = FormatFactory::instance().get_input(name)?; + pub fn input_sources(name: &str, mut multipart: Multipart, schema: DataSchemaRef, settings: FormatSettings, ports: Vec>) -> Result> { + let input_format = FormatFactory::instance().get_input(name, schema, settings)?; if ports.len() != 1 || input_format.support_parallel() { return Err(ErrorCode::UnImplement("Unimplemented parallel input format.")); @@ -40,7 +41,7 @@ impl MultipartFormat { } } Err(cause) => { - if let Err(cause) = tx.send(Err(ErrorCode::BadBytes(format!("Read part to field bytes error, cause {:?}", cause)))) { + if let Err(cause) = tx.send(Err(ErrorCode::BadBytes(format!("Read part to field bytes error, cause {:?}", cause)))).await { common_tracing::tracing::warn!("Multipart channel disconnect. {}", cause); } } @@ -62,7 +63,7 @@ pub struct SequentialInputFormatSource { state: State, finished: bool, output: Arc, - data_block: Option, + data_block: Vec, input_state: Box, input_format: Box, data_receiver: Receiver>>, @@ -78,7 +79,7 @@ impl SequentialInputFormatSource { data_receiver, finished: false, state: State::NeedReceiveData, - data_block: None, + data_block: vec![], }))) } } @@ -98,7 +99,7 @@ impl Processor for SequentialInputFormatSource { return Ok(Event::NeedConsume); } - if let Some(data_block) = self.data_block.take() { + if let Some(data_block) = self.data_block.pop() { self.output.push_data(Ok(data_block)); return Ok(Event::NeedConsume); } @@ -116,16 +117,27 @@ impl Processor for SequentialInputFormatSource { } fn process(&mut self) -> Result<()> { - if let State::ReceivedData(data) = replace(&mut self.state, State::NeedReceiveData) { - let read_size = self.input_format.read_buf(&data, &mut self.input_state)?; + match replace(&mut self.state, State::NeedReceiveData) { + State::ReceivedData(data) => { + let mut data_slice: &[u8] = &data; - if read_size < data.len() { - self.state = State::NeedDeserialize; - } - } + while !data_slice.is_empty() { + let len = data_slice.len(); + let read_size = self.input_format.read_buf(data_slice, &mut self.input_state)?; - if let State::NeedDeserialize = replace(&mut self.state, State::NeedReceiveData) { - self.data_block = Some(self.input_format.deserialize_data(&mut self.input_state)?); + data_slice = &data_slice[read_size..]; + + if read_size < len { + self.data_block.push(self.input_format.deserialize_data(&mut self.input_state)?); + } + } + } + State::NeedDeserialize => { + self.data_block.push(self.input_format.deserialize_data(&mut self.input_state)?); + } + _ => { + return Err(ErrorCode::LogicalError("State failure in Multipart format.")); + } } Ok(()) diff --git a/query/tests/it/format/format_csv.rs b/query/tests/it/format/format_csv.rs new file mode 100644 index 0000000000000..994d1b02aa0c0 --- /dev/null +++ b/query/tests/it/format/format_csv.rs @@ -0,0 +1,79 @@ +use std::sync::Arc; +use nom::AsBytes; +use common_datavalues::DataSchema; +use common_exception::Result; +use common_io::prelude::FormatSettings; +use databend_query::format::format_csv::CsvInputState; +use databend_query::format::format_csv::CsvInputFormat; +use databend_query::format::{InputFormat, InputState}; + +#[test] +fn test_accepted_multi_lines() -> Result<()> { + assert_complete_line("")?; + assert_complete_line("first\tsecond\n")?; + assert_complete_line("first\tsecond\r")?; + assert_complete_line("first\tsecond\r\n")?; + assert_complete_line("first\tsecond\n\r")?; + assert_complete_line("first\t\"\n\"second\n")?; + assert_complete_line("first\t\"\r\"second\n")?; + + assert_broken_line("first", 5)?; + assert_broken_line("first\t", 6)?; + assert_broken_line("first\ts", 7)?; + assert_broken_line("first\ts\"\n", 9)?; + assert_broken_line("first\ts\"\r", 9)?; + assert_broken_line("first\tsecond\ns", 13)?; + + let csv_input_format = CsvInputFormat::try_create( + "csv", + Arc::new(DataSchema::empty()), + FormatSettings::default(), + 2, + 10 * 1024 * 1024, + )?; + + let mut csv_input_state = csv_input_format.create_state(); + + let bytes = "first\tsecond\nfirst\t".as_bytes(); + assert_eq!(bytes.len(), csv_input_format.read_buf(bytes, &mut csv_input_state)?); + assert_eq!(bytes, &csv_input_state.as_any().downcast_mut::().unwrap().memory); + + let bytes = "second\nfirst\t".as_bytes(); + assert_eq!(7, csv_input_format.read_buf(bytes, &mut csv_input_state)?); + assert_eq!("first\tsecond\nfirst\tsecond\n".as_bytes(), csv_input_state.as_any().downcast_mut::().unwrap().memory); + Ok(()) +} + +fn assert_complete_line(content: &str) -> Result<()> { + let csv_input_format = CsvInputFormat::try_create( + "csv", + Arc::new(DataSchema::empty()), + FormatSettings::default(), + 1, + 10 * 1024 * 1024, + )?; + + let mut csv_input_state = csv_input_format.create_state(); + + let bytes = content.as_bytes(); + assert_eq!(bytes.len(), csv_input_format.read_buf(bytes, &mut csv_input_state)?); + assert_eq!(bytes, &csv_input_state.as_any().downcast_mut::().unwrap().memory); + Ok(()) +} + +fn assert_broken_line(content: &str, assert_size: usize) -> Result<()> { + let csv_input_format = CsvInputFormat::try_create( + "csv", + Arc::new(DataSchema::empty()), + FormatSettings::default(), + 1, + 10 * 1024 * 1024, + )?; + + let mut csv_input_state = csv_input_format.create_state(); + + let bytes = content.as_bytes(); + assert_eq!(assert_size, csv_input_format.read_buf(bytes, &mut csv_input_state)?); + assert_eq!(&bytes[0..assert_size], &csv_input_state.as_any().downcast_mut::().unwrap().memory); + Ok(()) +} diff --git a/query/tests/it/format/mod.rs b/query/tests/it/format/mod.rs new file mode 100644 index 0000000000000..e9860824687c9 --- /dev/null +++ b/query/tests/it/format/mod.rs @@ -0,0 +1 @@ +mod format_csv; \ No newline at end of file diff --git a/query/tests/it/main.rs b/query/tests/it/main.rs index c1b3ec9555b3c..85aec3b5ab4ea 100644 --- a/query/tests/it/main.rs +++ b/query/tests/it/main.rs @@ -28,3 +28,4 @@ mod storages; mod table_functions; mod tests; mod users; +mod format; From 1be2b591c0d06c663ab8d02d361518b5e6c626ca Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 6 May 2022 13:46:14 +0800 Subject: [PATCH 03/15] feat(format): implement csv format --- .../src/types/deserializations/boolean.rs | 16 +-- .../src/types/deserializations/number.rs | 8 +- .../src/types/deserializations/string.rs | 14 +-- .../src/types/deserializations/timestamp.rs | 20 +-- .../src/types/deserializations/variant.rs | 12 +- common/io/src/buffer/buffer_read_ext.rs | 5 + query/src/format/format.rs | 5 +- query/src/format/format_csv.rs | 115 ++++++++++++++--- query/src/format/format_factory.rs | 23 ++-- query/src/format/mod.rs | 4 +- query/src/lib.rs | 2 +- query/src/servers/http/v1/load.rs | 11 +- query/src/servers/http/v1/mod.rs | 2 +- query/src/servers/http/v1/multipart_format.rs | 98 +++++++++++---- query/tests/it/format/format_csv.rs | 118 ++++++++++++++++-- query/tests/it/format/mod.rs | 2 +- query/tests/it/main.rs | 2 +- 17 files changed, 352 insertions(+), 105 deletions(-) diff --git a/common/datavalues/src/types/deserializations/boolean.rs b/common/datavalues/src/types/deserializations/boolean.rs index 0185d5fa4e981..eaec852dfdee3 100644 --- a/common/datavalues/src/types/deserializations/boolean.rs +++ b/common/datavalues/src/types/deserializations/boolean.rs @@ -43,6 +43,14 @@ impl TypeDeserializer for BooleanDeserializer { Ok(()) } + fn de_json(&mut self, value: &serde_json::Value) -> Result<()> { + match value { + serde_json::Value::Bool(v) => self.builder.append_value(*v), + _ => return Err(ErrorCode::BadBytes("Incorrect boolean value")), + } + Ok(()) + } + fn de_whole_text(&mut self, reader: &[u8]) -> Result<()> { if reader.eq_ignore_ascii_case(b"true") { self.builder.append_value(true); @@ -67,14 +75,6 @@ impl TypeDeserializer for BooleanDeserializer { Ok(()) } - fn de_json(&mut self, value: &serde_json::Value) -> Result<()> { - match value { - serde_json::Value::Bool(v) => self.builder.append_value(*v), - _ => return Err(ErrorCode::BadBytes("Incorrect boolean value")), - } - Ok(()) - } - fn append_data_value(&mut self, value: DataValue) -> Result<()> { self.builder.append_value(value.as_bool()?); Ok(()) diff --git a/common/datavalues/src/types/deserializations/number.rs b/common/datavalues/src/types/deserializations/number.rs index 6f5064e59b1c3..95c5b01a0cf40 100644 --- a/common/datavalues/src/types/deserializations/number.rs +++ b/common/datavalues/src/types/deserializations/number.rs @@ -65,6 +65,10 @@ where } } + fn de_null(&mut self) -> bool { + false + } + fn de_whole_text(&mut self, reader: &[u8]) -> Result<()> { let mut reader = BufferReader::new(reader); let v: T = if !T::FLOATING { @@ -88,10 +92,6 @@ where Ok(()) } - fn de_null(&mut self) -> bool { - false - } - fn append_data_value(&mut self, value: DataValue) -> Result<()> { self.builder.append_data_value(value) } diff --git a/common/datavalues/src/types/deserializations/string.rs b/common/datavalues/src/types/deserializations/string.rs index f71e268b920f6..5d6318fb1948f 100644 --- a/common/datavalues/src/types/deserializations/string.rs +++ b/common/datavalues/src/types/deserializations/string.rs @@ -73,13 +73,6 @@ impl TypeDeserializer for StringDeserializer { } } - fn de_text_quoted(&mut self, reader: &mut CheckpointReader) -> Result<()> { - self.buffer.clear(); - reader.read_quoted_text(&mut self.buffer, b'\'')?; - self.builder.append_value(self.buffer.as_slice()); - Ok(()) - } - fn de_whole_text(&mut self, reader: &[u8]) -> Result<()> { self.builder.append_value(reader); Ok(()) @@ -92,6 +85,13 @@ impl TypeDeserializer for StringDeserializer { Ok(()) } + fn de_text_quoted(&mut self, reader: &mut CheckpointReader) -> Result<()> { + self.buffer.clear(); + reader.read_quoted_text(&mut self.buffer, b'\'')?; + self.builder.append_value(self.buffer.as_slice()); + Ok(()) + } + fn append_data_value(&mut self, value: DataValue) -> Result<()> { self.builder.append_data_value(value) } diff --git a/common/datavalues/src/types/deserializations/timestamp.rs b/common/datavalues/src/types/deserializations/timestamp.rs index d33abcd80804b..7a91e6e27ef8d 100644 --- a/common/datavalues/src/types/deserializations/timestamp.rs +++ b/common/datavalues/src/types/deserializations/timestamp.rs @@ -64,16 +64,6 @@ impl TypeDeserializer for TimestampDeserializer { } } - fn de_text_quoted(&mut self, reader: &mut CheckpointReader) -> Result<()> { - reader.must_ignore_byte(b'\'')?; - let ts = reader.read_timestamp_text(&self.tz)?; - let micros = ts.timestamp_micros(); - let _ = check_timestamp(micros)?; - reader.must_ignore_byte(b'\'')?; - self.builder.append_value(micros.as_()); - Ok(()) - } - fn de_whole_text(&mut self, reader: &[u8]) -> Result<()> { let mut reader = BufferReader::new(reader); let ts = reader.read_timestamp_text(&self.tz)?; @@ -115,6 +105,16 @@ impl TypeDeserializer for TimestampDeserializer { Ok(()) } + fn de_text_quoted(&mut self, reader: &mut CheckpointReader) -> Result<()> { + reader.must_ignore_byte(b'\'')?; + let ts = reader.read_timestamp_text(&self.tz)?; + let micros = ts.timestamp_micros(); + let _ = check_timestamp(micros)?; + reader.must_ignore_byte(b'\'')?; + self.builder.append_value(micros.as_()); + Ok(()) + } + fn append_data_value(&mut self, value: DataValue) -> Result<()> { let v = value.as_i64()?; let _ = check_timestamp(v)?; diff --git a/common/datavalues/src/types/deserializations/variant.rs b/common/datavalues/src/types/deserializations/variant.rs index d84e5de6cf33b..a60d3399733a9 100644 --- a/common/datavalues/src/types/deserializations/variant.rs +++ b/common/datavalues/src/types/deserializations/variant.rs @@ -69,16 +69,16 @@ impl TypeDeserializer for VariantDeserializer { Ok(()) } - fn de_text(&mut self, reader: &mut CheckpointReader) -> Result<()> { - self.buffer.clear(); - reader.read_escaped_string_text(&mut self.buffer)?; - let val = serde_json::from_slice(self.buffer.as_slice())?; + fn de_whole_text(&mut self, reader: &[u8]) -> Result<()> { + let val = serde_json::from_slice(reader)?; self.builder.append_value(val); Ok(()) } - fn de_whole_text(&mut self, reader: &[u8]) -> Result<()> { - let val = serde_json::from_slice(reader)?; + fn de_text(&mut self, reader: &mut CheckpointReader) -> Result<()> { + self.buffer.clear(); + reader.read_escaped_string_text(&mut self.buffer)?; + let val = serde_json::from_slice(self.buffer.as_slice())?; self.builder.append_value(val); Ok(()) } diff --git a/common/io/src/buffer/buffer_read_ext.rs b/common/io/src/buffer/buffer_read_ext.rs index ff6df93366838..9160f8907163b 100644 --- a/common/io/src/buffer/buffer_read_ext.rs +++ b/common/io/src/buffer/buffer_read_ext.rs @@ -55,6 +55,11 @@ pub trait BufferReadExt: BufferRead { Ok(()) } + fn eof(&mut self) -> Result { + let buffer = self.fill_buf()?; + Ok(buffer.is_empty()) + } + fn must_eof(&mut self) -> Result<()> { let buffer = self.fill_buf()?; if !buffer.is_empty() { diff --git a/query/src/format/format.rs b/query/src/format/format.rs index 0e322130676a5..f1e1035ae9662 100644 --- a/query/src/format/format.rs +++ b/query/src/format/format.rs @@ -1,4 +1,5 @@ use std::any::Any; + use common_datablocks::DataBlock; use common_datavalues::ColumnRef; use common_exception::Result; @@ -18,7 +19,3 @@ pub trait InputFormat: Send { fn read_buf(&self, buf: &[u8], state: &mut Box) -> Result; } - - - - diff --git a/query/src/format/format_csv.rs b/query/src/format/format_csv.rs index 036315481e89f..a6d9f73394268 100644 --- a/query/src/format/format_csv.rs +++ b/query/src/format/format_csv.rs @@ -1,14 +1,25 @@ use std::any::Any; use std::io::Cursor; use std::ops::Deref; + use common_arrow::arrow::io::csv; use common_arrow::arrow::io::csv::read::ByteRecord; use common_datablocks::DataBlock; -use common_datavalues::{DataSchemaRef, TypeDeserializerImpl}; -use crate::format::{FormatFactory, InputFormat, InputState}; +use common_datavalues::DataSchemaRef; +use common_datavalues::DataType; +use common_datavalues::TypeDeserializer; +use common_datavalues::TypeDeserializerImpl; +use common_exception::ErrorCode; use common_exception::Result; +use common_io::prelude::BufferRead; +use common_io::prelude::BufferReadExt; +use common_io::prelude::BufferReader; +use common_io::prelude::CheckpointReader; use common_io::prelude::FormatSettings; -use common_datavalues::DataType; + +use crate::format::FormatFactory; +use crate::format::InputFormat; +use crate::format::InputState; pub struct CsvInputState { pub quotes: bool, @@ -33,16 +44,46 @@ pub struct CsvInputFormat { impl CsvInputFormat { pub fn register(factory: &mut FormatFactory) { - factory.register_input("csv", Box::new(|name: &str, schema: DataSchemaRef, settings: FormatSettings| { - CsvInputFormat::try_create(name, schema, settings, 8192, 10 * 1024 * 1024) - })) + factory.register_input( + "csv", + Box::new( + |name: &str, schema: DataSchemaRef, settings: FormatSettings| { + CsvInputFormat::try_create(name, schema, settings, 8192, 10 * 1024 * 1024) + }, + ), + ) } - pub fn try_create(name: &str, schema: DataSchemaRef, settings: FormatSettings, min_accepted_rows: usize, min_accepted_bytes: usize) -> Result> { + pub fn try_create( + name: &str, + schema: DataSchemaRef, + settings: FormatSettings, + min_accepted_rows: usize, + min_accepted_bytes: usize, + ) -> Result> { + // let field_delimiter = match settings.field_delimiter.len() { + // n if n >= 1 => settings.field_delimiter[0], + // _ => b',', + // }; + // + // let record_delimiter = match settings.record_delimiter.len() { + // n if n >= 1 => settings.record_delimiter[0], + // _ => b'\n', + // }; + + // let record_delimiter = if record_delimiter == b'\n' || record_delimiter == b'\r' { + // Terminator::CRLF + // } else { + // Terminator::Any(record_delimiter) + // }; + + // let skip_header = settings.skip_header; + // let empty_as_default = settings.empty_as_default; + Ok(Box::new(CsvInputFormat { schema, min_accepted_rows, - min_accepted_bytes: min_accepted_bytes, + min_accepted_bytes, })) } @@ -57,7 +98,13 @@ impl CsvInputFormat { buf.len() } - fn find_delimiter(&self, buf: &[u8], pos: usize, state: &mut CsvInputState, more_data: &mut bool) -> usize { + fn find_delimiter( + &self, + buf: &[u8], + pos: usize, + state: &mut CsvInputState, + more_data: &mut bool, + ) -> usize { for index in pos..buf.len() { match buf[index] { b'"' => { @@ -66,7 +113,9 @@ impl CsvInputFormat { } b'\r' => { state.accepted_rows += 1; - if state.accepted_rows >= self.min_accepted_rows || (state.accepted_bytes + index) >= self.min_accepted_bytes { + if state.accepted_rows >= self.min_accepted_rows + || (state.accepted_bytes + index) >= self.min_accepted_bytes + { *more_data = false; } @@ -80,7 +129,9 @@ impl CsvInputFormat { } b'\n' => { state.accepted_rows += 1; - if state.accepted_rows >= self.min_accepted_rows || (state.accepted_bytes + index) >= self.min_accepted_bytes { + if state.accepted_rows >= self.min_accepted_rows + || (state.accepted_bytes + index) >= self.min_accepted_bytes + { *more_data = false; } @@ -113,12 +164,48 @@ impl InputFormat for CsvInputFormat { } fn deserialize_data(&self, state: &mut Box) -> Result { + let mut deserializers = Vec::with_capacity(self.schema.num_fields()); for field in self.schema.fields() { let data_type = field.data_type(); - let deserializer = data_type.create_deserializer(self.min_accepted_rows); - // deserializer + deserializers.push(data_type.create_deserializer(self.min_accepted_rows)); } - todo!() + + let state = state.as_any().downcast_mut::().unwrap(); + let cursor = Cursor::new(&state.memory); + let reader: Box = Box::new(BufferReader::new(cursor)); + let mut checkpoint_reader = CheckpointReader::new(reader); + + for row_index in 0..self.min_accepted_rows { + if checkpoint_reader.eof()? { + break; + } + + for column_index in 0..deserializers.len() { + if checkpoint_reader.ignore_byte(b'\t')? { + deserializers[column_index].de_default(); + } else { + deserializers[column_index].de_text_csv(&mut checkpoint_reader)?; + + if column_index + 1 != deserializers.len() { + checkpoint_reader.must_ignore_byte(b'\t')?; + } + } + } + + if !checkpoint_reader.ignore_byte(b'\n')? & !checkpoint_reader.ignore_byte(b'\r')? { + return Err(ErrorCode::BadBytes(format!( + "Parse csv error at line {}", + row_index + ))); + } + } + + let mut columns = Vec::with_capacity(deserializers.len()); + for deserializer in &mut deserializers { + columns.push(deserializer.finish_to_column()); + } + + Ok(DataBlock::create(self.schema.clone(), columns)) } fn read_buf(&self, buf: &[u8], state: &mut Box) -> Result { diff --git a/query/src/format/format_factory.rs b/query/src/format/format_factory.rs index 8d12a313d4879..3a18c9a3baf54 100644 --- a/query/src/format/format_factory.rs +++ b/query/src/format/format_factory.rs @@ -1,14 +1,18 @@ use std::collections::HashMap; use std::sync::Arc; -use once_cell::sync::Lazy; + use common_datavalues::DataSchemaRef; -use common_exception::{ErrorCode, Result}; +use common_exception::ErrorCode; +use common_exception::Result; use common_io::prelude::FormatSettings; +use once_cell::sync::Lazy; + use crate::format::format::InputFormat; use crate::format::format_csv::CsvInputFormat; use crate::pipelines::processors::FormatterSettings; -pub type InputFormatFactoryCreator = Box Result> + Send + Sync>; +pub type InputFormatFactoryCreator = + Box Result> + Send + Sync>; pub struct FormatFactory { case_insensitive_desc: HashMap, @@ -38,17 +42,22 @@ impl FormatFactory { case_insensitive_desc.insert(name.to_lowercase(), creator); } - pub fn get_input(&self, name: impl AsRef, schema: DataSchemaRef, settings: FormatSettings) -> Result> { + pub fn get_input( + &self, + name: impl AsRef, + schema: DataSchemaRef, + settings: FormatSettings, + ) -> Result> { let origin_name = name.as_ref(); let lowercase_name = origin_name.to_lowercase(); let creator = self .case_insensitive_desc .get(&lowercase_name) - .ok_or_else(|| ErrorCode::UnknownFormat(format!("Unsupported format: {}", origin_name)))?; - + .ok_or_else(|| { + ErrorCode::UnknownFormat(format!("Unsupported format: {}", origin_name)) + })?; creator(&origin_name, schema, settings) } } - diff --git a/query/src/format/mod.rs b/query/src/format/mod.rs index 4fbdae213614b..0b74610c094b0 100644 --- a/query/src/format/mod.rs +++ b/query/src/format/mod.rs @@ -1,7 +1,7 @@ pub mod format; -mod format_factory; pub mod format_csv; +mod format_factory; -pub use format::InputState; pub use format::InputFormat; +pub use format::InputState; pub use format_factory::FormatFactory; diff --git a/query/src/lib.rs b/query/src/lib.rs index 3c5d579d6354d..b8a94aba49e31 100644 --- a/query/src/lib.rs +++ b/query/src/lib.rs @@ -25,6 +25,7 @@ pub mod clusters; pub mod common; pub mod configs; pub mod databases; +pub mod format; pub mod interpreters; pub mod metrics; pub mod optimizers; @@ -36,4 +37,3 @@ pub mod sql; pub mod storages; pub mod table_functions; pub mod users; -pub mod format; diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index 41e4750d5ef55..c5f42c4b00f11 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -122,7 +122,7 @@ pub async fn streaming_load( multipart, max_block_size, ) - .await + .await } else if format.to_lowercase().as_str() == "parquet" { parquet_source_pipe_builder(context.clone(), &plan, multipart).await } else if format.to_lowercase().as_str() == "ndjson" @@ -341,7 +341,14 @@ async fn csv_source_pipe_builder( // builder.block_size(block_size); let ports = vec![OutputPort::create()]; let mut source_pipe_builder = SourcePipeBuilder::create(); - let sources = MultipartFormat::input_sources("csv", multipart, plan.schema(), format_settings.clone(), ports.clone()).unwrap(); + let sources = MultipartFormat::input_sources( + "csv", + multipart, + plan.schema(), + format_settings.clone(), + ports.clone(), + ) + .unwrap(); for (index, source) in sources.into_iter().enumerate() { source_pipe_builder.add_source(ports[index].clone(), source); diff --git a/query/src/servers/http/v1/mod.rs b/query/src/servers/http/v1/mod.rs index e728b10ea7fe6..1dcd3a51b1558 100644 --- a/query/src/servers/http/v1/mod.rs +++ b/query/src/servers/http/v1/mod.rs @@ -15,10 +15,10 @@ mod http_query_handlers; pub mod json_block; mod load; +mod multipart_format; mod query; mod stage; mod statement; -mod multipart_format; pub use http_query_handlers::make_final_uri; pub use http_query_handlers::make_page_uri; diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index f991fc49230ac..191c8145700c1 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -1,26 +1,40 @@ use std::mem::replace; use std::sync::Arc; -use poem::web::Multipart; + use common_base::tokio::io::AsyncReadExt; -use common_base::tokio::sync::mpsc::{Receiver, Sender}; +use common_base::tokio::sync::mpsc::Receiver; +use common_base::tokio::sync::mpsc::Sender; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; -use common_exception::{ErrorCode, Result}; +use common_exception::ErrorCode; +use common_exception::Result; use common_io::prelude::FormatSettings; -use crate::format::{FormatFactory, InputFormat, InputState}; +use poem::web::Multipart; + +use crate::format::FormatFactory; +use crate::format::InputFormat; +use crate::format::InputState; use crate::pipelines::new::processors::port::OutputPort; +use crate::pipelines::new::processors::processor::Event; +use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::Processor; -use crate::pipelines::new::processors::processor::{Event, ProcessorPtr}; - pub struct MultipartFormat; impl MultipartFormat { - pub fn input_sources(name: &str, mut multipart: Multipart, schema: DataSchemaRef, settings: FormatSettings, ports: Vec>) -> Result> { + pub fn input_sources( + name: &str, + mut multipart: Multipart, + schema: DataSchemaRef, + settings: FormatSettings, + ports: Vec>, + ) -> Result> { let input_format = FormatFactory::instance().get_input(name, schema, settings)?; if ports.len() != 1 || input_format.support_parallel() { - return Err(ErrorCode::UnImplement("Unimplemented parallel input format.")); + return Err(ErrorCode::UnImplement( + "Unimplemented parallel input format.", + )); } let (tx, rx) = common_base::tokio::sync::mpsc::channel(2); @@ -30,26 +44,44 @@ impl MultipartFormat { let mut buf = vec![0; 2048]; let mut async_reader = field.into_async_read(); match async_reader.read(&mut buf[..]).await { - Ok(0) => { break; } + Ok(0) => { + break; + } Ok(read_size) => { if read_size != buf.len() { buf = buf[0..read_size].to_vec(); } if let Err(cause) = tx.send(Ok(buf)).await { - common_tracing::tracing::warn!("Multipart channel disconnect. {}", cause); + common_tracing::tracing::warn!( + "Multipart channel disconnect. {}", + cause + ); } } Err(cause) => { - if let Err(cause) = tx.send(Err(ErrorCode::BadBytes(format!("Read part to field bytes error, cause {:?}", cause)))).await { - common_tracing::tracing::warn!("Multipart channel disconnect. {}", cause); + if let Err(cause) = tx + .send(Err(ErrorCode::BadBytes(format!( + "Read part to field bytes error, cause {:?}", + cause + )))) + .await + { + common_tracing::tracing::warn!( + "Multipart channel disconnect. {}", + cause + ); } } } } }); - Ok(vec![SequentialInputFormatSource::create(ports[0].clone(), input_format, rx)?]) + Ok(vec![SequentialInputFormatSource::create( + ports[0].clone(), + input_format, + rx, + )?]) } } @@ -70,17 +102,23 @@ pub struct SequentialInputFormatSource { } impl SequentialInputFormatSource { - pub fn create(output: Arc, input_format: Box, data_receiver: Receiver>>) -> Result { + pub fn create( + output: Arc, + input_format: Box, + data_receiver: Receiver>>, + ) -> Result { let input_state = input_format.create_state(); - Ok(ProcessorPtr::create(Box::new(SequentialInputFormatSource { - output, - input_state, - input_format, - data_receiver, - finished: false, - state: State::NeedReceiveData, - data_block: vec![], - }))) + Ok(ProcessorPtr::create(Box::new( + SequentialInputFormatSource { + output, + input_state, + input_format, + data_receiver, + finished: false, + state: State::NeedReceiveData, + data_block: vec![], + }, + ))) } } @@ -123,20 +161,26 @@ impl Processor for SequentialInputFormatSource { while !data_slice.is_empty() { let len = data_slice.len(); - let read_size = self.input_format.read_buf(data_slice, &mut self.input_state)?; + let read_size = self + .input_format + .read_buf(data_slice, &mut self.input_state)?; data_slice = &data_slice[read_size..]; if read_size < len { - self.data_block.push(self.input_format.deserialize_data(&mut self.input_state)?); + self.data_block + .push(self.input_format.deserialize_data(&mut self.input_state)?); } } } State::NeedDeserialize => { - self.data_block.push(self.input_format.deserialize_data(&mut self.input_state)?); + self.data_block + .push(self.input_format.deserialize_data(&mut self.input_state)?); } _ => { - return Err(ErrorCode::LogicalError("State failure in Multipart format.")); + return Err(ErrorCode::LogicalError( + "State failure in Multipart format.", + )); } } diff --git a/query/tests/it/format/format_csv.rs b/query/tests/it/format/format_csv.rs index 994d1b02aa0c0..e12d159b3c21f 100644 --- a/query/tests/it/format/format_csv.rs +++ b/query/tests/it/format/format_csv.rs @@ -1,11 +1,18 @@ use std::sync::Arc; -use nom::AsBytes; + +use common_datablocks::assert_blocks_eq; +use common_datavalues::type_primitive::UInt32Type; +use common_datavalues::DataField; use common_datavalues::DataSchema; +use common_datavalues::DataTypeImpl; +use common_datavalues::StringType; use common_exception::Result; use common_io::prelude::FormatSettings; -use databend_query::format::format_csv::CsvInputState; use databend_query::format::format_csv::CsvInputFormat; -use databend_query::format::{InputFormat, InputState}; +use databend_query::format::format_csv::CsvInputState; +use databend_query::format::InputFormat; +use databend_query::format::InputState; +use nom::AsBytes; #[test] fn test_accepted_multi_lines() -> Result<()> { @@ -35,12 +42,83 @@ fn test_accepted_multi_lines() -> Result<()> { let mut csv_input_state = csv_input_format.create_state(); let bytes = "first\tsecond\nfirst\t".as_bytes(); - assert_eq!(bytes.len(), csv_input_format.read_buf(bytes, &mut csv_input_state)?); - assert_eq!(bytes, &csv_input_state.as_any().downcast_mut::().unwrap().memory); + assert_eq!( + bytes.len(), + csv_input_format.read_buf(bytes, &mut csv_input_state)? + ); + assert_eq!( + bytes, + &csv_input_state + .as_any() + .downcast_mut::() + .unwrap() + .memory + ); let bytes = "second\nfirst\t".as_bytes(); assert_eq!(7, csv_input_format.read_buf(bytes, &mut csv_input_state)?); - assert_eq!("first\tsecond\nfirst\tsecond\n".as_bytes(), csv_input_state.as_any().downcast_mut::().unwrap().memory); + assert_eq!( + "first\tsecond\nfirst\tsecond\n".as_bytes(), + csv_input_state + .as_any() + .downcast_mut::() + .unwrap() + .memory + ); + Ok(()) +} + +#[test] +fn test_deserialize_multi_lines() -> Result<()> { + let csv_input_format = CsvInputFormat::try_create( + "csv", + Arc::new(DataSchema::new(vec![ + DataField::new("a", DataTypeImpl::UInt32(UInt32Type::default())), + DataField::new("b", DataTypeImpl::String(StringType::default())), + ])), + FormatSettings::default(), + 1, + 10 * 1024 * 1024, + )?; + + let mut csv_input_state = csv_input_format.create_state(); + + csv_input_format.read_buf("1\t\"second\"\n".as_bytes(), &mut csv_input_state)?; + assert_blocks_eq( + vec![ + "+---+----------+", + "| a | b |", + "+---+----------+", + "| 1 | \"second\" |", + "+---+----------+", + ], + &[csv_input_format.deserialize_data(&mut csv_input_state)?], + ); + + let csv_input_format = CsvInputFormat::try_create( + "csv", + Arc::new(DataSchema::new(vec![ + DataField::new("a", DataTypeImpl::UInt32(UInt32Type::default())), + DataField::new("b", DataTypeImpl::String(StringType::default())), + ])), + FormatSettings::default(), + 2, + 10 * 1024 * 1024, + )?; + + let mut csv_input_state = csv_input_format.create_state(); + + csv_input_format.read_buf("1\t\"second\"\n".as_bytes(), &mut csv_input_state)?; + assert_blocks_eq( + vec![ + "+---+----------+", + "| a | b |", + "+---+----------+", + "| 1 | \"second\" |", + "+---+----------+", + ], + &[csv_input_format.deserialize_data(&mut csv_input_state)?], + ); Ok(()) } @@ -56,8 +134,18 @@ fn assert_complete_line(content: &str) -> Result<()> { let mut csv_input_state = csv_input_format.create_state(); let bytes = content.as_bytes(); - assert_eq!(bytes.len(), csv_input_format.read_buf(bytes, &mut csv_input_state)?); - assert_eq!(bytes, &csv_input_state.as_any().downcast_mut::().unwrap().memory); + assert_eq!( + bytes.len(), + csv_input_format.read_buf(bytes, &mut csv_input_state)? + ); + assert_eq!( + bytes, + &csv_input_state + .as_any() + .downcast_mut::() + .unwrap() + .memory + ); Ok(()) } @@ -73,7 +161,17 @@ fn assert_broken_line(content: &str, assert_size: usize) -> Result<()> { let mut csv_input_state = csv_input_format.create_state(); let bytes = content.as_bytes(); - assert_eq!(assert_size, csv_input_format.read_buf(bytes, &mut csv_input_state)?); - assert_eq!(&bytes[0..assert_size], &csv_input_state.as_any().downcast_mut::().unwrap().memory); + assert_eq!( + assert_size, + csv_input_format.read_buf(bytes, &mut csv_input_state)? + ); + assert_eq!( + &bytes[0..assert_size], + &csv_input_state + .as_any() + .downcast_mut::() + .unwrap() + .memory + ); Ok(()) } diff --git a/query/tests/it/format/mod.rs b/query/tests/it/format/mod.rs index e9860824687c9..dc85b85bb3460 100644 --- a/query/tests/it/format/mod.rs +++ b/query/tests/it/format/mod.rs @@ -1 +1 @@ -mod format_csv; \ No newline at end of file +mod format_csv; diff --git a/query/tests/it/main.rs b/query/tests/it/main.rs index 85aec3b5ab4ea..cbe32beb3edbf 100644 --- a/query/tests/it/main.rs +++ b/query/tests/it/main.rs @@ -16,6 +16,7 @@ mod catalogs; mod clusters; mod common; mod configs; +mod format; mod functions; mod interpreters; mod metrics; @@ -28,4 +29,3 @@ mod storages; mod table_functions; mod tests; mod users; -mod format; From b419dc3b1207b94d0a0d43895abf79a4c899a297 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 6 May 2022 13:52:21 +0800 Subject: [PATCH 04/15] feat(format): apply code style --- query/src/{format => formats}/format.rs | 1 - query/src/{format => formats}/format_csv.rs | 24 +++++++------------ .../src/{format => formats}/format_factory.rs | 11 ++++----- query/src/{format => formats}/mod.rs | 0 query/src/lib.rs | 2 +- query/src/servers/http/v1/load.rs | 9 +++---- query/src/servers/http/v1/multipart_format.rs | 7 +++--- .../it/{format => formats}/format_csv.rs | 7 ++---- query/tests/it/{format => formats}/mod.rs | 0 query/tests/it/main.rs | 2 +- 10 files changed, 24 insertions(+), 39 deletions(-) rename query/src/{format => formats}/format.rs (93%) rename query/src/{format => formats}/format_csv.rs (93%) rename query/src/{format => formats}/format_factory.rs (84%) rename query/src/{format => formats}/mod.rs (100%) rename query/tests/it/{format => formats}/format_csv.rs (95%) rename query/tests/it/{format => formats}/mod.rs (100%) diff --git a/query/src/format/format.rs b/query/src/formats/format.rs similarity index 93% rename from query/src/format/format.rs rename to query/src/formats/format.rs index f1e1035ae9662..11f93222d3797 100644 --- a/query/src/format/format.rs +++ b/query/src/formats/format.rs @@ -1,7 +1,6 @@ use std::any::Any; use common_datablocks::DataBlock; -use common_datavalues::ColumnRef; use common_exception::Result; pub trait InputState: Send { diff --git a/query/src/format/format_csv.rs b/query/src/formats/format_csv.rs similarity index 93% rename from query/src/format/format_csv.rs rename to query/src/formats/format_csv.rs index a6d9f73394268..1c597318210bd 100644 --- a/query/src/format/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -1,14 +1,10 @@ use std::any::Any; use std::io::Cursor; -use std::ops::Deref; -use common_arrow::arrow::io::csv; -use common_arrow::arrow::io::csv::read::ByteRecord; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_datavalues::DataType; use common_datavalues::TypeDeserializer; -use common_datavalues::TypeDeserializerImpl; use common_exception::ErrorCode; use common_exception::Result; use common_io::prelude::BufferRead; @@ -17,9 +13,9 @@ use common_io::prelude::BufferReader; use common_io::prelude::CheckpointReader; use common_io::prelude::FormatSettings; -use crate::format::FormatFactory; -use crate::format::InputFormat; -use crate::format::InputState; +use crate::formats::FormatFactory; +use crate::formats::InputFormat; +use crate::formats::InputState; pub struct CsvInputState { pub quotes: bool, @@ -55,9 +51,9 @@ impl CsvInputFormat { } pub fn try_create( - name: &str, + _name: &str, schema: DataSchemaRef, - settings: FormatSettings, + _settings: FormatSettings, min_accepted_rows: usize, min_accepted_bytes: usize, ) -> Result> { @@ -88,8 +84,8 @@ impl CsvInputFormat { } fn find_quotes(buf: &[u8], pos: usize, state: &mut CsvInputState) -> usize { - for index in pos..buf.len() { - if buf[index] == b'"' { + for (index, byte) in buf.iter().enumerate().skip(pos) { + if *byte == b'"' { state.quotes = false; return index + 1; } @@ -217,10 +213,8 @@ impl InputFormat for CsvInputFormat { if buf[0] == b'\r' { index += 1; } - } else if state.ignore_if_first_is_n { - if buf[0] == b'\n' { - index += 1; - } + } else if state.ignore_if_first_is_n && buf[0] == b'\n' { + index += 1; } while index < buf.len() && need_more_data { diff --git a/query/src/format/format_factory.rs b/query/src/formats/format_factory.rs similarity index 84% rename from query/src/format/format_factory.rs rename to query/src/formats/format_factory.rs index 3a18c9a3baf54..9a27051181167 100644 --- a/query/src/format/format_factory.rs +++ b/query/src/formats/format_factory.rs @@ -7,9 +7,8 @@ use common_exception::Result; use common_io::prelude::FormatSettings; use once_cell::sync::Lazy; -use crate::format::format::InputFormat; -use crate::format::format_csv::CsvInputFormat; -use crate::pipelines::processors::FormatterSettings; +use crate::formats::format::InputFormat; +use crate::formats::format_csv::CsvInputFormat; pub type InputFormatFactoryCreator = Box Result> + Send + Sync>; @@ -27,7 +26,7 @@ static FORMAT_FACTORY: Lazy> = Lazy::new(|| { }); impl FormatFactory { - pub(in crate::format::format_factory) fn create() -> FormatFactory { + pub(in crate::formats::format_factory) fn create() -> FormatFactory { FormatFactory { case_insensitive_desc: Default::default(), } @@ -55,9 +54,9 @@ impl FormatFactory { .case_insensitive_desc .get(&lowercase_name) .ok_or_else(|| { - ErrorCode::UnknownFormat(format!("Unsupported format: {}", origin_name)) + ErrorCode::UnknownFormat(format!("Unsupported formats: {}", origin_name)) })?; - creator(&origin_name, schema, settings) + creator(origin_name, schema, settings) } } diff --git a/query/src/format/mod.rs b/query/src/formats/mod.rs similarity index 100% rename from query/src/format/mod.rs rename to query/src/formats/mod.rs diff --git a/query/src/lib.rs b/query/src/lib.rs index b8a94aba49e31..d76db2afdd84c 100644 --- a/query/src/lib.rs +++ b/query/src/lib.rs @@ -25,7 +25,7 @@ pub mod clusters; pub mod common; pub mod configs; pub mod databases; -pub mod format; +pub mod formats; pub mod interpreters; pub mod metrics; pub mod optimizers; diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index c5f42c4b00f11..a89b8481aa29d 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -330,15 +330,12 @@ fn build_csv_stream( } async fn csv_source_pipe_builder( - ctx: Arc, + _ctx: Arc, plan: &PlanNode, format_settings: &FormatSettings, - mut multipart: Multipart, - block_size: usize, + multipart: Multipart, + _block_size: usize, ) -> PoemResult { - /// CREATE FORMAT sk - // let mut builder = CsvSourceBuilder::create(plan.schema(), format_settings.clone()); - // builder.block_size(block_size); let ports = vec![OutputPort::create()]; let mut source_pipe_builder = SourcePipeBuilder::create(); let sources = MultipartFormat::input_sources( diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index 191c8145700c1..748592691a348 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -3,7 +3,6 @@ use std::sync::Arc; use common_base::tokio::io::AsyncReadExt; use common_base::tokio::sync::mpsc::Receiver; -use common_base::tokio::sync::mpsc::Sender; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; @@ -11,9 +10,9 @@ use common_exception::Result; use common_io::prelude::FormatSettings; use poem::web::Multipart; -use crate::format::FormatFactory; -use crate::format::InputFormat; -use crate::format::InputState; +use crate::formats::FormatFactory; +use crate::formats::InputFormat; +use crate::formats::InputState; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::Event; use crate::pipelines::new::processors::processor::ProcessorPtr; diff --git a/query/tests/it/format/format_csv.rs b/query/tests/it/formats/format_csv.rs similarity index 95% rename from query/tests/it/format/format_csv.rs rename to query/tests/it/formats/format_csv.rs index e12d159b3c21f..826c6056b6a88 100644 --- a/query/tests/it/format/format_csv.rs +++ b/query/tests/it/formats/format_csv.rs @@ -8,11 +8,8 @@ use common_datavalues::DataTypeImpl; use common_datavalues::StringType; use common_exception::Result; use common_io::prelude::FormatSettings; -use databend_query::format::format_csv::CsvInputFormat; -use databend_query::format::format_csv::CsvInputState; -use databend_query::format::InputFormat; -use databend_query::format::InputState; -use nom::AsBytes; +use databend_query::formats::format_csv::CsvInputFormat; +use databend_query::formats::format_csv::CsvInputState; #[test] fn test_accepted_multi_lines() -> Result<()> { diff --git a/query/tests/it/format/mod.rs b/query/tests/it/formats/mod.rs similarity index 100% rename from query/tests/it/format/mod.rs rename to query/tests/it/formats/mod.rs diff --git a/query/tests/it/main.rs b/query/tests/it/main.rs index cbe32beb3edbf..41670b9fa564a 100644 --- a/query/tests/it/main.rs +++ b/query/tests/it/main.rs @@ -16,7 +16,7 @@ mod catalogs; mod clusters; mod common; mod configs; -mod format; +mod formats; mod functions; mod interpreters; mod metrics; From 94288b46bf9b23e662866c938e46edef803b4f10 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 6 May 2022 14:17:17 +0800 Subject: [PATCH 05/15] feat(format): add file license header --- query/src/formats/format.rs | 14 ++++++++++++++ query/src/formats/format_csv.rs | 14 ++++++++++++++ query/src/formats/format_factory.rs | 14 ++++++++++++++ query/src/formats/mod.rs | 14 ++++++++++++++ query/src/servers/http/v1/multipart_format.rs | 14 ++++++++++++++ query/tests/it/formats/format_csv.rs | 14 ++++++++++++++ query/tests/it/formats/mod.rs | 14 ++++++++++++++ 7 files changed, 98 insertions(+) diff --git a/query/src/formats/format.rs b/query/src/formats/format.rs index 11f93222d3797..7b2347c4295a4 100644 --- a/query/src/formats/format.rs +++ b/query/src/formats/format.rs @@ -1,3 +1,17 @@ +// Copyright 2022 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::any::Any; use common_datablocks::DataBlock; diff --git a/query/src/formats/format_csv.rs b/query/src/formats/format_csv.rs index 1c597318210bd..815bb59508d97 100644 --- a/query/src/formats/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -1,3 +1,17 @@ +// Copyright 2022 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::any::Any; use std::io::Cursor; diff --git a/query/src/formats/format_factory.rs b/query/src/formats/format_factory.rs index 9a27051181167..e13fc8ac1fa8a 100644 --- a/query/src/formats/format_factory.rs +++ b/query/src/formats/format_factory.rs @@ -1,3 +1,17 @@ +// Copyright 2022 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::collections::HashMap; use std::sync::Arc; diff --git a/query/src/formats/mod.rs b/query/src/formats/mod.rs index 0b74610c094b0..9bcafc2008c1c 100644 --- a/query/src/formats/mod.rs +++ b/query/src/formats/mod.rs @@ -1,3 +1,17 @@ +// Copyright 2022 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + pub mod format; pub mod format_csv; mod format_factory; diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index 748592691a348..e8e4a173f9e51 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -1,3 +1,17 @@ +// Copyright 2022 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::mem::replace; use std::sync::Arc; diff --git a/query/tests/it/formats/format_csv.rs b/query/tests/it/formats/format_csv.rs index 826c6056b6a88..b20dabc37c0cb 100644 --- a/query/tests/it/formats/format_csv.rs +++ b/query/tests/it/formats/format_csv.rs @@ -1,3 +1,17 @@ +// Copyright 2022 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::sync::Arc; use common_datablocks::assert_blocks_eq; diff --git a/query/tests/it/formats/mod.rs b/query/tests/it/formats/mod.rs index dc85b85bb3460..0e948f332fc7e 100644 --- a/query/tests/it/formats/mod.rs +++ b/query/tests/it/formats/mod.rs @@ -1 +1,15 @@ +// Copyright 2022 Datafuse Labs. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + mod format_csv; From 130d3fa0b791044dc9a3305f94a4a2c3af16f358 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 7 May 2022 16:45:13 +0800 Subject: [PATCH 06/15] feat(format): support csv format --- .../src/types/deserializations/number.rs | 22 +- .../src/types/deserializations/string.rs | 7 + .../src/types/deserializations/variant.rs | 9 + common/io/src/buffer/buffer_read_ext.rs | 28 ++- query/src/formats/format.rs | 2 + query/src/formats/format_csv.rs | 192 ++++++++++-------- query/src/interpreters/interpreter_insert.rs | 4 +- query/src/servers/http/v1/load.rs | 116 ++++++++--- query/src/servers/http/v1/multipart_format.rs | 145 +++++++++---- 9 files changed, 360 insertions(+), 165 deletions(-) diff --git a/common/datavalues/src/types/deserializations/number.rs b/common/datavalues/src/types/deserializations/number.rs index 95c5b01a0cf40..d70f903e1ee1c 100644 --- a/common/datavalues/src/types/deserializations/number.rs +++ b/common/datavalues/src/types/deserializations/number.rs @@ -24,9 +24,9 @@ pub struct NumberDeserializer { } impl TypeDeserializer for NumberDeserializer -where - T: PrimitiveType, - T: Unmarshal + StatBuffer + FromLexical, + where + T: PrimitiveType, + T: Unmarshal + StatBuffer + FromLexical, { fn de_binary(&mut self, reader: &mut &[u8]) -> Result<()> { let value: T = reader.read_scalar()?; @@ -92,6 +92,22 @@ where Ok(()) } + fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; + let v: T = if !T::FLOATING { + reader.read_int_text() + } else { + reader.read_float_text() + }?; + + if maybe_quote { + reader.must_ignore(|f| f == b'\'' || f == b'"')?; + } + + self.builder.append_value(v); + Ok(()) + } + fn append_data_value(&mut self, value: DataValue) -> Result<()> { self.builder.append_data_value(value) } diff --git a/common/datavalues/src/types/deserializations/string.rs b/common/datavalues/src/types/deserializations/string.rs index 5d6318fb1948f..92bc51470c910 100644 --- a/common/datavalues/src/types/deserializations/string.rs +++ b/common/datavalues/src/types/deserializations/string.rs @@ -92,6 +92,13 @@ impl TypeDeserializer for StringDeserializer { Ok(()) } + fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + self.buffer.clear(); + reader.read_quoted_text(&mut self.buffer, b'"')?; + self.builder.append_value(self.buffer.as_slice()); + Ok(()) + } + fn append_data_value(&mut self, value: DataValue) -> Result<()> { self.builder.append_data_value(value) } diff --git a/common/datavalues/src/types/deserializations/variant.rs b/common/datavalues/src/types/deserializations/variant.rs index a60d3399733a9..7c921dfb3fba0 100644 --- a/common/datavalues/src/types/deserializations/variant.rs +++ b/common/datavalues/src/types/deserializations/variant.rs @@ -92,6 +92,15 @@ impl TypeDeserializer for VariantDeserializer { Ok(()) } + fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + self.buffer.clear(); + reader.read_quoted_text(&mut self.buffer, b'"')?; + + let val = serde_json::from_slice(self.buffer.as_slice())?; + self.builder.append_value(val); + Ok(()) + } + fn append_data_value(&mut self, value: DataValue) -> Result<()> { self.builder.append_data_value(value) } diff --git a/common/io/src/buffer/buffer_read_ext.rs b/common/io/src/buffer/buffer_read_ext.rs index 9160f8907163b..f549c1febd9ef 100644 --- a/common/io/src/buffer/buffer_read_ext.rs +++ b/common/io/src/buffer/buffer_read_ext.rs @@ -24,6 +24,7 @@ pub trait BufferReadExt: BufferRead { fn ignore_bytes(&mut self, bs: &[u8]) -> Result; fn ignore_insensitive_bytes(&mut self, bs: &[u8]) -> Result; fn ignore_white_spaces(&mut self) -> Result; + fn ignore_white_spaces_and_byte(&mut self, b: u8) -> Result; fn until(&mut self, delim: u8, buf: &mut Vec) -> Result; fn keep_read(&mut self, buf: &mut Vec, f: impl Fn(u8) -> bool) -> Result; @@ -83,9 +84,21 @@ pub trait BufferReadExt: BufferRead { fn must_ignore_byte(&mut self, b: u8) -> Result<()> { if !self.ignore_byte(b)? { + let buf = self.fill_buf()?; return Err(std::io::Error::new( ErrorKind::InvalidData, - format!("Expected to have char {}", b as char), + format!("Expected to have char {}, {}", b as char, buf[0] as char), + )); + } + Ok(()) + } + + fn must_ignore_white_spaces_and_byte(&mut self, b: u8) -> Result<()> { + if !self.ignore_white_spaces_and_byte(b)? { + let buf = self.fill_buf()?; + return Err(std::io::Error::new( + ErrorKind::InvalidData, + format!("Expected to have char {}, {}", b as char, buf[0] as char), )); } Ok(()) @@ -113,7 +126,7 @@ pub trait BufferReadExt: BufferRead { } impl BufferReadExt for R -where R: BufferRead + where R: BufferRead { fn ignores(&mut self, f: impl Fn(u8) -> bool) -> Result { let mut bytes = 0; @@ -177,6 +190,17 @@ where R: BufferRead Ok(cnt > 0) } + fn ignore_white_spaces_and_byte(&mut self, b: u8) -> Result { + self.ignores(|c: u8| c == b' ' || c == b'\t')?; + + if self.ignore_byte(b)? { + self.ignores(|c: u8| c == b' ' || c == b'\t')?; + return Ok(true); + } + + Ok(false) + } + fn until(&mut self, delim: u8, buf: &mut Vec) -> Result { self.read_until(delim, buf) } diff --git a/query/src/formats/format.rs b/query/src/formats/format.rs index 7b2347c4295a4..9a7507a7555e0 100644 --- a/query/src/formats/format.rs +++ b/query/src/formats/format.rs @@ -31,4 +31,6 @@ pub trait InputFormat: Send { fn deserialize_data(&self, state: &mut Box) -> Result; fn read_buf(&self, buf: &[u8], state: &mut Box) -> Result; + + fn skip_header(&self, buf: &[u8], state: &mut Box) -> Result; } diff --git a/query/src/formats/format_csv.rs b/query/src/formats/format_csv.rs index 815bb59508d97..d690f0affb6e4 100644 --- a/query/src/formats/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::any::Any; -use std::io::Cursor; +use std::io::{Cursor, Read}; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; @@ -21,7 +21,7 @@ use common_datavalues::DataType; use common_datavalues::TypeDeserializer; use common_exception::ErrorCode; use common_exception::Result; -use common_io::prelude::BufferRead; +use common_io::prelude::{BufferRead, CpBufferReader}; use common_io::prelude::BufferReadExt; use common_io::prelude::BufferReader; use common_io::prelude::CheckpointReader; @@ -36,8 +36,8 @@ pub struct CsvInputState { pub memory: Vec, pub accepted_rows: usize, pub accepted_bytes: usize, - pub ignore_if_first_is_r: bool, - pub ignore_if_first_is_n: bool, + pub need_more_data: bool, + pub ignore_if_first: Option, } impl InputState for CsvInputState { @@ -48,50 +48,49 @@ impl InputState for CsvInputState { pub struct CsvInputFormat { schema: DataSchemaRef, + field_delimiter: u8, + need_skip_header: bool, + row_delimiter: Option, min_accepted_rows: usize, min_accepted_bytes: usize, } impl CsvInputFormat { pub fn register(factory: &mut FormatFactory) { - factory.register_input( - "csv", - Box::new( - |name: &str, schema: DataSchemaRef, settings: FormatSettings| { - CsvInputFormat::try_create(name, schema, settings, 8192, 10 * 1024 * 1024) - }, - ), - ) + factory.register_input("csv", Box::new( + |name: &str, schema: DataSchemaRef, settings: FormatSettings| { + CsvInputFormat::try_create(name, schema, settings, 8192, 10 * 1024 * 1024) + }, + )) } pub fn try_create( _name: &str, schema: DataSchemaRef, - _settings: FormatSettings, + settings: FormatSettings, min_accepted_rows: usize, min_accepted_bytes: usize, ) -> Result> { - // let field_delimiter = match settings.field_delimiter.len() { - // n if n >= 1 => settings.field_delimiter[0], - // _ => b',', - // }; - // - // let record_delimiter = match settings.record_delimiter.len() { - // n if n >= 1 => settings.record_delimiter[0], - // _ => b'\n', - // }; - - // let record_delimiter = if record_delimiter == b'\n' || record_delimiter == b'\r' { - // Terminator::CRLF - // } else { - // Terminator::Any(record_delimiter) - // }; - - // let skip_header = settings.skip_header; - // let empty_as_default = settings.empty_as_default; + let field_delimiter = match settings.field_delimiter.len() { + n if n >= 1 => settings.field_delimiter[0], + _ => b',', + }; + + let mut row_delimiter = None; + + if !settings.record_delimiter.is_empty() + && settings.record_delimiter[0] != b'\n' + && settings.record_delimiter[0] != b'\r' { + row_delimiter = Some(settings.record_delimiter[0]); + } + + let need_skip_header = settings.skip_header; Ok(Box::new(CsvInputFormat { schema, + row_delimiter, + field_delimiter, + need_skip_header, min_accepted_rows, min_accepted_bytes, })) @@ -108,56 +107,49 @@ impl CsvInputFormat { buf.len() } - fn find_delimiter( - &self, - buf: &[u8], - pos: usize, - state: &mut CsvInputState, - more_data: &mut bool, - ) -> usize { + fn find_delimiter(&self, buf: &[u8], pos: usize, state: &mut CsvInputState) -> usize { for index in pos..buf.len() { - match buf[index] { - b'"' => { - state.quotes = true; - return index + 1; + if buf[index] == b'"' { + state.quotes = true; + return index + 1; + } + + if let Some(b) = &self.row_delimiter { + if buf[index] == *b { + return self.accept_row::<0>(buf, pos, state, index); } - b'\r' => { - state.accepted_rows += 1; - if state.accepted_rows >= self.min_accepted_rows - || (state.accepted_bytes + index) >= self.min_accepted_bytes - { - *more_data = false; - } + } else { + if buf[index] == b'\r' { + return self.accept_row::(buf, pos, state, index); + } else if buf[index] == b'\n' { + return self.accept_row::(buf, pos, state, index); + } + } + } - if buf.len() <= index + 1 { - state.ignore_if_first_is_n = true; - } else if buf[index + 1] == b'\n' { - return index + 2; - } + buf.len() + } - return index + 1; - } - b'\n' => { - state.accepted_rows += 1; - if state.accepted_rows >= self.min_accepted_rows - || (state.accepted_bytes + index) >= self.min_accepted_bytes - { - *more_data = false; - } + #[inline(always)] + fn accept_row(&self, buf: &[u8], pos: usize, state: &mut CsvInputState, index: usize) -> usize { + state.accepted_rows += 1; + state.accepted_bytes += (index - pos); - if buf.len() <= index + 1 { - state.ignore_if_first_is_r = true; - } else if buf[index + 1] == b'\r' { - return index + 2; - } + if state.accepted_rows >= self.min_accepted_rows + || (state.accepted_bytes + index) >= self.min_accepted_bytes + { + state.need_more_data = false; + } - return index + 1; - } - _ => { /*do nothing*/ } + if C != 0 { + if buf.len() <= index + 1 { + state.ignore_if_first = Some(C); + } else if buf[index + 1] == C { + return index + 2; } } - buf.len() + return index + 1; } } @@ -168,8 +160,8 @@ impl InputFormat for CsvInputFormat { memory: vec![], accepted_rows: 0, accepted_bytes: 0, - ignore_if_first_is_r: false, - ignore_if_first_is_n: false, + need_more_data: false, + ignore_if_first: None, }) } @@ -190,19 +182,32 @@ impl InputFormat for CsvInputFormat { break; } + println!("deserializers len {}", deserializers.len()); for column_index in 0..deserializers.len() { - if checkpoint_reader.ignore_byte(b'\t')? { + if checkpoint_reader.ignore_white_spaces_and_byte(self.field_delimiter)? { deserializers[column_index].de_default(); } else { deserializers[column_index].de_text_csv(&mut checkpoint_reader)?; + println!("deserializers index {} len {}", column_index, deserializers.len()); if column_index + 1 != deserializers.len() { - checkpoint_reader.must_ignore_byte(b'\t')?; + checkpoint_reader.must_ignore_white_spaces_and_byte(self.field_delimiter)?; + let buffer = checkpoint_reader.fill_buf()?; + println!("expect field delimiter. {}, {}, {}", column_index, deserializers.len(), buffer[0] as char); } } } - if !checkpoint_reader.ignore_byte(b'\n')? & !checkpoint_reader.ignore_byte(b'\r')? { + checkpoint_reader.ignore_white_spaces_and_byte(self.field_delimiter)?; + + if let Some(delimiter) = &self.row_delimiter { + if !checkpoint_reader.ignore_white_spaces_and_byte(*delimiter)? { + return Err(ErrorCode::BadBytes(format!( + "Parse csv error at line {}", + row_index + ))); + } + } else if !checkpoint_reader.ignore_white_spaces_and_byte(b'\n')? & !checkpoint_reader.ignore_white_spaces_and_byte(b'\r')? { return Err(ErrorCode::BadBytes(format!( "Parse csv error at line {}", row_index @@ -220,25 +225,44 @@ impl InputFormat for CsvInputFormat { fn read_buf(&self, buf: &[u8], state: &mut Box) -> Result { let mut index = 0; - let mut need_more_data = true; let state = state.as_any().downcast_mut::().unwrap(); - if state.ignore_if_first_is_r { - if buf[0] == b'\r' { + if let Some(first) = state.ignore_if_first.take() { + if buf[0] == first { index += 1; } - } else if state.ignore_if_first_is_n && buf[0] == b'\n' { - index += 1; } - while index < buf.len() && need_more_data { + state.need_more_data = true; + while index < buf.len() && state.need_more_data { index = match state.quotes { true => Self::find_quotes(buf, index, state), - false => self.find_delimiter(buf, index, state, &mut need_more_data), + false => self.find_delimiter(buf, index, state), } } state.memory.extend_from_slice(&buf[0..index]); Ok(index) } + + fn skip_header(&self, buf: &[u8], state: &mut Box) -> Result { + if self.need_skip_header { + let mut index = 0; + let state = state.as_any().downcast_mut::().unwrap(); + + while index < buf.len() { + index = match state.quotes { + true => Self::find_quotes(buf, index, state), + false => self.find_delimiter(buf, index, state), + }; + + if state.accepted_rows == 1 { + println!("skip header {}", index); + return Ok(index); + } + } + } + + Ok(0) + } } diff --git a/query/src/interpreters/interpreter_insert.rs b/query/src/interpreters/interpreter_insert.rs index 724bc2ab24170..c2d1e8cf92792 100644 --- a/query/src/interpreters/interpreter_insert.rs +++ b/query/src/interpreters/interpreter_insert.rs @@ -280,8 +280,8 @@ impl Interpreter for InsertInterpreter { } fn create_new_pipeline(&self) -> Result { - let new_pipeline = NewPipeline::create(); - Ok(new_pipeline) + let insert_pipeline = NewPipeline::create(); + Ok(insert_pipeline) } fn set_source_pipe_builder(&self, builder: Option) -> Result<()> { diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index a89b8481aa29d..b4c6aec1bfc4e 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -12,16 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::Future; use std::sync::Arc; use async_compat::CompatExt; use async_stream::stream; -use common_base::ProgressValues; +use common_base::{ProgressValues, Thread, TrySpawn}; use common_exception::ErrorCode; use common_exception::ToErrorCode; use common_io::prelude::parse_escape_string; use common_io::prelude::FormatSettings; -use common_planners::InsertInputSource; +use common_planners::{InsertInputSource, InsertPlan}; use common_planners::PlanNode; use common_streams::CsvSourceBuilder; use common_streams::NDJsonSourceBuilder; @@ -39,16 +40,18 @@ use poem::web::Multipart; use poem::Request; use serde::Deserialize; use serde::Serialize; +use common_datavalues::{DataSchema, DataSchemaRef}; use super::HttpQueryContext; use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::StreamSourceV2; use crate::pipelines::new::SourcePipeBuilder; -use crate::servers::http::v1::multipart_format::MultipartFormat; +use crate::servers::http::v1::multipart_format::{MultipartFormat, MultipartWorker}; use crate::sessions::QueryContext; use crate::sessions::SessionType; use crate::sql::PlanParser; +use common_exception::Result; #[derive(Serialize, Deserialize, Debug)] pub struct LoadResponse { @@ -58,12 +61,68 @@ pub struct LoadResponse { pub error: Option, } +fn get_input_format(node: &PlanNode) -> Result<&str> { + match node { + PlanNode::Insert(insert) => match &insert.source { + InsertInputSource::StreamingWithFormat(format) => Ok(format), + _ => Err(ErrorCode::UnknownFormat("Not found format name in plan")), + } + _ => Err(ErrorCode::UnknownFormat("Not found format name in plan")), + } +} + +fn execute_query(context: Arc, node: PlanNode, source_builder: SourcePipeBuilder) -> impl Future> { + async move { + let interpreter = InterpreterFactory::get(context, node)?; + + if let Err(cause) = interpreter.start().await { + tracing::error!("interpreter.start error: {:?}", cause); + } + + // TODO(Winter): very hack code. need remove it. + interpreter.set_source_pipe_builder(Option::from(source_builder))?; + + let mut data_stream = interpreter.execute(None).await?; + + while let Some(_block) = data_stream.next().await {} + + // Write Finish to query log table. + if let Err(cause) = interpreter.finish().await { + tracing::error!("interpreter.finish error: {:?}", cause); + } + + Ok(()) + } +} + +async fn new_processor_format(ctx: &Arc, node: &PlanNode, mut multipart: Multipart) -> Result> { + let format = get_input_format(node)?; + let format_settings = ctx.get_format_settings()?; + + let (mut worker, builder) = + format_source_pipe_builder(format, node.schema(), multipart, &format_settings)?; + + let handler = ctx.spawn(execute_query(ctx.clone(), node.clone(), builder)); + + + worker.work().await; + + match handler.await { + Ok(Ok(_)) => Ok(()), + Ok(Err(cause)) => Err(cause), + Err(_) => Err(ErrorCode::TokioError("Maybe panic.")), + }?; + + Ok(Json(LoadResponse { + error: None, + state: "SUCCESS".to_string(), + id: uuid::Uuid::new_v4().to_string(), + stats: ctx.get_scan_progress_value(), + })) +} + #[poem::handler] -pub async fn streaming_load( - ctx: &HttpQueryContext, - req: &Request, - mut multipart: Multipart, -) -> PoemResult> { +pub async fn streaming_load(ctx: &HttpQueryContext, req: &Request, mut multipart: Multipart) -> PoemResult> { let session = ctx .create_session(SessionType::HTTPStreamingLoad) .await @@ -115,15 +174,16 @@ pub async fn streaming_load( PlanNode::Insert(insert) => match &insert.source { InsertInputSource::StreamingWithFormat(format) => { if format.to_lowercase().as_str() == "csv" { - csv_source_pipe_builder( - context.clone(), - &plan, - &format_settings, - multipart, - max_block_size, - ) - .await - } else if format.to_lowercase().as_str() == "parquet" { + return match new_processor_format(&context, &plan, multipart).await { + Ok(res) => Ok(res), + Err(cause) => { + println!("catch error {:?}", cause); + Err(InternalServerError(cause)) + } + }; + } + + if format.to_lowercase().as_str() == "parquet" { parquet_source_pipe_builder(context.clone(), &plan, multipart).await } else if format.to_lowercase().as_str() == "ndjson" || format.to_lowercase().as_str() == "jsoneachrow" @@ -329,29 +389,27 @@ fn build_csv_stream( Ok(Box::pin(stream)) } -async fn csv_source_pipe_builder( - _ctx: Arc, - plan: &PlanNode, - format_settings: &FormatSettings, +fn format_source_pipe_builder( + format: &str, + schema: DataSchemaRef, multipart: Multipart, - _block_size: usize, -) -> PoemResult { + format_settings: &FormatSettings, +) -> Result<(MultipartWorker, SourcePipeBuilder)> { let ports = vec![OutputPort::create()]; let mut source_pipe_builder = SourcePipeBuilder::create(); - let sources = MultipartFormat::input_sources( - "csv", + let (worker, sources) = MultipartFormat::input_sources( + format, multipart, - plan.schema(), + schema, format_settings.clone(), ports.clone(), - ) - .unwrap(); + )?; for (index, source) in sources.into_iter().enumerate() { source_pipe_builder.add_source(ports[index].clone(), source); } - Ok(source_pipe_builder) + Ok((worker, source_pipe_builder)) } async fn parquet_source_pipe_builder( diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index e8e4a173f9e51..3b7ad5d1e618e 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -14,15 +14,17 @@ use std::mem::replace; use std::sync::Arc; +use poem::error::ParseMultipartError; use common_base::tokio::io::AsyncReadExt; -use common_base::tokio::sync::mpsc::Receiver; +use common_base::tokio::sync::mpsc::{Receiver, Sender}; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; use common_exception::Result; use common_io::prelude::FormatSettings; use poem::web::Multipart; +use common_base::{Runtime, TrySpawn}; use crate::formats::FormatFactory; use crate::formats::InputFormat; @@ -31,9 +33,83 @@ use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::Event; use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::Processor; +use crate::sessions::QueryContext; pub struct MultipartFormat; +pub struct MultipartWorker { + multipart: Multipart, + tx: Option>>>, +} + +impl MultipartWorker { + pub async fn work(&mut self) { + if let Some(tx) = self.tx.take() { + 'outer: loop { + match self.multipart.next_field().await { + Err(cause) => { + if let Err(cause) = tx.send(Err(ErrorCode::BadBytes(format!("Parse multipart error, cause {:?}", cause)))) + .await { + common_tracing::tracing::warn!( + "Multipart channel disconnect. {}", + cause + ); + + break 'outer; + } + } + Ok(None) => { break 'outer; } + Ok(Some(field)) => { + let mut async_reader = field.into_async_read(); + + 'read: loop { + let mut buf = vec![0; 2048]; + let read_res = async_reader.read(&mut buf[..]).await; + + match read_res { + Ok(0) => { + break 'read; + } + Ok(sz) => { + if sz != buf.len() { + buf = buf[..sz].to_vec(); + } + + if let Err(cause) = tx.send(Ok(buf)).await { + common_tracing::tracing::warn!( + "Multipart channel disconnect. {}", + cause + ); + + break 'outer; + } + } + Err(cause) => { + if let Err(cause) = tx + .send(Err(ErrorCode::BadBytes(format!( + "Read part to field bytes error, cause {:?}", + cause + )))) + .await + { + common_tracing::tracing::warn!( + "Multipart channel disconnect. {}", + cause + ); + break 'outer; + } + + break 'outer; + } + } + } + } + } + } + } + } +} + impl MultipartFormat { pub fn input_sources( name: &str, @@ -41,7 +117,7 @@ impl MultipartFormat { schema: DataSchemaRef, settings: FormatSettings, ports: Vec>, - ) -> Result> { + ) -> Result<(MultipartWorker, Vec)> { let input_format = FormatFactory::instance().get_input(name, schema, settings)?; if ports.len() != 1 || input_format.support_parallel() { @@ -52,49 +128,11 @@ impl MultipartFormat { let (tx, rx) = common_base::tokio::sync::mpsc::channel(2); - common_base::tokio::spawn(async move { - while let Ok(Some(field)) = multipart.next_field().await { - let mut buf = vec![0; 2048]; - let mut async_reader = field.into_async_read(); - match async_reader.read(&mut buf[..]).await { - Ok(0) => { - break; - } - Ok(read_size) => { - if read_size != buf.len() { - buf = buf[0..read_size].to_vec(); - } - - if let Err(cause) = tx.send(Ok(buf)).await { - common_tracing::tracing::warn!( - "Multipart channel disconnect. {}", - cause - ); - } - } - Err(cause) => { - if let Err(cause) = tx - .send(Err(ErrorCode::BadBytes(format!( - "Read part to field bytes error, cause {:?}", - cause - )))) - .await - { - common_tracing::tracing::warn!( - "Multipart channel disconnect. {}", - cause - ); - } - } - } - } - }); - - Ok(vec![SequentialInputFormatSource::create( + Ok((MultipartWorker { multipart, tx: Some(tx) }, vec![SequentialInputFormatSource::create( ports[0].clone(), input_format, rx, - )?]) + )?])) } } @@ -107,6 +145,7 @@ enum State { pub struct SequentialInputFormatSource { state: State, finished: bool, + skipped_header: bool, output: Arc, data_block: Vec, input_state: Box, @@ -130,6 +169,7 @@ impl SequentialInputFormatSource { finished: false, state: State::NeedReceiveData, data_block: vec![], + skipped_header: false, }, ))) } @@ -151,11 +191,12 @@ impl Processor for SequentialInputFormatSource { } if let Some(data_block) = self.data_block.pop() { + println!("Push block {:?}", data_block); self.output.push_data(Ok(data_block)); return Ok(Event::NeedConsume); } - if self.finished { + if self.finished && !matches!(&self.state, State::NeedDeserialize) { self.output.finish(); return Ok(Event::Finished); } @@ -172,6 +213,18 @@ impl Processor for SequentialInputFormatSource { State::ReceivedData(data) => { let mut data_slice: &[u8] = &data; + if !self.skipped_header { + let len = data_slice.len(); + let skip_size = self.input_format.skip_header(data_slice, &mut self.input_state)?; + + data_slice = &data_slice[skip_size..]; + + if skip_size < len { + self.skipped_header = true; + self.input_state = self.input_format.create_state(); + } + } + while !data_slice.is_empty() { let len = data_slice.len(); let read_size = self @@ -187,8 +240,10 @@ impl Processor for SequentialInputFormatSource { } } State::NeedDeserialize => { + let block = self.input_format.deserialize_data(&mut self.input_state)?; + println!("deserialize block {:?}", block); self.data_block - .push(self.input_format.deserialize_data(&mut self.input_state)?); + .push(block); } _ => { return Err(ErrorCode::LogicalError( @@ -201,7 +256,7 @@ impl Processor for SequentialInputFormatSource { } async fn async_process(&mut self) -> Result<()> { - if let State::NeedReceiveData = std::mem::replace(&mut self.state, State::NeedReceiveData) { + if let State::NeedReceiveData = replace(&mut self.state, State::NeedReceiveData) { if let Some(receive_res) = self.data_receiver.recv().await { self.state = State::ReceivedData(receive_res?); return Ok(()); From 87ff32e3ebb3adf4bca0b2c18787a8c7e26c977b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 7 May 2022 18:19:36 +0800 Subject: [PATCH 07/15] feat(format): implement csv input format --- .../src/types/deserializations/date.rs | 16 ++++++++-------- .../datavalues/src/types/deserializations/mod.rs | 2 +- .../src/types/deserializations/number.rs | 2 +- .../src/types/deserializations/string.rs | 10 ++++++++-- .../src/types/deserializations/timestamp.rs | 2 +- .../src/types/deserializations/variant.rs | 2 +- query/src/formats/format_csv.rs | 7 +------ query/src/servers/http/v1/multipart_format.rs | 15 ++++++--------- 8 files changed, 27 insertions(+), 29 deletions(-) diff --git a/common/datavalues/src/types/deserializations/date.rs b/common/datavalues/src/types/deserializations/date.rs index 2c7cfdc102df3..3afa8b7a7c9ed 100644 --- a/common/datavalues/src/types/deserializations/date.rs +++ b/common/datavalues/src/types/deserializations/date.rs @@ -28,10 +28,10 @@ pub struct DateDeserializer { pub const EPOCH_DAYS_FROM_CE: i32 = 719_163; impl TypeDeserializer for DateDeserializer -where - i32: AsPrimitive, - T: PrimitiveType, - T: Unmarshal + StatBuffer + FromLexical, + where + i32: AsPrimitive, + T: PrimitiveType, + T: Unmarshal + StatBuffer + FromLexical, { fn de_binary(&mut self, reader: &mut &[u8]) -> Result<()> { let value: T = reader.read_scalar()?; @@ -97,7 +97,7 @@ where Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; let date = reader.read_date_text()?; let days = uniform(date); @@ -138,9 +138,9 @@ where #[inline] fn uniform(date: NaiveDate) -> T -where - i32: AsPrimitive, - T: PrimitiveType, + where + i32: AsPrimitive, + T: PrimitiveType, { (date.num_days_from_ce() - EPOCH_DAYS_FROM_CE).as_() } diff --git a/common/datavalues/src/types/deserializations/mod.rs b/common/datavalues/src/types/deserializations/mod.rs index f6ec37975cc0c..9b625b421d593 100644 --- a/common/datavalues/src/types/deserializations/mod.rs +++ b/common/datavalues/src/types/deserializations/mod.rs @@ -55,7 +55,7 @@ pub trait TypeDeserializer: Send + Sync { fn de_text(&mut self, reader: &mut CheckpointReader) -> Result<()>; - fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { self.de_text(reader) } diff --git a/common/datavalues/src/types/deserializations/number.rs b/common/datavalues/src/types/deserializations/number.rs index d70f903e1ee1c..75c54e5d609d0 100644 --- a/common/datavalues/src/types/deserializations/number.rs +++ b/common/datavalues/src/types/deserializations/number.rs @@ -92,7 +92,7 @@ impl TypeDeserializer for NumberDeserializer Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; let v: T = if !T::FLOATING { reader.read_int_text() diff --git a/common/datavalues/src/types/deserializations/string.rs b/common/datavalues/src/types/deserializations/string.rs index 92bc51470c910..9d182de7ee47c 100644 --- a/common/datavalues/src/types/deserializations/string.rs +++ b/common/datavalues/src/types/deserializations/string.rs @@ -92,9 +92,15 @@ impl TypeDeserializer for StringDeserializer { Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + fn de_text_csv(&mut self, reader: &mut CheckpointReader, delimiter: u8) -> Result<()> { self.buffer.clear(); - reader.read_quoted_text(&mut self.buffer, b'"')?; + if reader.ignore_byte(b'"')? { + reader.keep_read(&mut self.buffer, |b| b != b'"')?; + reader.must_ignore_byte(b'"')?; + } else { + reader.keep_read(&mut self.buffer, |b| b != delimiter)?; + } + self.builder.append_value(self.buffer.as_slice()); Ok(()) } diff --git a/common/datavalues/src/types/deserializations/timestamp.rs b/common/datavalues/src/types/deserializations/timestamp.rs index 7a91e6e27ef8d..76c3968787c72 100644 --- a/common/datavalues/src/types/deserializations/timestamp.rs +++ b/common/datavalues/src/types/deserializations/timestamp.rs @@ -82,7 +82,7 @@ impl TypeDeserializer for TimestampDeserializer { Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; let ts = reader.read_timestamp_text(&self.tz)?; let micros = ts.timestamp_micros(); diff --git a/common/datavalues/src/types/deserializations/variant.rs b/common/datavalues/src/types/deserializations/variant.rs index 7c921dfb3fba0..790be60c1f210 100644 --- a/common/datavalues/src/types/deserializations/variant.rs +++ b/common/datavalues/src/types/deserializations/variant.rs @@ -92,7 +92,7 @@ impl TypeDeserializer for VariantDeserializer { Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader) -> Result<()> { + fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { self.buffer.clear(); reader.read_quoted_text(&mut self.buffer, b'"')?; diff --git a/query/src/formats/format_csv.rs b/query/src/formats/format_csv.rs index d690f0affb6e4..aec15e71c27fa 100644 --- a/query/src/formats/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -182,18 +182,14 @@ impl InputFormat for CsvInputFormat { break; } - println!("deserializers len {}", deserializers.len()); for column_index in 0..deserializers.len() { if checkpoint_reader.ignore_white_spaces_and_byte(self.field_delimiter)? { deserializers[column_index].de_default(); } else { - deserializers[column_index].de_text_csv(&mut checkpoint_reader)?; + deserializers[column_index].de_text_csv(&mut checkpoint_reader, self.field_delimiter)?; - println!("deserializers index {} len {}", column_index, deserializers.len()); if column_index + 1 != deserializers.len() { checkpoint_reader.must_ignore_white_spaces_and_byte(self.field_delimiter)?; - let buffer = checkpoint_reader.fill_buf()?; - println!("expect field delimiter. {}, {}, {}", column_index, deserializers.len(), buffer[0] as char); } } } @@ -257,7 +253,6 @@ impl InputFormat for CsvInputFormat { }; if state.accepted_rows == 1 { - println!("skip header {}", index); return Ok(index); } } diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index 3b7ad5d1e618e..0633f30eccd49 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -63,7 +63,8 @@ impl MultipartWorker { let mut async_reader = field.into_async_read(); 'read: loop { - let mut buf = vec![0; 2048]; + // 1048576 from clickhouse DBMS_DEFAULT_BUFFER_SIZE + let mut buf = vec![0; 1048576]; let read_res = async_reader.read(&mut buf[..]).await; match read_res { @@ -77,9 +78,9 @@ impl MultipartWorker { if let Err(cause) = tx.send(Ok(buf)).await { common_tracing::tracing::warn!( - "Multipart channel disconnect. {}", - cause - ); + "Multipart channel disconnect. {}", + cause + ); break 'outer; } @@ -191,7 +192,6 @@ impl Processor for SequentialInputFormatSource { } if let Some(data_block) = self.data_block.pop() { - println!("Push block {:?}", data_block); self.output.push_data(Ok(data_block)); return Ok(Event::NeedConsume); } @@ -240,10 +240,7 @@ impl Processor for SequentialInputFormatSource { } } State::NeedDeserialize => { - let block = self.input_format.deserialize_data(&mut self.input_state)?; - println!("deserialize block {:?}", block); - self.data_block - .push(block); + self.data_block.push(self.input_format.deserialize_data(&mut self.input_state)?); } _ => { return Err(ErrorCode::LogicalError( From 0268792b17ea0cbdfc4cf606d7679752f51debd1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 7 May 2022 18:28:40 +0800 Subject: [PATCH 08/15] feat(format): apply code style --- .../src/types/deserializations/date.rs | 20 ++++--- .../src/types/deserializations/mod.rs | 6 +- .../src/types/deserializations/number.rs | 12 ++-- .../src/types/deserializations/string.rs | 6 +- .../src/types/deserializations/timestamp.rs | 6 +- .../src/types/deserializations/variant.rs | 6 +- common/io/src/buffer/buffer_read_ext.rs | 2 +- query/src/formats/format_csv.rs | 51 ++++++++++------- query/src/servers/http/v1/load.rs | 33 +++++++---- query/src/servers/http/v1/multipart_format.rs | 56 ++++++++++++------- 10 files changed, 131 insertions(+), 67 deletions(-) diff --git a/common/datavalues/src/types/deserializations/date.rs b/common/datavalues/src/types/deserializations/date.rs index 3afa8b7a7c9ed..99ad19d3378c8 100644 --- a/common/datavalues/src/types/deserializations/date.rs +++ b/common/datavalues/src/types/deserializations/date.rs @@ -28,10 +28,10 @@ pub struct DateDeserializer { pub const EPOCH_DAYS_FROM_CE: i32 = 719_163; impl TypeDeserializer for DateDeserializer - where - i32: AsPrimitive, - T: PrimitiveType, - T: Unmarshal + StatBuffer + FromLexical, +where + i32: AsPrimitive, + T: PrimitiveType, + T: Unmarshal + StatBuffer + FromLexical, { fn de_binary(&mut self, reader: &mut &[u8]) -> Result<()> { let value: T = reader.read_scalar()?; @@ -97,7 +97,11 @@ impl TypeDeserializer for DateDeserializer Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { + fn de_text_csv( + &mut self, + reader: &mut CheckpointReader, + _delimiter: u8, + ) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; let date = reader.read_date_text()?; let days = uniform(date); @@ -138,9 +142,9 @@ impl TypeDeserializer for DateDeserializer #[inline] fn uniform(date: NaiveDate) -> T - where - i32: AsPrimitive, - T: PrimitiveType, +where + i32: AsPrimitive, + T: PrimitiveType, { (date.num_days_from_ce() - EPOCH_DAYS_FROM_CE).as_() } diff --git a/common/datavalues/src/types/deserializations/mod.rs b/common/datavalues/src/types/deserializations/mod.rs index 9b625b421d593..47a9f529d72a9 100644 --- a/common/datavalues/src/types/deserializations/mod.rs +++ b/common/datavalues/src/types/deserializations/mod.rs @@ -55,7 +55,11 @@ pub trait TypeDeserializer: Send + Sync { fn de_text(&mut self, reader: &mut CheckpointReader) -> Result<()>; - fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { + fn de_text_csv( + &mut self, + reader: &mut CheckpointReader, + _delimiter: u8, + ) -> Result<()> { self.de_text(reader) } diff --git a/common/datavalues/src/types/deserializations/number.rs b/common/datavalues/src/types/deserializations/number.rs index 75c54e5d609d0..01b2850ba2dd5 100644 --- a/common/datavalues/src/types/deserializations/number.rs +++ b/common/datavalues/src/types/deserializations/number.rs @@ -24,9 +24,9 @@ pub struct NumberDeserializer { } impl TypeDeserializer for NumberDeserializer - where - T: PrimitiveType, - T: Unmarshal + StatBuffer + FromLexical, +where + T: PrimitiveType, + T: Unmarshal + StatBuffer + FromLexical, { fn de_binary(&mut self, reader: &mut &[u8]) -> Result<()> { let value: T = reader.read_scalar()?; @@ -92,7 +92,11 @@ impl TypeDeserializer for NumberDeserializer Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { + fn de_text_csv( + &mut self, + reader: &mut CheckpointReader, + _delimiter: u8, + ) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; let v: T = if !T::FLOATING { reader.read_int_text() diff --git a/common/datavalues/src/types/deserializations/string.rs b/common/datavalues/src/types/deserializations/string.rs index 9d182de7ee47c..c8bc485ca359d 100644 --- a/common/datavalues/src/types/deserializations/string.rs +++ b/common/datavalues/src/types/deserializations/string.rs @@ -92,7 +92,11 @@ impl TypeDeserializer for StringDeserializer { Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader, delimiter: u8) -> Result<()> { + fn de_text_csv( + &mut self, + reader: &mut CheckpointReader, + delimiter: u8, + ) -> Result<()> { self.buffer.clear(); if reader.ignore_byte(b'"')? { reader.keep_read(&mut self.buffer, |b| b != b'"')?; diff --git a/common/datavalues/src/types/deserializations/timestamp.rs b/common/datavalues/src/types/deserializations/timestamp.rs index 76c3968787c72..d95f511aac82f 100644 --- a/common/datavalues/src/types/deserializations/timestamp.rs +++ b/common/datavalues/src/types/deserializations/timestamp.rs @@ -82,7 +82,11 @@ impl TypeDeserializer for TimestampDeserializer { Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { + fn de_text_csv( + &mut self, + reader: &mut CheckpointReader, + _delimiter: u8, + ) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; let ts = reader.read_timestamp_text(&self.tz)?; let micros = ts.timestamp_micros(); diff --git a/common/datavalues/src/types/deserializations/variant.rs b/common/datavalues/src/types/deserializations/variant.rs index 790be60c1f210..ee27408bb88ae 100644 --- a/common/datavalues/src/types/deserializations/variant.rs +++ b/common/datavalues/src/types/deserializations/variant.rs @@ -92,7 +92,11 @@ impl TypeDeserializer for VariantDeserializer { Ok(()) } - fn de_text_csv(&mut self, reader: &mut CheckpointReader, _delimiter: u8) -> Result<()> { + fn de_text_csv( + &mut self, + reader: &mut CheckpointReader, + _delimiter: u8, + ) -> Result<()> { self.buffer.clear(); reader.read_quoted_text(&mut self.buffer, b'"')?; diff --git a/common/io/src/buffer/buffer_read_ext.rs b/common/io/src/buffer/buffer_read_ext.rs index f549c1febd9ef..315f104251389 100644 --- a/common/io/src/buffer/buffer_read_ext.rs +++ b/common/io/src/buffer/buffer_read_ext.rs @@ -126,7 +126,7 @@ pub trait BufferReadExt: BufferRead { } impl BufferReadExt for R - where R: BufferRead +where R: BufferRead { fn ignores(&mut self, f: impl Fn(u8) -> bool) -> Result { let mut bytes = 0; diff --git a/query/src/formats/format_csv.rs b/query/src/formats/format_csv.rs index aec15e71c27fa..f6471c3988edf 100644 --- a/query/src/formats/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -13,7 +13,8 @@ // limitations under the License. use std::any::Any; -use std::io::{Cursor, Read}; +use std::io::Cursor; +use std::io::Read; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; @@ -21,7 +22,7 @@ use common_datavalues::DataType; use common_datavalues::TypeDeserializer; use common_exception::ErrorCode; use common_exception::Result; -use common_io::prelude::{BufferRead, CpBufferReader}; +use common_io::prelude::BufferRead; use common_io::prelude::BufferReadExt; use common_io::prelude::BufferReader; use common_io::prelude::CheckpointReader; @@ -57,11 +58,14 @@ pub struct CsvInputFormat { impl CsvInputFormat { pub fn register(factory: &mut FormatFactory) { - factory.register_input("csv", Box::new( - |name: &str, schema: DataSchemaRef, settings: FormatSettings| { - CsvInputFormat::try_create(name, schema, settings, 8192, 10 * 1024 * 1024) - }, - )) + factory.register_input( + "csv", + Box::new( + |name: &str, schema: DataSchemaRef, settings: FormatSettings| { + CsvInputFormat::try_create(name, schema, settings, 8192, 10 * 1024 * 1024) + }, + ), + ) } pub fn try_create( @@ -80,7 +84,8 @@ impl CsvInputFormat { if !settings.record_delimiter.is_empty() && settings.record_delimiter[0] != b'\n' - && settings.record_delimiter[0] != b'\r' { + && settings.record_delimiter[0] != b'\r' + { row_delimiter = Some(settings.record_delimiter[0]); } @@ -118,12 +123,10 @@ impl CsvInputFormat { if buf[index] == *b { return self.accept_row::<0>(buf, pos, state, index); } - } else { - if buf[index] == b'\r' { - return self.accept_row::(buf, pos, state, index); - } else if buf[index] == b'\n' { - return self.accept_row::(buf, pos, state, index); - } + } else if buf[index] == b'\r' { + return self.accept_row::(buf, pos, state, index); + } else if buf[index] == b'\n' { + return self.accept_row::(buf, pos, state, index); } } @@ -131,9 +134,15 @@ impl CsvInputFormat { } #[inline(always)] - fn accept_row(&self, buf: &[u8], pos: usize, state: &mut CsvInputState, index: usize) -> usize { + fn accept_row( + &self, + buf: &[u8], + pos: usize, + state: &mut CsvInputState, + index: usize, + ) -> usize { state.accepted_rows += 1; - state.accepted_bytes += (index - pos); + state.accepted_bytes += index - pos; if state.accepted_rows >= self.min_accepted_rows || (state.accepted_bytes + index) >= self.min_accepted_bytes @@ -186,10 +195,12 @@ impl InputFormat for CsvInputFormat { if checkpoint_reader.ignore_white_spaces_and_byte(self.field_delimiter)? { deserializers[column_index].de_default(); } else { - deserializers[column_index].de_text_csv(&mut checkpoint_reader, self.field_delimiter)?; + deserializers[column_index] + .de_text_csv(&mut checkpoint_reader, self.field_delimiter)?; if column_index + 1 != deserializers.len() { - checkpoint_reader.must_ignore_white_spaces_and_byte(self.field_delimiter)?; + checkpoint_reader + .must_ignore_white_spaces_and_byte(self.field_delimiter)?; } } } @@ -203,7 +214,9 @@ impl InputFormat for CsvInputFormat { row_index ))); } - } else if !checkpoint_reader.ignore_white_spaces_and_byte(b'\n')? & !checkpoint_reader.ignore_white_spaces_and_byte(b'\r')? { + } else if !checkpoint_reader.ignore_white_spaces_and_byte(b'\n')? + & !checkpoint_reader.ignore_white_spaces_and_byte(b'\r')? + { return Err(ErrorCode::BadBytes(format!( "Parse csv error at line {}", row_index diff --git a/query/src/servers/http/v1/load.rs b/query/src/servers/http/v1/load.rs index b4c6aec1bfc4e..eb9a4f3bf8d8b 100644 --- a/query/src/servers/http/v1/load.rs +++ b/query/src/servers/http/v1/load.rs @@ -17,12 +17,15 @@ use std::sync::Arc; use async_compat::CompatExt; use async_stream::stream; -use common_base::{ProgressValues, Thread, TrySpawn}; +use common_base::ProgressValues; +use common_base::TrySpawn; +use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; +use common_exception::Result; use common_exception::ToErrorCode; use common_io::prelude::parse_escape_string; use common_io::prelude::FormatSettings; -use common_planners::{InsertInputSource, InsertPlan}; +use common_planners::InsertInputSource; use common_planners::PlanNode; use common_streams::CsvSourceBuilder; use common_streams::NDJsonSourceBuilder; @@ -40,18 +43,17 @@ use poem::web::Multipart; use poem::Request; use serde::Deserialize; use serde::Serialize; -use common_datavalues::{DataSchema, DataSchemaRef}; use super::HttpQueryContext; use crate::interpreters::InterpreterFactory; use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::StreamSourceV2; use crate::pipelines::new::SourcePipeBuilder; -use crate::servers::http::v1::multipart_format::{MultipartFormat, MultipartWorker}; +use crate::servers::http::v1::multipart_format::MultipartFormat; +use crate::servers::http::v1::multipart_format::MultipartWorker; use crate::sessions::QueryContext; use crate::sessions::SessionType; use crate::sql::PlanParser; -use common_exception::Result; #[derive(Serialize, Deserialize, Debug)] pub struct LoadResponse { @@ -66,12 +68,16 @@ fn get_input_format(node: &PlanNode) -> Result<&str> { PlanNode::Insert(insert) => match &insert.source { InsertInputSource::StreamingWithFormat(format) => Ok(format), _ => Err(ErrorCode::UnknownFormat("Not found format name in plan")), - } + }, _ => Err(ErrorCode::UnknownFormat("Not found format name in plan")), } } -fn execute_query(context: Arc, node: PlanNode, source_builder: SourcePipeBuilder) -> impl Future> { +fn execute_query( + context: Arc, + node: PlanNode, + source_builder: SourcePipeBuilder, +) -> impl Future> { async move { let interpreter = InterpreterFactory::get(context, node)?; @@ -95,7 +101,11 @@ fn execute_query(context: Arc, node: PlanNode, source_builder: Sou } } -async fn new_processor_format(ctx: &Arc, node: &PlanNode, mut multipart: Multipart) -> Result> { +async fn new_processor_format( + ctx: &Arc, + node: &PlanNode, + multipart: Multipart, +) -> Result> { let format = get_input_format(node)?; let format_settings = ctx.get_format_settings()?; @@ -104,7 +114,6 @@ async fn new_processor_format(ctx: &Arc, node: &PlanNode, mut mult let handler = ctx.spawn(execute_query(ctx.clone(), node.clone(), builder)); - worker.work().await; match handler.await { @@ -122,7 +131,11 @@ async fn new_processor_format(ctx: &Arc, node: &PlanNode, mut mult } #[poem::handler] -pub async fn streaming_load(ctx: &HttpQueryContext, req: &Request, mut multipart: Multipart) -> PoemResult> { +pub async fn streaming_load( + ctx: &HttpQueryContext, + req: &Request, + mut multipart: Multipart, +) -> PoemResult> { let session = ctx .create_session(SessionType::HTTPStreamingLoad) .await diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index 0633f30eccd49..d27a93502524f 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -14,17 +14,16 @@ use std::mem::replace; use std::sync::Arc; -use poem::error::ParseMultipartError; use common_base::tokio::io::AsyncReadExt; -use common_base::tokio::sync::mpsc::{Receiver, Sender}; +use common_base::tokio::sync::mpsc::Receiver; +use common_base::tokio::sync::mpsc::Sender; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; use common_exception::Result; use common_io::prelude::FormatSettings; use poem::web::Multipart; -use common_base::{Runtime, TrySpawn}; use crate::formats::FormatFactory; use crate::formats::InputFormat; @@ -33,7 +32,6 @@ use crate::pipelines::new::processors::port::OutputPort; use crate::pipelines::new::processors::processor::Event; use crate::pipelines::new::processors::processor::ProcessorPtr; use crate::pipelines::new::processors::Processor; -use crate::sessions::QueryContext; pub struct MultipartFormat; @@ -48,17 +46,24 @@ impl MultipartWorker { 'outer: loop { match self.multipart.next_field().await { Err(cause) => { - if let Err(cause) = tx.send(Err(ErrorCode::BadBytes(format!("Parse multipart error, cause {:?}", cause)))) - .await { + if let Err(cause) = tx + .send(Err(ErrorCode::BadBytes(format!( + "Parse multipart error, cause {:?}", + cause + )))) + .await + { common_tracing::tracing::warn!( - "Multipart channel disconnect. {}", - cause - ); + "Multipart channel disconnect. {}", + cause + ); break 'outer; } } - Ok(None) => { break 'outer; } + Ok(None) => { + break 'outer; + } Ok(Some(field)) => { let mut async_reader = field.into_async_read(); @@ -94,9 +99,9 @@ impl MultipartWorker { .await { common_tracing::tracing::warn!( - "Multipart channel disconnect. {}", - cause - ); + "Multipart channel disconnect. {}", + cause + ); break 'outer; } @@ -114,7 +119,7 @@ impl MultipartWorker { impl MultipartFormat { pub fn input_sources( name: &str, - mut multipart: Multipart, + multipart: Multipart, schema: DataSchemaRef, settings: FormatSettings, ports: Vec>, @@ -129,11 +134,17 @@ impl MultipartFormat { let (tx, rx) = common_base::tokio::sync::mpsc::channel(2); - Ok((MultipartWorker { multipart, tx: Some(tx) }, vec![SequentialInputFormatSource::create( - ports[0].clone(), - input_format, - rx, - )?])) + Ok(( + MultipartWorker { + multipart, + tx: Some(tx), + }, + vec![SequentialInputFormatSource::create( + ports[0].clone(), + input_format, + rx, + )?], + )) } } @@ -215,7 +226,9 @@ impl Processor for SequentialInputFormatSource { if !self.skipped_header { let len = data_slice.len(); - let skip_size = self.input_format.skip_header(data_slice, &mut self.input_state)?; + let skip_size = self + .input_format + .skip_header(data_slice, &mut self.input_state)?; data_slice = &data_slice[skip_size..]; @@ -240,7 +253,8 @@ impl Processor for SequentialInputFormatSource { } } State::NeedDeserialize => { - self.data_block.push(self.input_format.deserialize_data(&mut self.input_state)?); + self.data_block + .push(self.input_format.deserialize_data(&mut self.input_state)?); } _ => { return Err(ErrorCode::LogicalError( From 07755e0a2c9f3659f13f73f65119ec15bafca0f8 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 7 May 2022 19:28:18 +0800 Subject: [PATCH 09/15] feat(format): try fixes format test --- .../src/types/deserializations/number.rs | 8 ++- query/tests/it/formats/format_csv.rs | 52 +++++++++---------- 2 files changed, 33 insertions(+), 27 deletions(-) diff --git a/common/datavalues/src/types/deserializations/number.rs b/common/datavalues/src/types/deserializations/number.rs index d55716e0c7802..db341068b99b8 100644 --- a/common/datavalues/src/types/deserializations/number.rs +++ b/common/datavalues/src/types/deserializations/number.rs @@ -105,9 +105,15 @@ where fn de_text_csv( &mut self, reader: &mut CheckpointReader, - _format: &FormatSettings, + settings: &FormatSettings, ) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; + + if maybe_quote && reader.ignore(|f| f == b'\'' || f == b'"')? && settings.empty_as_default { + self.de_default(settings); + return Ok(()); + } + let v: T = if !T::FLOATING { reader.read_int_text() } else { diff --git a/query/tests/it/formats/format_csv.rs b/query/tests/it/formats/format_csv.rs index b20dabc37c0cb..cc160893b188a 100644 --- a/query/tests/it/formats/format_csv.rs +++ b/query/tests/it/formats/format_csv.rs @@ -28,19 +28,19 @@ use databend_query::formats::format_csv::CsvInputState; #[test] fn test_accepted_multi_lines() -> Result<()> { assert_complete_line("")?; - assert_complete_line("first\tsecond\n")?; - assert_complete_line("first\tsecond\r")?; - assert_complete_line("first\tsecond\r\n")?; - assert_complete_line("first\tsecond\n\r")?; - assert_complete_line("first\t\"\n\"second\n")?; - assert_complete_line("first\t\"\r\"second\n")?; + assert_complete_line("first,second\n")?; + assert_complete_line("first,second\r")?; + assert_complete_line("first,second\r\n")?; + assert_complete_line("first,second\n\r")?; + assert_complete_line("first,\"\n\"second\n")?; + assert_complete_line("first,\"\r\"second\n")?; assert_broken_line("first", 5)?; - assert_broken_line("first\t", 6)?; - assert_broken_line("first\ts", 7)?; - assert_broken_line("first\ts\"\n", 9)?; - assert_broken_line("first\ts\"\r", 9)?; - assert_broken_line("first\tsecond\ns", 13)?; + assert_broken_line("first,", 6)?; + assert_broken_line("first,s", 7)?; + assert_broken_line("first,s\"\n", 9)?; + assert_broken_line("first,s\"\r", 9)?; + assert_broken_line("first,second\ns", 13)?; let csv_input_format = CsvInputFormat::try_create( "csv", @@ -52,7 +52,7 @@ fn test_accepted_multi_lines() -> Result<()> { let mut csv_input_state = csv_input_format.create_state(); - let bytes = "first\tsecond\nfirst\t".as_bytes(); + let bytes = "first,second\nfirst,".as_bytes(); assert_eq!( bytes.len(), csv_input_format.read_buf(bytes, &mut csv_input_state)? @@ -66,10 +66,10 @@ fn test_accepted_multi_lines() -> Result<()> { .memory ); - let bytes = "second\nfirst\t".as_bytes(); + let bytes = "second\nfirst,".as_bytes(); assert_eq!(7, csv_input_format.read_buf(bytes, &mut csv_input_state)?); assert_eq!( - "first\tsecond\nfirst\tsecond\n".as_bytes(), + "first,second\nfirst,second\n".as_bytes(), csv_input_state .as_any() .downcast_mut::() @@ -94,14 +94,14 @@ fn test_deserialize_multi_lines() -> Result<()> { let mut csv_input_state = csv_input_format.create_state(); - csv_input_format.read_buf("1\t\"second\"\n".as_bytes(), &mut csv_input_state)?; + csv_input_format.read_buf("1,\"second\"\n".as_bytes(), &mut csv_input_state)?; assert_blocks_eq( vec![ - "+---+----------+", - "| a | b |", - "+---+----------+", - "| 1 | \"second\" |", - "+---+----------+", + "+---+--------+", + "| a | b |", + "+---+--------+", + "| 1 | second |", + "+---+--------+", ], &[csv_input_format.deserialize_data(&mut csv_input_state)?], ); @@ -119,14 +119,14 @@ fn test_deserialize_multi_lines() -> Result<()> { let mut csv_input_state = csv_input_format.create_state(); - csv_input_format.read_buf("1\t\"second\"\n".as_bytes(), &mut csv_input_state)?; + csv_input_format.read_buf("1,\"second\"\n".as_bytes(), &mut csv_input_state)?; assert_blocks_eq( vec![ - "+---+----------+", - "| a | b |", - "+---+----------+", - "| 1 | \"second\" |", - "+---+----------+", + "+---+--------+", + "| a | b |", + "+---+--------+", + "| 1 | second |", + "+---+--------+", ], &[csv_input_format.deserialize_data(&mut csv_input_state)?], ); From 036df8532e8e72698d13d1d756fcda5e766b5189 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 7 May 2022 22:51:51 +0800 Subject: [PATCH 10/15] feat(format): ignore tabs --- common/io/src/buffer/buffer_read_ext.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/common/io/src/buffer/buffer_read_ext.rs b/common/io/src/buffer/buffer_read_ext.rs index 315f104251389..1d840e713f4f6 100644 --- a/common/io/src/buffer/buffer_read_ext.rs +++ b/common/io/src/buffer/buffer_read_ext.rs @@ -126,7 +126,7 @@ pub trait BufferReadExt: BufferRead { } impl BufferReadExt for R -where R: BufferRead + where R: BufferRead { fn ignores(&mut self, f: impl Fn(u8) -> bool) -> Result { let mut bytes = 0; @@ -191,10 +191,10 @@ where R: BufferRead } fn ignore_white_spaces_and_byte(&mut self, b: u8) -> Result { - self.ignores(|c: u8| c == b' ' || c == b'\t')?; + self.ignores(|c: u8| c == b' ')?; if self.ignore_byte(b)? { - self.ignores(|c: u8| c == b' ' || c == b'\t')?; + self.ignores(|c: u8| c == b' ')?; return Ok(true); } From 4f9a2e99a6ab1bf55b7888c808d374de5ffd28e1 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 7 May 2022 23:14:03 +0800 Subject: [PATCH 11/15] feat(format): replace state each deserialize --- query/src/formats/format_csv.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/query/src/formats/format_csv.rs b/query/src/formats/format_csv.rs index 373227c568605..257de3b4813ab 100644 --- a/query/src/formats/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -182,6 +182,7 @@ impl InputFormat for CsvInputFormat { deserializers.push(data_type.create_deserializer(self.min_accepted_rows)); } + let mut state = std::mem::replace(state, self.create_state()); let state = state.as_any().downcast_mut::().unwrap(); let cursor = Cursor::new(&state.memory); let reader: Box = Box::new(BufferReader::new(cursor)); From 6a2244b3c39a4ea02f7c900a77dca5cef8970fca Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 8 May 2022 12:04:07 +0800 Subject: [PATCH 12/15] feat(format): support use \\t as delimiter in csv format --- .../src/types/deserializations/number.rs | 7 +- .../src/types/deserializations/string.rs | 138 ++++++++++++++++-- common/io/src/buffer/buffer_read_ext.rs | 6 +- .../io/src/buffer/buffer_read_number_ext.rs | 14 +- query/src/formats/format_csv.rs | 24 ++- query/src/servers/http/v1/multipart_format.rs | 19 ++- 6 files changed, 175 insertions(+), 33 deletions(-) diff --git a/common/datavalues/src/types/deserializations/number.rs b/common/datavalues/src/types/deserializations/number.rs index db341068b99b8..95e91b966a767 100644 --- a/common/datavalues/src/types/deserializations/number.rs +++ b/common/datavalues/src/types/deserializations/number.rs @@ -105,15 +105,10 @@ where fn de_text_csv( &mut self, reader: &mut CheckpointReader, - settings: &FormatSettings, + _settings: &FormatSettings, ) -> Result<()> { let maybe_quote = reader.ignore(|f| f == b'\'' || f == b'"')?; - if maybe_quote && reader.ignore(|f| f == b'\'' || f == b'"')? && settings.empty_as_default { - self.de_default(settings); - return Ok(()); - } - let v: T = if !T::FLOATING { reader.read_int_text() } else { diff --git a/common/datavalues/src/types/deserializations/string.rs b/common/datavalues/src/types/deserializations/string.rs index 6685e9c9cc1a2..7df6807b17bd5 100644 --- a/common/datavalues/src/types/deserializations/string.rs +++ b/common/datavalues/src/types/deserializations/string.rs @@ -109,20 +109,136 @@ impl TypeDeserializer for StringDeserializer { fn de_text_csv( &mut self, reader: &mut CheckpointReader, - format: &FormatSettings, + settings: &FormatSettings, ) -> Result<()> { - self.buffer.clear(); - if reader.ignore_byte(b'"')? { - reader.keep_read(&mut self.buffer, |b| b != b'"')?; - reader.must_ignore_byte(b'"')?; - } else if format.field_delimiter.is_empty() { - reader.keep_read(&mut self.buffer, |b| b != b',')?; - } else { - reader.keep_read(&mut self.buffer, |b| b != format.field_delimiter[0])?; + let mut read_buffer = reader.fill_buf()?; + + if read_buffer.is_empty() { + return Err(ErrorCode::BadBytes("Read string after eof.")); } - self.builder.append_value(self.buffer.as_slice()); - Ok(()) + let maybe_quote = read_buffer[0]; + if maybe_quote == b'\'' || maybe_quote == b'"' { + let mut index = 1; + let mut bytes = 0; + + loop { + let begin = index; + while index < read_buffer.len() { + if read_buffer[index] == maybe_quote { + self.builder + .values_mut() + .extend_from_slice(&read_buffer[begin..index]); + self.builder.add_offset(bytes + index - begin); + reader.consume(index + 1); + return Ok(()); + } + + index += 1; + } + + bytes += index - begin; + self.builder + .values_mut() + .extend_from_slice(&read_buffer[begin..]); + reader.consume(index - begin); + + index = 0; + read_buffer = reader.fill_buf()?; + + if read_buffer.is_empty() { + break; + } + } + + Err(ErrorCode::BadBytes(format!( + "Not found '{}' before eof in parse string.", + maybe_quote as char + ))) + } else { + // Unquoted case. Look for field_delimiter or record_delimiter. + let mut field_delimiter = b','; + + if !settings.field_delimiter.is_empty() { + field_delimiter = settings.field_delimiter[0]; + } + + if settings.record_delimiter.is_empty() + || settings.record_delimiter[0] == b'\r' + || settings.record_delimiter[0] == b'\n' + { + let mut index = 0; + let mut bytes = 0; + + 'outer1: loop { + while index < read_buffer.len() { + if read_buffer[index] == field_delimiter + || read_buffer[index] == b'\r' + || read_buffer[index] == b'\n' + { + break 'outer1; + } + index += 1; + } + + bytes += index; + self.builder + .values_mut() + .extend_from_slice(&read_buffer[..index]); + reader.consume(index); + + index = 0; + read_buffer = reader.fill_buf()?; + + if read_buffer.is_empty() { + break 'outer1; + } + } + + self.builder + .values_mut() + .extend_from_slice(&read_buffer[..index]); + self.builder.add_offset(bytes + index); + reader.consume(index); + } else { + let record_delimiter = settings.record_delimiter[0]; + + let mut index = 0; + let mut bytes = 0; + + 'outer2: loop { + while index < read_buffer.len() { + if read_buffer[index] == field_delimiter + || read_buffer[index] == record_delimiter + { + break 'outer2; + } + index += 1; + } + + bytes += index; + self.builder + .values_mut() + .extend_from_slice(&read_buffer[..index]); + reader.consume(index); + + index = 0; + read_buffer = reader.fill_buf()?; + + if read_buffer.is_empty() { + break 'outer2; + } + } + + self.builder + .values_mut() + .extend_from_slice(&read_buffer[..index]); + self.builder.add_offset(bytes + index); + reader.consume(index); + } + + Ok(()) + } } fn append_data_value(&mut self, value: DataValue, _format: &FormatSettings) -> Result<()> { diff --git a/common/io/src/buffer/buffer_read_ext.rs b/common/io/src/buffer/buffer_read_ext.rs index 1d840e713f4f6..ea19f778e44d0 100644 --- a/common/io/src/buffer/buffer_read_ext.rs +++ b/common/io/src/buffer/buffer_read_ext.rs @@ -84,10 +84,9 @@ pub trait BufferReadExt: BufferRead { fn must_ignore_byte(&mut self, b: u8) -> Result<()> { if !self.ignore_byte(b)? { - let buf = self.fill_buf()?; return Err(std::io::Error::new( ErrorKind::InvalidData, - format!("Expected to have char {}, {}", b as char, buf[0] as char), + format!("Expected to have char {}.", b as char), )); } Ok(()) @@ -95,10 +94,9 @@ pub trait BufferReadExt: BufferRead { fn must_ignore_white_spaces_and_byte(&mut self, b: u8) -> Result<()> { if !self.ignore_white_spaces_and_byte(b)? { - let buf = self.fill_buf()?; return Err(std::io::Error::new( ErrorKind::InvalidData, - format!("Expected to have char {}, {}", b as char, buf[0] as char), + format!("Expected to have char {}", b as char), )); } Ok(()) diff --git a/common/io/src/buffer/buffer_read_number_ext.rs b/common/io/src/buffer/buffer_read_number_ext.rs index 0145bed13ce36..45c9c116c1df4 100644 --- a/common/io/src/buffer/buffer_read_number_ext.rs +++ b/common/io/src/buffer/buffer_read_number_ext.rs @@ -66,9 +66,17 @@ where R: BufferRead let _ = self.ignores(|f| (b'0'..=b'9').contains(&f))?; } - FromLexical::from_lexical(buf.as_slice()).map_err_to_code(ErrorCode::BadBytes, || { - format!("Cannot parse value:{:?} to number type", buf) - }) + match buf.is_empty() { + true => Ok(T::default()), + false => match FromLexical::from_lexical(buf.as_slice()) { + Ok(value) => Ok(value), + Err(cause) => Err(ErrorCode::BadBytes(format!( + "Cannot parse value:{:?} to number type, cause: {:?}", + String::from_utf8(buf), + cause + ))), + }, + } } fn read_float_text(&mut self) -> Result { diff --git a/query/src/formats/format_csv.rs b/query/src/formats/format_csv.rs index 257de3b4813ab..f695e648660e8 100644 --- a/query/src/formats/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -210,19 +210,27 @@ impl InputFormat for CsvInputFormat { checkpoint_reader.ignore_white_spaces_and_byte(self.field_delimiter)?; if let Some(delimiter) = &self.row_delimiter { - if !checkpoint_reader.ignore_white_spaces_and_byte(*delimiter)? { + if !checkpoint_reader.ignore_white_spaces_and_byte(*delimiter)? + && !checkpoint_reader.eof()? + { return Err(ErrorCode::BadBytes(format!( "Parse csv error at line {}", row_index ))); } - } else if !checkpoint_reader.ignore_white_spaces_and_byte(b'\n')? - & !checkpoint_reader.ignore_white_spaces_and_byte(b'\r')? - { - return Err(ErrorCode::BadBytes(format!( - "Parse csv error at line {}", - row_index - ))); + } else { + if (!checkpoint_reader.ignore_white_spaces_and_byte(b'\n')? + & !checkpoint_reader.ignore_white_spaces_and_byte(b'\r')?) + && !checkpoint_reader.eof()? + { + return Err(ErrorCode::BadBytes(format!( + "Parse csv error at line {}", + row_index + ))); + } + + // \r\n + checkpoint_reader.ignore_white_spaces_and_byte(b'\n')?; } } diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index d27a93502524f..2313f05ddd35e 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -65,6 +65,15 @@ impl MultipartWorker { break 'outer; } Ok(Some(field)) => { + if let Err(cause) = tx.send(Ok(vec![])).await { + common_tracing::tracing::warn!( + "Multipart channel disconnect. {}", + cause + ); + + break 'outer; + } + let mut async_reader = field.into_async_read(); 'read: loop { @@ -269,7 +278,15 @@ impl Processor for SequentialInputFormatSource { async fn async_process(&mut self) -> Result<()> { if let State::NeedReceiveData = replace(&mut self.state, State::NeedReceiveData) { if let Some(receive_res) = self.data_receiver.recv().await { - self.state = State::ReceivedData(receive_res?); + let receive_bytes = receive_res?; + + if !receive_bytes.is_empty() { + self.state = State::ReceivedData(receive_bytes); + } else { + self.skipped_header = false; + self.state = State::NeedDeserialize; + } + return Ok(()); } } From 1389274586241d613bfaf1257fb040eed9dc114f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 8 May 2022 13:28:42 +0800 Subject: [PATCH 13/15] feat(format): implement csv deserialize for array type --- .../src/types/deserializations/array.rs | 33 +++++++++++++++++-- common/io/src/buffer/buffer_read_ext.rs | 2 +- 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/common/datavalues/src/types/deserializations/array.rs b/common/datavalues/src/types/deserializations/array.rs index c9ef65a7e5e4a..bc45c21e67f2a 100644 --- a/common/datavalues/src/types/deserializations/array.rs +++ b/common/datavalues/src/types/deserializations/array.rs @@ -85,8 +85,37 @@ impl TypeDeserializer for ArrayDeserializer { } let mut values = Vec::with_capacity(idx); for _ in 0..idx { - let value = self.inner.pop_data_value().unwrap(); - values.push(value); + values.push(self.inner.pop_data_value()?); + } + values.reverse(); + self.builder.append_value(ArrayValue::new(values)); + Ok(()) + } + + fn de_text_csv( + &mut self, + reader: &mut CheckpointReader, + format: &FormatSettings, + ) -> Result<()> { + reader.must_ignore_byte(b'[')?; + let mut idx = 0; + loop { + let _ = reader.ignore_white_spaces()?; + if let Ok(res) = reader.ignore_byte(b']') { + if res { + break; + } + } + if idx != 0 { + let _ = reader.must_ignore_byte(b',')?; + } + let _ = reader.ignore_white_spaces()?; + self.inner.de_text_csv(reader, format)?; + idx += 1; + } + let mut values = Vec::with_capacity(idx); + for _ in 0..idx { + values.push(self.inner.pop_data_value()?); } values.reverse(); self.builder.append_value(ArrayValue::new(values)); diff --git a/common/io/src/buffer/buffer_read_ext.rs b/common/io/src/buffer/buffer_read_ext.rs index ea19f778e44d0..53ea92c19c01f 100644 --- a/common/io/src/buffer/buffer_read_ext.rs +++ b/common/io/src/buffer/buffer_read_ext.rs @@ -124,7 +124,7 @@ pub trait BufferReadExt: BufferRead { } impl BufferReadExt for R - where R: BufferRead +where R: BufferRead { fn ignores(&mut self, f: impl Fn(u8) -> bool) -> Result { let mut bytes = 0; From cdb6934f98b47158281d8580de8016a75e09a373 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 8 May 2022 17:24:39 +0800 Subject: [PATCH 14/15] feat(format): support single quote --- query/src/formats/format_csv.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/query/src/formats/format_csv.rs b/query/src/formats/format_csv.rs index f695e648660e8..891f8ba465ea7 100644 --- a/query/src/formats/format_csv.rs +++ b/query/src/formats/format_csv.rs @@ -21,7 +21,6 @@ use common_datavalues::DataType; use common_datavalues::TypeDeserializer; use common_exception::ErrorCode; use common_exception::Result; -use common_io::prelude::BufferRead; use common_io::prelude::BufferReadExt; use common_io::prelude::BufferReader; use common_io::prelude::CheckpointReader; @@ -32,7 +31,7 @@ use crate::formats::InputFormat; use crate::formats::InputState; pub struct CsvInputState { - pub quotes: bool, + pub quotes: u8, pub memory: Vec, pub accepted_rows: usize, pub accepted_bytes: usize, @@ -104,8 +103,8 @@ impl CsvInputFormat { fn find_quotes(buf: &[u8], pos: usize, state: &mut CsvInputState) -> usize { for (index, byte) in buf.iter().enumerate().skip(pos) { - if *byte == b'"' { - state.quotes = false; + if *byte == b'"' || *byte == b'\'' { + state.quotes = 0; return index + 1; } } @@ -115,8 +114,8 @@ impl CsvInputFormat { fn find_delimiter(&self, buf: &[u8], pos: usize, state: &mut CsvInputState) -> usize { for index in pos..buf.len() { - if buf[index] == b'"' { - state.quotes = true; + if buf[index] == b'"' || buf[index] == b'\'' { + state.quotes = buf[index]; return index + 1; } @@ -166,7 +165,7 @@ impl CsvInputFormat { impl InputFormat for CsvInputFormat { fn create_state(&self) -> Box { Box::new(CsvInputState { - quotes: false, + quotes: 0, memory: vec![], accepted_rows: 0, accepted_bytes: 0, @@ -185,7 +184,7 @@ impl InputFormat for CsvInputFormat { let mut state = std::mem::replace(state, self.create_state()); let state = state.as_any().downcast_mut::().unwrap(); let cursor = Cursor::new(&state.memory); - let reader: Box = Box::new(BufferReader::new(cursor)); + let reader = BufferReader::new(cursor); let mut checkpoint_reader = CheckpointReader::new(reader); for row_index in 0..self.min_accepted_rows { @@ -254,7 +253,7 @@ impl InputFormat for CsvInputFormat { state.need_more_data = true; while index < buf.len() && state.need_more_data { - index = match state.quotes { + index = match state.quotes != 0 { true => Self::find_quotes(buf, index, state), false => self.find_delimiter(buf, index, state), } @@ -270,7 +269,7 @@ impl InputFormat for CsvInputFormat { let state = state.as_any().downcast_mut::().unwrap(); while index < buf.len() { - index = match state.quotes { + index = match state.quotes != 0 { true => Self::find_quotes(buf, index, state), false => self.find_delimiter(buf, index, state), }; From 7b46cad6d29ff518d58b851c4e7eeb05874f2426 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 8 May 2022 20:01:29 +0800 Subject: [PATCH 15/15] feat(format): try fix build failure after merge main --- query/src/servers/http/v1/multipart_format.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/query/src/servers/http/v1/multipart_format.rs b/query/src/servers/http/v1/multipart_format.rs index 2313f05ddd35e..4746786b6b76b 100644 --- a/query/src/servers/http/v1/multipart_format.rs +++ b/query/src/servers/http/v1/multipart_format.rs @@ -15,9 +15,9 @@ use std::mem::replace; use std::sync::Arc; -use common_base::tokio::io::AsyncReadExt; -use common_base::tokio::sync::mpsc::Receiver; -use common_base::tokio::sync::mpsc::Sender; +use common_base::base::tokio::io::AsyncReadExt; +use common_base::base::tokio::sync::mpsc::Receiver; +use common_base::base::tokio::sync::mpsc::Sender; use common_datablocks::DataBlock; use common_datavalues::DataSchemaRef; use common_exception::ErrorCode; @@ -141,7 +141,7 @@ impl MultipartFormat { )); } - let (tx, rx) = common_base::tokio::sync::mpsc::channel(2); + let (tx, rx) = common_base::base::tokio::sync::mpsc::channel(2); Ok(( MultipartWorker {