mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-16 02:02:56 +00:00
* change dep Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * feat: adapt to arrow's interval array * chore: fix compile errors in datatypes crate * chore: fix api crate compiler errors * chore: fix compiler errors in common-grpc * chore: fix common-datasource errors * chore: fix deprecated code in common-datasource * fix promql and physical plan related Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * wip: upgrading network deps Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * block on updating `sqlparser` * upgrade sqlparser Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * adapt new df's trait requirements Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * chore: fix compiler errors in mito2 * chore: fix common-function crate errors * chore: fix catalog errors * change import path Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * chore: fix some errors in query crate * chore: fix some errors in query crate * aggr expr and some other tiny fixes Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * chore: fix expr related errors in query crate * chore: fix query serializer and admin command * chore: fix grpc services * feat: axum serve * chore: fix http server * remove handle_error handler * refactor timeout layer * serve axum * chore: fix flow aggr functions * chore: fix flow * feat: fix errors in meta-srv * boxed() * use TokioIo * feat!: Remove script crate and python feature (#5321) * feat: exclude script crate * chore: simplify feature * feat: remove the script crate * chore: remove python feature and some comments * chore: fix warning * chore: fix servers tests compiler errors * feat: fix tests-integration errors * chore: fix unused * test: fix catalog test * chore: fix compiler errors for crates using common-meta testing feature is enabled when check with --workspace * test: use display for logical plan test * test: implement rewrite for ScanHintRule * fix: http server build panic * test: fix mito test * fix: sql parser type alias error * test: fix TestClient not listen * test: some flow tests * test(flow): more fix * fix: test_otlp_logs * test: fix promql test that using deprecated method fun() * fix: sql type replace supports Int8 ~ Int64, UInt8 ~ UInt64 * test: fix infer schema test case * test: fix tests related to plan display * chore: fix last flow test * test: fix function format related assertion * test: use larger port range for tests * fix: test_otlp_traces * fix: test_otlp_metrics * fix range query and dist plan Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * fix: flow handle distinct use deprecated field * fix: can't pass Join plan expressions to LogicalPlan::with_new_exprs * test: fix deserialize test * test: reduce split key case num * tests: lower case aggr func name * test: fix some sqlness tests * tests: more sqlness fix * tests: fixed sqlness test * commit non-bug changes Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * fix: make our udf correct * fix: implement empty methods of ContextProvider for DfContextProviderAdapter * test: update sqlness test result * chore: remove unused * fix: provide alias name for AggregateExprBuilder in range plan * test: update range query result * fix: implement missing ContextProvider methods for DfContextProviderAdapter * test: update timestamps, cte result * fix: supports empty projection in mito * test: update comment for cte test * fix: support projection for numbers * test: update test cases after projection fix * fix: fix range select first_value/last_value * fix: handle CAST and time index conflict * fix: handle order by correctly in range first_value/last_value * test: update sqlness result * test: update view test result * test: update decimal test wait for https://github.com/apache/datafusion/pull/14126 to fix this * feat: remove redundant physical optimization todo(ruihang): Check if we can remove this. * test: update sqlness test result * chore: range select default sort use nulls_first = false * test: update filter push down test result * test: comment deciaml test to avoid different panic message * test: update some distributed test result * test: update test for distributed count and filter push down * test: update subqueries test * fix: SessionState may overwrite our UDFs * chore: fix compiler errors after merging main * fix: fix elasticsearch and dashboard router panic * chore: fix common-functions tests * chore: update sqlness result * test: fix id keyword and update sqlness result * test: fix flow_null test * fix: enlarge thread size in debug mode to avoid overflow * chore: fix warnings in common-function * chore: fix warning in flow * chore: fix warnings in query crate * chore: remove unused warnings * chore: fix deprecated warnings for parquet * chore: fix deprecated warning in servers crate * style: fix clippy * test: enlarge mito cache tttl test ttl time * chore: fix typo * style: fmt toml * refactor: reimplement PartialOrd for RangeSelect * chore: remove script crate files introduced by merge * fix: return error if sql option is not kv * chore: do not use ..default::default() * chore: per review * chore: update error message in BuildAdminFunctionArgsSnafu Co-authored-by: jeremyhi <jiachun_feng@proton.me> * refactor: typed precision * update sqlness view case Signed-off-by: Ruihang Xia <waynestxia@gmail.com> * chore: flow per review * chore: add example in comment * chore: warn if parquet stats of timestamp is not INT64 * style: add a newline before derive to make the comment more clear * test: update sqlness result * fix: flow from substrait * chore: change update_range_context log to debug level * chore: move axum-extra axum-macros to workspace --------- Signed-off-by: Ruihang Xia <waynestxia@gmail.com> Co-authored-by: Ruihang Xia <waynestxia@gmail.com> Co-authored-by: luofucong <luofc@foxmail.com> Co-authored-by: discord9 <discord9@163.com> Co-authored-by: shuiyisong <xixing.sys@gmail.com> Co-authored-by: jeremyhi <jiachun_feng@proton.me>
337 lines
10 KiB
Rust
337 lines
10 KiB
Rust
// Copyright 2023 Greptime Team
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
use std::collections::HashMap;
|
|
use std::str::FromStr;
|
|
use std::sync::Arc;
|
|
|
|
use arrow::csv;
|
|
use arrow::csv::reader::Format;
|
|
use arrow::record_batch::RecordBatch;
|
|
use arrow_schema::{Schema, SchemaRef};
|
|
use async_trait::async_trait;
|
|
use common_runtime;
|
|
use datafusion::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener};
|
|
use datafusion::error::Result as DataFusionResult;
|
|
use datafusion::physical_plan::SendableRecordBatchStream;
|
|
use derive_builder::Builder;
|
|
use object_store::ObjectStore;
|
|
use snafu::ResultExt;
|
|
use tokio_util::compat::FuturesAsyncReadCompatExt;
|
|
use tokio_util::io::SyncIoBridge;
|
|
|
|
use super::stream_to_file;
|
|
use crate::buffered_writer::DfRecordBatchEncoder;
|
|
use crate::compression::CompressionType;
|
|
use crate::error::{self, Result};
|
|
use crate::file_format::{self, open_with_decoder, FileFormat};
|
|
use crate::share_buffer::SharedBuffer;
|
|
|
|
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
|
|
pub struct CsvFormat {
|
|
pub has_header: bool,
|
|
pub delimiter: u8,
|
|
pub schema_infer_max_record: Option<usize>,
|
|
pub compression_type: CompressionType,
|
|
}
|
|
|
|
impl TryFrom<&HashMap<String, String>> for CsvFormat {
|
|
type Error = error::Error;
|
|
|
|
fn try_from(value: &HashMap<String, String>) -> Result<Self> {
|
|
let mut format = CsvFormat::default();
|
|
if let Some(delimiter) = value.get(file_format::FORMAT_DELIMITER) {
|
|
// TODO(weny): considers to support parse like "\t" (not only b'\t')
|
|
format.delimiter = u8::from_str(delimiter).map_err(|_| {
|
|
error::ParseFormatSnafu {
|
|
key: file_format::FORMAT_DELIMITER,
|
|
value: delimiter,
|
|
}
|
|
.build()
|
|
})?;
|
|
};
|
|
if let Some(compression_type) = value.get(file_format::FORMAT_COMPRESSION_TYPE) {
|
|
format.compression_type = CompressionType::from_str(compression_type)?;
|
|
};
|
|
if let Some(schema_infer_max_record) =
|
|
value.get(file_format::FORMAT_SCHEMA_INFER_MAX_RECORD)
|
|
{
|
|
format.schema_infer_max_record =
|
|
Some(schema_infer_max_record.parse::<usize>().map_err(|_| {
|
|
error::ParseFormatSnafu {
|
|
key: file_format::FORMAT_SCHEMA_INFER_MAX_RECORD,
|
|
value: schema_infer_max_record,
|
|
}
|
|
.build()
|
|
})?);
|
|
};
|
|
if let Some(has_header) = value.get(file_format::FORMAT_HAS_HEADER) {
|
|
format.has_header = has_header.parse().map_err(|_| {
|
|
error::ParseFormatSnafu {
|
|
key: file_format::FORMAT_HAS_HEADER,
|
|
value: has_header,
|
|
}
|
|
.build()
|
|
})?;
|
|
}
|
|
Ok(format)
|
|
}
|
|
}
|
|
|
|
impl Default for CsvFormat {
|
|
fn default() -> Self {
|
|
Self {
|
|
has_header: true,
|
|
delimiter: b',',
|
|
schema_infer_max_record: Some(file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD),
|
|
compression_type: CompressionType::Uncompressed,
|
|
}
|
|
}
|
|
}
|
|
|
|
#[derive(Debug, Clone, Builder)]
|
|
pub struct CsvConfig {
|
|
batch_size: usize,
|
|
file_schema: SchemaRef,
|
|
#[builder(default = "None")]
|
|
file_projection: Option<Vec<usize>>,
|
|
#[builder(default = "true")]
|
|
has_header: bool,
|
|
#[builder(default = "b','")]
|
|
delimiter: u8,
|
|
}
|
|
|
|
impl CsvConfig {
|
|
fn builder(&self) -> csv::ReaderBuilder {
|
|
let mut builder = csv::ReaderBuilder::new(self.file_schema.clone())
|
|
.with_delimiter(self.delimiter)
|
|
.with_batch_size(self.batch_size)
|
|
.with_header(self.has_header);
|
|
|
|
if let Some(proj) = &self.file_projection {
|
|
builder = builder.with_projection(proj.clone());
|
|
}
|
|
|
|
builder
|
|
}
|
|
}
|
|
|
|
#[derive(Debug, Clone)]
|
|
pub struct CsvOpener {
|
|
config: Arc<CsvConfig>,
|
|
object_store: Arc<ObjectStore>,
|
|
compression_type: CompressionType,
|
|
}
|
|
|
|
impl CsvOpener {
|
|
/// Return a new [`CsvOpener`]. The caller must ensure [`CsvConfig`].file_schema must correspond to the opening file.
|
|
pub fn new(
|
|
config: CsvConfig,
|
|
object_store: ObjectStore,
|
|
compression_type: CompressionType,
|
|
) -> Self {
|
|
CsvOpener {
|
|
config: Arc::new(config),
|
|
object_store: Arc::new(object_store),
|
|
compression_type,
|
|
}
|
|
}
|
|
}
|
|
|
|
impl FileOpener for CsvOpener {
|
|
fn open(&self, meta: FileMeta) -> DataFusionResult<FileOpenFuture> {
|
|
open_with_decoder(
|
|
self.object_store.clone(),
|
|
meta.location().to_string(),
|
|
self.compression_type,
|
|
|| Ok(self.config.builder().build_decoder()),
|
|
)
|
|
}
|
|
}
|
|
|
|
#[async_trait]
|
|
impl FileFormat for CsvFormat {
|
|
async fn infer_schema(&self, store: &ObjectStore, path: &str) -> Result<Schema> {
|
|
let meta = store
|
|
.stat(path)
|
|
.await
|
|
.context(error::ReadObjectSnafu { path })?;
|
|
|
|
let reader = store
|
|
.reader(path)
|
|
.await
|
|
.context(error::ReadObjectSnafu { path })?
|
|
.into_futures_async_read(0..meta.content_length())
|
|
.await
|
|
.context(error::ReadObjectSnafu { path })?
|
|
.compat();
|
|
|
|
let decoded = self.compression_type.convert_async_read(reader);
|
|
|
|
let delimiter = self.delimiter;
|
|
let schema_infer_max_record = self.schema_infer_max_record;
|
|
let has_header = self.has_header;
|
|
|
|
common_runtime::spawn_blocking_global(move || {
|
|
let reader = SyncIoBridge::new(decoded);
|
|
|
|
let format = Format::default()
|
|
.with_delimiter(delimiter)
|
|
.with_header(has_header);
|
|
let (schema, _records_read) = format
|
|
.infer_schema(reader, schema_infer_max_record)
|
|
.context(error::InferSchemaSnafu)?;
|
|
Ok(schema)
|
|
})
|
|
.await
|
|
.context(error::JoinHandleSnafu)?
|
|
}
|
|
}
|
|
|
|
pub async fn stream_to_csv(
|
|
stream: SendableRecordBatchStream,
|
|
store: ObjectStore,
|
|
path: &str,
|
|
threshold: usize,
|
|
concurrency: usize,
|
|
) -> Result<usize> {
|
|
stream_to_file(stream, store, path, threshold, concurrency, |buffer| {
|
|
csv::Writer::new(buffer)
|
|
})
|
|
.await
|
|
}
|
|
|
|
impl DfRecordBatchEncoder for csv::Writer<SharedBuffer> {
|
|
fn write(&mut self, batch: &RecordBatch) -> Result<()> {
|
|
self.write(batch).context(error::WriteRecordBatchSnafu)
|
|
}
|
|
}
|
|
|
|
#[cfg(test)]
|
|
mod tests {
|
|
|
|
use common_test_util::find_workspace_path;
|
|
|
|
use super::*;
|
|
use crate::file_format::{
|
|
FileFormat, FORMAT_COMPRESSION_TYPE, FORMAT_DELIMITER, FORMAT_HAS_HEADER,
|
|
FORMAT_SCHEMA_INFER_MAX_RECORD,
|
|
};
|
|
use crate::test_util::{format_schema, test_store};
|
|
|
|
fn test_data_root() -> String {
|
|
find_workspace_path("/src/common/datasource/tests/csv")
|
|
.display()
|
|
.to_string()
|
|
}
|
|
|
|
#[tokio::test]
|
|
async fn infer_schema_basic() {
|
|
let csv = CsvFormat::default();
|
|
let store = test_store(&test_data_root());
|
|
let schema = csv.infer_schema(&store, "simple.csv").await.unwrap();
|
|
let formatted: Vec<_> = format_schema(schema);
|
|
|
|
assert_eq!(
|
|
vec![
|
|
"c1: Utf8: NULL",
|
|
"c2: Int64: NULL",
|
|
"c3: Int64: NULL",
|
|
"c4: Int64: NULL",
|
|
"c5: Int64: NULL",
|
|
"c6: Int64: NULL",
|
|
"c7: Int64: NULL",
|
|
"c8: Int64: NULL",
|
|
"c9: Int64: NULL",
|
|
"c10: Utf8: NULL",
|
|
"c11: Float64: NULL",
|
|
"c12: Float64: NULL",
|
|
"c13: Utf8: NULL"
|
|
],
|
|
formatted,
|
|
);
|
|
}
|
|
|
|
#[tokio::test]
|
|
async fn infer_schema_with_limit() {
|
|
let json = CsvFormat {
|
|
schema_infer_max_record: Some(3),
|
|
..CsvFormat::default()
|
|
};
|
|
let store = test_store(&test_data_root());
|
|
let schema = json
|
|
.infer_schema(&store, "schema_infer_limit.csv")
|
|
.await
|
|
.unwrap();
|
|
let formatted: Vec<_> = format_schema(schema);
|
|
|
|
assert_eq!(
|
|
vec![
|
|
"a: Int64: NULL",
|
|
"b: Float64: NULL",
|
|
"c: Int64: NULL",
|
|
"d: Int64: NULL"
|
|
],
|
|
formatted
|
|
);
|
|
|
|
let json = CsvFormat::default();
|
|
let store = test_store(&test_data_root());
|
|
let schema = json
|
|
.infer_schema(&store, "schema_infer_limit.csv")
|
|
.await
|
|
.unwrap();
|
|
let formatted: Vec<_> = format_schema(schema);
|
|
|
|
assert_eq!(
|
|
vec![
|
|
"a: Int64: NULL",
|
|
"b: Float64: NULL",
|
|
"c: Int64: NULL",
|
|
"d: Utf8: NULL"
|
|
],
|
|
formatted
|
|
);
|
|
}
|
|
|
|
#[test]
|
|
fn test_try_from() {
|
|
let map = HashMap::new();
|
|
let format: CsvFormat = CsvFormat::try_from(&map).unwrap();
|
|
|
|
assert_eq!(format, CsvFormat::default());
|
|
|
|
let map = HashMap::from([
|
|
(
|
|
FORMAT_SCHEMA_INFER_MAX_RECORD.to_string(),
|
|
"2000".to_string(),
|
|
),
|
|
(FORMAT_COMPRESSION_TYPE.to_string(), "zstd".to_string()),
|
|
(FORMAT_DELIMITER.to_string(), b'\t'.to_string()),
|
|
(FORMAT_HAS_HEADER.to_string(), "false".to_string()),
|
|
]);
|
|
let format = CsvFormat::try_from(&map).unwrap();
|
|
|
|
assert_eq!(
|
|
format,
|
|
CsvFormat {
|
|
compression_type: CompressionType::Zstd,
|
|
schema_infer_max_record: Some(2000),
|
|
delimiter: b'\t',
|
|
has_header: false,
|
|
}
|
|
);
|
|
}
|
|
}
|