From a3d567f0c9d9e4c14310973941583b69b9ade28c Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Wed, 11 Sep 2024 11:31:52 +0800 Subject: [PATCH 001/128] perf(flow): use batch mode for flow (#4599) * generic bundle trait * feat: impl get/let * fix: drop batch * test: tumble batch * feat: use batch eval flow * fix: div use arrow::div not mul * perf: not append batch * perf: use bool mask for reduce * perf: tiny opt * perf: refactor slow path * feat: opt if then * fix: WIP * perf: if then * chore: use trace instead * fix: reduce missing non-first batch * perf: flow if then using interleave * docs: add TODO * perf: remove unnecessary eq * chore: remove unused import * fix: run_available no longer loop forever * feat: blocking on high input buf * chore: increase threhold * chore: after rebase * chore: per review * chore: per review * fix: allow empty values in reduce&test * tests: more flow doc example tests * chore: per review * chore: per review --- src/flow/src/adapter.rs | 350 ++++----- src/flow/src/adapter/flownode_impl.rs | 4 +- src/flow/src/adapter/node_context.rs | 97 ++- src/flow/src/adapter/worker.rs | 34 +- src/flow/src/compute/render.rs | 91 ++- src/flow/src/compute/render/reduce.rs | 422 +++++++---- src/flow/src/compute/render/src_sink.rs | 19 +- src/flow/src/compute/types.rs | 34 +- src/flow/src/expr.rs | 109 ++- src/flow/src/expr/df_func.rs | 16 +- src/flow/src/expr/error.rs | 9 +- src/flow/src/expr/func.rs | 278 +++++--- src/flow/src/expr/linear.rs | 11 +- src/flow/src/expr/relation.rs | 1 + src/flow/src/expr/relation/func.rs | 70 +- src/flow/src/expr/scalar.rs | 186 ++--- src/flow/src/repr.rs | 5 +- src/flow/src/utils.rs | 8 +- .../standalone/common/flow/flow_basic.result | 670 +++++++++++++++++- .../standalone/common/flow/flow_basic.sql | 413 ++++++++++- 20 files changed, 2054 insertions(+), 773 deletions(-) diff --git a/src/flow/src/adapter.rs b/src/flow/src/adapter.rs index 04f7fd80b3..3a8acd60e7 100644 --- a/src/flow/src/adapter.rs +++ b/src/flow/src/adapter.rs @@ -50,11 +50,9 @@ use crate::adapter::util::column_schemas_to_proto; use crate::adapter::worker::{create_worker, Worker, WorkerHandle}; use crate::compute::ErrCollector; use crate::df_optimizer::sql_to_flow_plan; -use crate::error::{ExternalSnafu, InternalSnafu, TableNotFoundSnafu, UnexpectedSnafu}; -use crate::expr::GlobalId; -use crate::metrics::{ - METRIC_FLOW_INPUT_BUF_SIZE, METRIC_FLOW_INSERT_ELAPSED, METRIC_FLOW_RUN_INTERVAL_MS, -}; +use crate::error::{EvalSnafu, ExternalSnafu, InternalSnafu, TableNotFoundSnafu, UnexpectedSnafu}; +use crate::expr::{Batch, GlobalId}; +use crate::metrics::{METRIC_FLOW_INSERT_ELAPSED, METRIC_FLOW_RUN_INTERVAL_MS}; use crate::repr::{self, DiffRow, Row, BATCH_SIZE}; mod flownode_impl; @@ -227,11 +225,24 @@ pub fn diff_row_to_request(rows: Vec) -> Vec { reqs } +pub fn batches_to_rows_req(batches: Vec) -> Result, Error> { + let mut reqs = Vec::new(); + for batch in batches { + let mut rows = Vec::with_capacity(batch.row_count()); + for i in 0..batch.row_count() { + let row = batch.get_row(i).context(EvalSnafu)?; + rows.push((Row::new(row), 0)); + } + reqs.push(DiffRequest::Insert(rows)); + } + Ok(reqs) +} + /// This impl block contains methods to send writeback requests to frontend impl FlowWorkerManager { /// Return the number of requests it made pub async fn send_writeback_requests(&self) -> Result { - let all_reqs = self.generate_writeback_request().await; + let all_reqs = self.generate_writeback_request().await?; if all_reqs.is_empty() || all_reqs.iter().all(|v| v.1.is_empty()) { return Ok(0); } @@ -242,122 +253,16 @@ impl FlowWorkerManager { } let (catalog, schema) = (table_name[0].clone(), table_name[1].clone()); let ctx = Arc::new(QueryContext::with(&catalog, &schema)); - // TODO(discord9): instead of auto build table from request schema, actually build table - // before `create flow` to be able to assign pk and ts etc. - let (primary_keys, schema, is_ts_placeholder) = if let Some(table_id) = self - .table_info_source - .get_table_id_from_name(&table_name) - .await? - { - let table_info = self - .table_info_source - .get_table_info_value(&table_id) - .await? - .unwrap(); - let meta = table_info.table_info.meta; - let primary_keys = meta - .primary_key_indices - .into_iter() - .map(|i| meta.schema.column_schemas[i].name.clone()) - .collect_vec(); - let schema = meta.schema.column_schemas; - // check if the last column is the auto created timestamp column, hence the table is auto created from - // flow's plan type - let is_auto_create = { - let correct_name = schema - .last() - .map(|s| s.name == AUTO_CREATED_PLACEHOLDER_TS_COL) - .unwrap_or(false); - let correct_time_index = meta.schema.timestamp_index == Some(schema.len() - 1); - correct_name && correct_time_index - }; - (primary_keys, schema, is_auto_create) - } else { - // TODO(discord9): condiser remove buggy auto create by schema - let node_ctx = self.node_context.read().await; - let gid: GlobalId = node_ctx - .table_repr - .get_by_name(&table_name) - .map(|x| x.1) - .unwrap(); - let schema = node_ctx - .schema - .get(&gid) - .with_context(|| TableNotFoundSnafu { - name: format!("Table name = {:?}", table_name), - })? - .clone(); - // TODO(discord9): use default key from schema - let primary_keys = schema - .typ() - .keys - .first() - .map(|v| { - v.column_indices - .iter() - .map(|i| { - schema - .get_name(*i) - .clone() - .unwrap_or_else(|| format!("col_{i}")) - }) - .collect_vec() - }) - .unwrap_or_default(); - let update_at = ColumnSchema::new( - UPDATE_AT_TS_COL, - ConcreteDataType::timestamp_millisecond_datatype(), - true, - ); + let (is_ts_placeholder, proto_schema) = + self.try_fetch_or_create_table(&table_name).await?; + let schema_len = proto_schema.len(); - let original_schema = schema - .typ() - .column_types - .clone() - .into_iter() - .enumerate() - .map(|(idx, typ)| { - let name = schema - .names - .get(idx) - .cloned() - .flatten() - .unwrap_or(format!("col_{}", idx)); - let ret = ColumnSchema::new(name, typ.scalar_type, typ.nullable); - if schema.typ().time_index == Some(idx) { - ret.with_time_index(true) - } else { - ret - } - }) - .collect_vec(); - - let mut with_auto_added_col = original_schema.clone(); - with_auto_added_col.push(update_at); - - // if no time index, add one as placeholder - let no_time_index = schema.typ().time_index.is_none(); - if no_time_index { - let ts_col = ColumnSchema::new( - AUTO_CREATED_PLACEHOLDER_TS_COL, - ConcreteDataType::timestamp_millisecond_datatype(), - true, - ) - .with_time_index(true); - with_auto_added_col.push(ts_col); - } - - (primary_keys, with_auto_added_col, no_time_index) - }; - let schema_len = schema.len(); - let proto_schema = column_schemas_to_proto(schema, &primary_keys)?; - - debug!( - "Sending {} writeback requests to table {}, reqs={:?}", + trace!( + "Sending {} writeback requests to table {}, reqs total rows={}", reqs.len(), table_name.join("."), - reqs + reqs.iter().map(|r| r.len()).sum::() ); let now = self.tick_manager.tick(); for req in reqs { @@ -450,8 +355,12 @@ impl FlowWorkerManager { } /// Generate writeback request for all sink table - pub async fn generate_writeback_request(&self) -> BTreeMap> { + pub async fn generate_writeback_request( + &self, + ) -> Result>, Error> { + trace!("Start to generate writeback request"); let mut output = BTreeMap::new(); + let mut total_row_count = 0; for (name, sink_recv) in self .node_context .write() @@ -460,14 +369,133 @@ impl FlowWorkerManager { .iter_mut() .map(|(n, (_s, r))| (n, r)) { - let mut rows = Vec::new(); - while let Ok(row) = sink_recv.try_recv() { - rows.push(row); + let mut batches = Vec::new(); + while let Ok(batch) = sink_recv.try_recv() { + total_row_count += batch.row_count(); + batches.push(batch); } - let reqs = diff_row_to_request(rows); + let reqs = batches_to_rows_req(batches)?; output.insert(name.clone(), reqs); } - output + trace!("Prepare writeback req: total row count={}", total_row_count); + Ok(output) + } + + /// Fetch table info or create table from flow's schema if not exist + async fn try_fetch_or_create_table( + &self, + table_name: &TableName, + ) -> Result<(bool, Vec), Error> { + // TODO(discord9): instead of auto build table from request schema, actually build table + // before `create flow` to be able to assign pk and ts etc. + let (primary_keys, schema, is_ts_placeholder) = if let Some(table_id) = self + .table_info_source + .get_table_id_from_name(table_name) + .await? + { + let table_info = self + .table_info_source + .get_table_info_value(&table_id) + .await? + .unwrap(); + let meta = table_info.table_info.meta; + let primary_keys = meta + .primary_key_indices + .into_iter() + .map(|i| meta.schema.column_schemas[i].name.clone()) + .collect_vec(); + let schema = meta.schema.column_schemas; + // check if the last column is the auto created timestamp column, hence the table is auto created from + // flow's plan type + let is_auto_create = { + let correct_name = schema + .last() + .map(|s| s.name == AUTO_CREATED_PLACEHOLDER_TS_COL) + .unwrap_or(false); + let correct_time_index = meta.schema.timestamp_index == Some(schema.len() - 1); + correct_name && correct_time_index + }; + (primary_keys, schema, is_auto_create) + } else { + // TODO(discord9): condiser remove buggy auto create by schema + + let node_ctx = self.node_context.read().await; + let gid: GlobalId = node_ctx + .table_repr + .get_by_name(table_name) + .map(|x| x.1) + .unwrap(); + let schema = node_ctx + .schema + .get(&gid) + .with_context(|| TableNotFoundSnafu { + name: format!("Table name = {:?}", table_name), + })? + .clone(); + // TODO(discord9): use default key from schema + let primary_keys = schema + .typ() + .keys + .first() + .map(|v| { + v.column_indices + .iter() + .map(|i| { + schema + .get_name(*i) + .clone() + .unwrap_or_else(|| format!("col_{i}")) + }) + .collect_vec() + }) + .unwrap_or_default(); + let update_at = ColumnSchema::new( + UPDATE_AT_TS_COL, + ConcreteDataType::timestamp_millisecond_datatype(), + true, + ); + + let original_schema = schema + .typ() + .column_types + .clone() + .into_iter() + .enumerate() + .map(|(idx, typ)| { + let name = schema + .names + .get(idx) + .cloned() + .flatten() + .unwrap_or(format!("col_{}", idx)); + let ret = ColumnSchema::new(name, typ.scalar_type, typ.nullable); + if schema.typ().time_index == Some(idx) { + ret.with_time_index(true) + } else { + ret + } + }) + .collect_vec(); + + let mut with_auto_added_col = original_schema.clone(); + with_auto_added_col.push(update_at); + + // if no time index, add one as placeholder + let no_time_index = schema.typ().time_index.is_none(); + if no_time_index { + let ts_col = ColumnSchema::new( + AUTO_CREATED_PLACEHOLDER_TS_COL, + ConcreteDataType::timestamp_millisecond_datatype(), + true, + ) + .with_time_index(true); + with_auto_added_col.push(ts_col); + } + + (primary_keys, with_auto_added_col, no_time_index) + }; + let proto_schema = column_schemas_to_proto(schema, &primary_keys)?; + Ok((is_ts_placeholder, proto_schema)) } } @@ -498,10 +526,6 @@ impl FlowWorkerManager { } } - async fn get_buf_size(&self) -> usize { - self.node_context.read().await.get_send_buf_size().await - } - /// Trigger dataflow running, and then send writeback request to the source sender /// /// note that this method didn't handle input mirror request, as this should be handled by grpc server @@ -575,43 +599,37 @@ impl FlowWorkerManager { /// TODO(discord9): add flag for subgraph that have input since last run pub async fn run_available(&self, blocking: bool) -> Result { let mut row_cnt = 0; - loop { - let now = self.tick_manager.tick(); - for worker in self.worker_handles.iter() { - // TODO(discord9): consider how to handle error in individual worker - if blocking { - worker.lock().await.run_available(now, blocking).await?; - } else if let Ok(worker) = worker.try_lock() { - worker.run_available(now, blocking).await?; - } else { - return Ok(row_cnt); - } - } - // check row send and rows remain in send buf - let (flush_res, _buf_len) = if blocking { - let ctx = self.node_context.read().await; - (ctx.flush_all_sender().await, ctx.get_send_buf_size().await) + + let now = self.tick_manager.tick(); + for worker in self.worker_handles.iter() { + // TODO(discord9): consider how to handle error in individual worker + if blocking { + worker.lock().await.run_available(now, blocking).await?; + } else if let Ok(worker) = worker.try_lock() { + worker.run_available(now, blocking).await?; } else { - match self.node_context.try_read() { - Ok(ctx) => (ctx.flush_all_sender().await, ctx.get_send_buf_size().await), - Err(_) => return Ok(row_cnt), - } - }; - match flush_res { - Ok(r) => { - common_telemetry::trace!("Flushed {} rows", r); - row_cnt += r; - // send buf is likely to be somewhere empty now, wait - if r < BATCH_SIZE / 2 { - break; - } - } - Err(err) => { - common_telemetry::error!("Flush send buf errors: {:?}", err); - break; - } - }; + return Ok(row_cnt); + } } + // check row send and rows remain in send buf + let flush_res = if blocking { + let ctx = self.node_context.read().await; + ctx.flush_all_sender().await + } else { + match self.node_context.try_read() { + Ok(ctx) => ctx.flush_all_sender().await, + Err(_) => return Ok(row_cnt), + } + }; + match flush_res { + Ok(r) => { + common_telemetry::trace!("Total flushed {} rows", r); + row_cnt += r; + } + Err(err) => { + common_telemetry::error!("Flush send buf errors: {:?}", err); + } + }; Ok(row_cnt) } @@ -624,14 +642,14 @@ impl FlowWorkerManager { ) -> Result<(), Error> { let rows_len = rows.len(); let table_id = region_id.table_id(); - METRIC_FLOW_INPUT_BUF_SIZE.add(rows_len as _); let _timer = METRIC_FLOW_INSERT_ELAPSED .with_label_values(&[table_id.to_string().as_str()]) .start_timer(); self.node_context.read().await.send(table_id, rows).await?; - debug!( + trace!( "Handling write request for table_id={} with {} rows", - table_id, rows_len + table_id, + rows_len ); Ok(()) } diff --git a/src/flow/src/adapter/flownode_impl.rs b/src/flow/src/adapter/flownode_impl.rs index c2da7af95c..739804be93 100644 --- a/src/flow/src/adapter/flownode_impl.rs +++ b/src/flow/src/adapter/flownode_impl.rs @@ -23,7 +23,7 @@ use api::v1::region::InsertRequests; use common_error::ext::BoxedError; use common_meta::error::{ExternalSnafu, Result, UnexpectedSnafu}; use common_meta::node_manager::Flownode; -use common_telemetry::debug; +use common_telemetry::{debug, trace}; use itertools::Itertools; use snafu::{OptionExt, ResultExt}; use store_api::storage::RegionId; @@ -189,7 +189,7 @@ impl Flownode for FlowWorkerManager { }) .try_collect()?; if !fetch_order.iter().enumerate().all(|(i, &v)| i == v) { - debug!("Reordering columns: {:?}", fetch_order) + trace!("Reordering columns: {:?}", fetch_order) } fetch_order }; diff --git a/src/flow/src/adapter/node_context.rs b/src/flow/src/adapter/node_context.rs index 99bd9f97e9..26e1a6483a 100644 --- a/src/flow/src/adapter/node_context.rs +++ b/src/flow/src/adapter/node_context.rs @@ -15,10 +15,10 @@ //! Node context, prone to change with every incoming requests use std::collections::{BTreeMap, BTreeSet, HashMap}; -use std::sync::atomic::AtomicUsize; +use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; -use common_telemetry::debug; +use common_telemetry::trace; use session::context::QueryContext; use snafu::{OptionExt, ResultExt}; use table::metadata::TableId; @@ -27,9 +27,9 @@ use tokio::sync::{broadcast, mpsc, RwLock}; use crate::adapter::{FlowId, TableName, TableSource}; use crate::error::{Error, EvalSnafu, TableNotFoundSnafu}; use crate::expr::error::InternalSnafu; -use crate::expr::GlobalId; +use crate::expr::{Batch, GlobalId}; use crate::metrics::METRIC_FLOW_INPUT_BUF_SIZE; -use crate::repr::{DiffRow, RelationDesc, BROADCAST_CAP, SEND_BUF_CAP}; +use crate::repr::{DiffRow, RelationDesc, BATCH_SIZE, BROADCAST_CAP, SEND_BUF_CAP}; /// A context that holds the information of the dataflow #[derive(Default, Debug)] @@ -47,13 +47,8 @@ pub struct FlownodeContext { /// /// and send it back to the client, since we are mocking the sink table as a client, we should use table name as the key /// note that the sink receiver should only have one, and we are using broadcast as mpsc channel here - pub sink_receiver: BTreeMap< - TableName, - ( - mpsc::UnboundedSender, - mpsc::UnboundedReceiver, - ), - >, + pub sink_receiver: + BTreeMap, mpsc::UnboundedReceiver)>, /// the schema of the table, query from metasrv or inferred from TypedPlan pub schema: HashMap, /// All the tables that have been registered in the worker @@ -61,25 +56,27 @@ pub struct FlownodeContext { pub query_context: Option>, } -/// a simple broadcast sender with backpressure and unbound capacity +/// a simple broadcast sender with backpressure, bounded capacity and blocking on send when send buf is full +/// note that it wouldn't evict old data, so it's possible to block forever if the receiver is slow /// /// receiver still use tokio broadcast channel, since only sender side need to know /// backpressure and adjust dataflow running duration to avoid blocking #[derive(Debug)] pub struct SourceSender { // TODO(discord9): make it all Vec? - sender: broadcast::Sender, - send_buf_tx: mpsc::Sender>, - send_buf_rx: RwLock>>, + sender: broadcast::Sender, + send_buf_tx: mpsc::Sender, + send_buf_rx: RwLock>, send_buf_row_cnt: AtomicUsize, } impl Default for SourceSender { fn default() -> Self { + // TODO(discord9): the capacity is arbitrary, we can adjust it later, might also want to limit the max number of rows in send buf let (send_buf_tx, send_buf_rx) = mpsc::channel(SEND_BUF_CAP); Self { // TODO(discord9): found a better way then increase this to prevent lagging and hence missing input data - sender: broadcast::Sender::new(BROADCAST_CAP * 2), + sender: broadcast::Sender::new(SEND_BUF_CAP), send_buf_tx, send_buf_rx: RwLock::new(send_buf_rx), send_buf_row_cnt: AtomicUsize::new(0), @@ -90,7 +87,7 @@ impl Default for SourceSender { impl SourceSender { /// max number of iterations to try flush send buf const MAX_ITERATIONS: usize = 16; - pub fn get_receiver(&self) -> broadcast::Receiver { + pub fn get_receiver(&self) -> broadcast::Receiver { self.sender.subscribe() } @@ -106,30 +103,27 @@ impl SourceSender { break; } // TODO(discord9): send rows instead so it's just moving a point - if let Some(rows) = send_buf.recv().await { - let len = rows.len(); - self.send_buf_row_cnt - .fetch_sub(len, std::sync::atomic::Ordering::SeqCst); - for row in rows { - self.sender - .send(row) - .map_err(|err| { - InternalSnafu { - reason: format!("Failed to send row, error = {:?}", err), - } - .build() - }) - .with_context(|_| EvalSnafu)?; - row_cnt += 1; - } + if let Some(batch) = send_buf.recv().await { + let len = batch.row_count(); + self.send_buf_row_cnt.fetch_sub(len, Ordering::SeqCst); + row_cnt += len; + self.sender + .send(batch) + .map_err(|err| { + InternalSnafu { + reason: format!("Failed to send row, error = {:?}", err), + } + .build() + }) + .with_context(|_| EvalSnafu)?; } } if row_cnt > 0 { - debug!("Send {} rows", row_cnt); + trace!("Source Flushed {} rows", row_cnt); METRIC_FLOW_INPUT_BUF_SIZE.sub(row_cnt as _); - debug!( - "Remaining Send buf.len() = {}", - self.send_buf_rx.read().await.len() + trace!( + "Remaining Source Send buf.len() = {}", + METRIC_FLOW_INPUT_BUF_SIZE.get() ); } @@ -138,12 +132,23 @@ impl SourceSender { /// return number of rows it actual send(including what's in the buffer) pub async fn send_rows(&self, rows: Vec) -> Result { - self.send_buf_tx.send(rows).await.map_err(|e| { + METRIC_FLOW_INPUT_BUF_SIZE.add(rows.len() as _); + while self.send_buf_row_cnt.load(Ordering::SeqCst) >= BATCH_SIZE * 4 { + tokio::task::yield_now().await; + } + // row count metrics is approx so relaxed order is ok + self.send_buf_row_cnt + .fetch_add(rows.len(), Ordering::SeqCst); + let batch = Batch::try_from_rows(rows.into_iter().map(|(row, _, _)| row).collect()) + .context(EvalSnafu)?; + common_telemetry::trace!("Send one batch to worker with {} rows", batch.row_count()); + self.send_buf_tx.send(batch).await.map_err(|e| { crate::error::InternalSnafu { reason: format!("Failed to send row, error = {:?}", e), } .build() })?; + Ok(0) } } @@ -159,8 +164,6 @@ impl FlownodeContext { .with_context(|| TableNotFoundSnafu { name: table_id.to_string(), })?; - - debug!("FlownodeContext::send: trying to send {} rows", rows.len()); sender.send_rows(rows).await } @@ -174,16 +177,6 @@ impl FlownodeContext { } Ok(sum) } - - /// Return the sum number of rows in all send buf - /// TODO(discord9): remove this since we can't get correct row cnt anyway - pub async fn get_send_buf_size(&self) -> usize { - let mut sum = 0; - for sender in self.source_sender.values() { - sum += sender.send_buf_rx.read().await.len(); - } - sum - } } impl FlownodeContext { @@ -230,7 +223,7 @@ impl FlownodeContext { pub fn add_sink_receiver(&mut self, table_name: TableName) { self.sink_receiver .entry(table_name) - .or_insert_with(mpsc::unbounded_channel::); + .or_insert_with(mpsc::unbounded_channel); } pub fn get_source_by_global_id(&self, id: &GlobalId) -> Result<&SourceSender, Error> { @@ -254,7 +247,7 @@ impl FlownodeContext { pub fn get_sink_by_global_id( &self, id: &GlobalId, - ) -> Result, Error> { + ) -> Result, Error> { let table_name = self .table_repr .get_by_global_id(id) diff --git a/src/flow/src/adapter/worker.rs b/src/flow/src/adapter/worker.rs index 72ac9df127..978d3c608c 100644 --- a/src/flow/src/adapter/worker.rs +++ b/src/flow/src/adapter/worker.rs @@ -27,7 +27,7 @@ use tokio::sync::{broadcast, mpsc, oneshot, Mutex}; use crate::adapter::FlowId; use crate::compute::{Context, DataflowState, ErrCollector}; use crate::error::{Error, FlowAlreadyExistSnafu, InternalSnafu, UnexpectedSnafu}; -use crate::expr::GlobalId; +use crate::expr::{Batch, GlobalId}; use crate::plan::TypedPlan; use crate::repr::{self, DiffRow}; @@ -89,6 +89,8 @@ impl<'subgraph> ActiveDataflowState<'subgraph> { err_collector: self.err_collector.clone(), input_collection: Default::default(), local_scope: Default::default(), + input_collection_batch: Default::default(), + local_scope_batch: Default::default(), } } @@ -156,13 +158,13 @@ impl WorkerHandle { /// /// the returned error is unrecoverable, and the worker should be shutdown/rebooted pub async fn run_available(&self, now: repr::Timestamp, blocking: bool) -> Result<(), Error> { - common_telemetry::debug!("Running available with blocking={}", blocking); + common_telemetry::trace!("Running available with blocking={}", blocking); if blocking { let resp = self .itc_client .call_with_resp(Request::RunAvail { now, blocking }) .await?; - common_telemetry::debug!("Running available with response={:?}", resp); + common_telemetry::trace!("Running available with response={:?}", resp); Ok(()) } else { self.itc_client @@ -225,9 +227,9 @@ impl<'s> Worker<'s> { flow_id: FlowId, plan: TypedPlan, sink_id: GlobalId, - sink_sender: mpsc::UnboundedSender, + sink_sender: mpsc::UnboundedSender, source_ids: &[GlobalId], - src_recvs: Vec>, + src_recvs: Vec>, // TODO(discord9): set expire duration for all arrangement and compare to sys timestamp instead expire_after: Option, create_if_not_exists: bool, @@ -249,12 +251,12 @@ impl<'s> Worker<'s> { { let mut ctx = cur_task_state.new_ctx(sink_id); for (source_id, src_recv) in source_ids.iter().zip(src_recvs) { - let bundle = ctx.render_source(src_recv)?; - ctx.insert_global(*source_id, bundle); + let bundle = ctx.render_source_batch(src_recv)?; + ctx.insert_global_batch(*source_id, bundle); } - let rendered = ctx.render_plan(plan)?; - ctx.render_unbounded_sink(rendered, sink_sender); + let rendered = ctx.render_plan_batch(plan)?; + ctx.render_unbounded_sink_batch(rendered, sink_sender); } self.task_states.insert(flow_id, cur_task_state); Ok(Some(flow_id)) @@ -370,9 +372,9 @@ pub enum Request { flow_id: FlowId, plan: TypedPlan, sink_id: GlobalId, - sink_sender: mpsc::UnboundedSender, + sink_sender: mpsc::UnboundedSender, source_ids: Vec, - src_recvs: Vec>, + src_recvs: Vec>, expire_after: Option, create_if_not_exists: bool, err_collector: ErrCollector, @@ -472,7 +474,7 @@ mod test { use super::*; use crate::expr::Id; use crate::plan::Plan; - use crate::repr::{RelationType, Row}; + use crate::repr::RelationType; #[test] fn drop_handle() { @@ -497,8 +499,8 @@ mod test { }); let handle = rx.await.unwrap(); let src_ids = vec![GlobalId::User(1)]; - let (tx, rx) = broadcast::channel::(1024); - let (sink_tx, mut sink_rx) = mpsc::unbounded_channel::(); + let (tx, rx) = broadcast::channel::(1024); + let (sink_tx, mut sink_rx) = mpsc::unbounded_channel::(); let (flow_id, plan) = ( 1, TypedPlan { @@ -523,9 +525,9 @@ mod test { handle.create_flow(create_reqs).await.unwrap(), Some(flow_id) ); - tx.send((Row::empty(), 0, 0)).unwrap(); + tx.send(Batch::empty()).unwrap(); handle.run_available(0, true).await.unwrap(); - assert_eq!(sink_rx.recv().await.unwrap().0, Row::empty()); + assert_eq!(sink_rx.recv().await.unwrap(), Batch::empty()); drop(handle); worker_thread_handle.join().unwrap(); } diff --git a/src/flow/src/compute/render.rs b/src/flow/src/compute/render.rs index dcb2194a1d..90f0d86c73 100644 --- a/src/flow/src/compute/render.rs +++ b/src/flow/src/compute/render.rs @@ -49,6 +49,14 @@ pub struct Context<'referred, 'df> { /// /// TODO(discord9): consider if use Vec<(LocalId, CollectionBundle)> instead pub local_scope: Vec>, + /// a list of all collections being used in the operator + /// + /// TODO(discord9): remove extra clone by counting usage and remove it on last usage? + pub input_collection_batch: BTreeMap>, + /// used by `Get`/`Let` Plan for getting/setting local variables + /// + /// TODO(discord9): consider if use Vec<(LocalId, CollectionBundle)> instead + pub local_scope_batch: Vec>>, // Collect all errors in this operator's evaluation pub err_collector: ErrCollector, } @@ -67,6 +75,19 @@ impl<'referred, 'df> Drop for Context<'referred, 'df> { bundle.collection.into_inner().drop(self.df); drop(bundle.arranged); } + + for bundle in std::mem::take(&mut self.input_collection_batch) + .into_values() + .chain( + std::mem::take(&mut self.local_scope_batch) + .into_iter() + .flat_map(|v| v.into_iter()) + .map(|(_k, v)| v), + ) + { + bundle.collection.into_inner().drop(self.df); + drop(bundle.arranged); + } // The automatically generated "drop glue" which recursively calls the destructors of all the fields (including the now empty `input_collection`) } } @@ -84,6 +105,19 @@ impl<'referred, 'df> Context<'referred, 'df> { self.local_scope.push(first); } } + + pub fn insert_global_batch(&mut self, id: GlobalId, collection: CollectionBundle) { + self.input_collection_batch.insert(id, collection); + } + + pub fn insert_local_batch(&mut self, id: LocalId, collection: CollectionBundle) { + if let Some(last) = self.local_scope_batch.last_mut() { + last.insert(id, collection); + } else { + let first = BTreeMap::from([(id, collection)]); + self.local_scope_batch.push(first); + } + } } impl<'referred, 'df> Context<'referred, 'df> { @@ -91,14 +125,8 @@ impl<'referred, 'df> Context<'referred, 'df> { pub fn render_plan_batch(&mut self, plan: TypedPlan) -> Result, Error> { match plan.plan { Plan::Constant { rows } => Ok(self.render_constant_batch(rows)), - Plan::Get { .. } => NotImplementedSnafu { - reason: "Get is still WIP in batchmode", - } - .fail(), - Plan::Let { .. } => NotImplementedSnafu { - reason: "Let is still WIP in batchmode", - } - .fail(), + Plan::Get { id } => self.get_batch_by_id(id), + Plan::Let { id, value, body } => self.eval_batch_let(id, value, body), Plan::Mfp { input, mfp } => self.render_mfp_batch(input, mfp), Plan::Reduce { input, @@ -225,6 +253,32 @@ impl<'referred, 'df> Context<'referred, 'df> { CollectionBundle::from_collection(Collection::from_port(recv_port)) } + pub fn get_batch_by_id(&mut self, id: expr::Id) -> Result, Error> { + let ret = match id { + expr::Id::Local(local) => { + let bundle = self + .local_scope_batch + .iter() + .rev() + .find_map(|scope| scope.get(&local)) + .with_context(|| InvalidQuerySnafu { + reason: format!("Local variable {:?} not found", local), + })?; + bundle.clone(self.df) + } + expr::Id::Global(id) => { + let bundle = + self.input_collection_batch + .get(&id) + .with_context(|| InvalidQuerySnafu { + reason: format!("Collection {:?} not found", id), + })?; + bundle.clone(self.df) + } + }; + Ok(ret) + } + pub fn get_by_id(&mut self, id: expr::Id) -> Result { let ret = match id { expr::Id::Local(local) => { @@ -251,6 +305,21 @@ impl<'referred, 'df> Context<'referred, 'df> { Ok(ret) } + /// Eval `Let` operator, useful for assigning a value to a local variable + pub fn eval_batch_let( + &mut self, + id: LocalId, + value: Box, + body: Box, + ) -> Result, Error> { + let value = self.render_plan_batch(*value)?; + + self.local_scope_batch.push(Default::default()); + self.insert_local_batch(id, value); + let ret = self.render_plan_batch(*body)?; + Ok(ret) + } + /// Eval `Let` operator, useful for assigning a value to a local variable pub fn eval_let( &mut self, @@ -268,11 +337,11 @@ impl<'referred, 'df> Context<'referred, 'df> { } /// The Common argument for all `Subgraph` in the render process -struct SubgraphArg<'a> { +struct SubgraphArg<'a, T = Toff> { now: repr::Timestamp, err_collector: &'a ErrCollector, scheduler: &'a Scheduler, - send: &'a PortCtx, + send: &'a PortCtx, } #[cfg(test)] @@ -345,6 +414,8 @@ mod test { compute_state: state, input_collection: BTreeMap::new(), local_scope: Default::default(), + input_collection_batch: BTreeMap::new(), + local_scope_batch: Default::default(), err_collector, } } diff --git a/src/flow/src/compute/render/reduce.rs b/src/flow/src/compute/render/reduce.rs index e43d3d22a4..8de6b7a17a 100644 --- a/src/flow/src/compute/render/reduce.rs +++ b/src/flow/src/compute/render/reduce.rs @@ -12,14 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, BTreeSet}; use std::ops::Range; use std::sync::Arc; +use common_telemetry::trace; use datatypes::data_type::ConcreteDataType; use datatypes::prelude::DataType; use datatypes::value::{ListValue, Value}; -use datatypes::vectors::NullVector; +use datatypes::vectors::{BooleanVector, NullVector}; use hydroflow::scheduled::graph_ext::GraphExt; use itertools::Itertools; use snafu::{ensure, OptionExt, ResultExt}; @@ -27,8 +28,8 @@ use snafu::{ensure, OptionExt, ResultExt}; use crate::compute::render::{Context, SubgraphArg}; use crate::compute::types::{Arranged, Collection, CollectionBundle, ErrCollector, Toff}; use crate::error::{Error, NotImplementedSnafu, PlanSnafu}; -use crate::expr::error::{DataAlreadyExpiredSnafu, DataTypeSnafu, InternalSnafu}; -use crate::expr::{Batch, EvalError, ScalarExpr}; +use crate::expr::error::{ArrowSnafu, DataAlreadyExpiredSnafu, DataTypeSnafu, InternalSnafu}; +use crate::expr::{Accum, Accumulator, Batch, EvalError, ScalarExpr, VectorDiff}; use crate::plan::{AccumulablePlan, AggrWithIndex, KeyValPlan, ReducePlan, TypedPlan}; use crate::repr::{self, DiffRow, KeyValDiffRow, RelationType, Row}; use crate::utils::{ArrangeHandler, ArrangeReader, ArrangeWriter, KeyExpiryManager}; @@ -93,152 +94,39 @@ impl<'referred, 'df> Context<'referred, 'df> { // TODO(discord9): better way to schedule future run let scheduler = self.compute_state.get_scheduler(); + let scheduler_inner = scheduler.clone(); + let (out_send_port, out_recv_port) = self.df.make_edge::<_, Toff>(Self::REDUCE_BATCH); - let subgraph = - self.df.add_subgraph_in_out( - Self::REDUCE_BATCH, - input.collection.into_inner(), - out_send_port, - move |_ctx, recv, send| { - let now = *(now.borrow()); - let arrange = arrange_handler_inner.clone(); - // mfp only need to passively receive updates from recvs - let src_data = recv - .take_inner() - .into_iter() - .flat_map(|v| v.into_iter()) - .collect_vec(); + let subgraph = self.df.add_subgraph_in_out( + Self::REDUCE_BATCH, + input.collection.into_inner(), + out_send_port, + move |_ctx, recv, send| { + let now = *(now.borrow()); + let arrange = arrange_handler_inner.clone(); + // mfp only need to passively receive updates from recvs + let src_data = recv + .take_inner() + .into_iter() + .flat_map(|v| v.into_iter()) + .collect_vec(); - let mut key_to_many_vals = BTreeMap::::new(); - for batch in src_data { - err_collector.run(|| { - let (key_batch, val_batch) = - batch_split_by_key_val(&batch, &key_val_plan, &err_collector); - ensure!( - key_batch.row_count() == val_batch.row_count(), - InternalSnafu { - reason: format!( - "Key and val batch should have the same row count, found {} and {}", - key_batch.row_count(), - val_batch.row_count() - ) - } - ); - - for row_idx in 0..key_batch.row_count() { - let key_row = key_batch.get_row(row_idx).unwrap(); - let val_row = val_batch.slice(row_idx, 1)?; - let val_batch = - key_to_many_vals.entry(Row::new(key_row)).or_default(); - val_batch.append_batch(val_row)?; - } - - Ok(()) - }); - } - - // write lock the arrange for the rest of the function body - // to prevent wired race condition - let mut arrange = arrange.write(); - let mut all_arrange_updates = Vec::with_capacity(key_to_many_vals.len()); - let mut all_output_rows = Vec::with_capacity(key_to_many_vals.len()); - - for (key, val_batch) in key_to_many_vals { - err_collector.run(|| -> Result<(), _> { - let (accums, _, _) = arrange.get(now, &key).unwrap_or_default(); - let accum_list = from_accum_values_to_live_accums( - accums.unpack(), - accum_plan.simple_aggrs.len(), - )?; - - let mut accum_output = AccumOutput::new(); - for AggrWithIndex { - expr, - input_idx, - output_idx, - } in accum_plan.simple_aggrs.iter() - { - let cur_old_accum = accum_list.get(*output_idx).cloned().unwrap_or_default(); - // if batch is empty, input null instead - let cur_input = val_batch.batch().get(*input_idx).cloned().unwrap_or_else(||Arc::new(NullVector::new(val_batch.row_count()))); - - let (output, new_accum) = - expr.func.eval_batch(cur_old_accum, cur_input, None)?; - - accum_output.insert_accum(*output_idx, new_accum); - accum_output.insert_output(*output_idx, output); - } - - let (new_accums, res_val_row) = accum_output.into_accum_output()?; - - let arrange_update = ((key.clone(), Row::new(new_accums)), now, 1); - all_arrange_updates.push(arrange_update); - - let mut key_val = key; - key_val.extend(res_val_row); - all_output_rows.push((key_val, now, 1)); - - Ok(()) - }); - } - - err_collector.run(|| { - arrange.apply_updates(now, all_arrange_updates)?; - arrange.compact_to(now) - }); - - // this output part is not supposed to be resource intensive - // (because for every batch there wouldn't usually be as many output row?), - // so we can do some costly operation here - let output_types = all_output_rows.first().map(|(row, _, _)| { - row.iter() - .map(|v| v.data_type()) - .collect::>() - }); - - if let Some(output_types) = output_types { - err_collector.run(|| { - let column_cnt = output_types.len(); - let row_cnt = all_output_rows.len(); - - let mut output_builder = output_types - .into_iter() - .map(|t| t.create_mutable_vector(row_cnt)) - .collect_vec(); - - for (row, _, _) in all_output_rows { - for (i, v) in row.into_iter().enumerate() { - output_builder - .get_mut(i) - .context(InternalSnafu{ - reason: format!( - "Output builder should have the same length as the row, expected at most {} but got {}", - column_cnt-1, - i - ) - })? - .try_push_value_ref(v.as_value_ref()) - .context(DataTypeSnafu { - msg: "Failed to push value", - })?; - } - } - - let output_columns = output_builder - .into_iter() - .map(|mut b| b.to_vector()) - .collect_vec(); - - let output_batch = Batch::try_new(output_columns, row_cnt)?; - send.give(vec![output_batch]); - - Ok(()) - }); - } - }, - ); + reduce_batch_subgraph( + &arrange, + src_data, + &key_val_plan, + &accum_plan, + SubgraphArg { + now, + err_collector: &err_collector, + scheduler: &scheduler_inner, + send, + }, + ) + }, + ); scheduler.set_cur_subgraph(subgraph); @@ -461,6 +349,245 @@ fn split_rows_to_key_val( ) } +fn reduce_batch_subgraph( + arrange: &ArrangeHandler, + src_data: impl IntoIterator, + key_val_plan: &KeyValPlan, + accum_plan: &AccumulablePlan, + SubgraphArg { + now, + err_collector, + scheduler: _, + send, + }: SubgraphArg>, +) { + let mut key_to_many_vals = BTreeMap::>::new(); + let mut input_row_count = 0; + let mut input_batch_count = 0; + + for batch in src_data { + input_batch_count += 1; + input_row_count += batch.row_count(); + err_collector.run(|| { + let (key_batch, val_batch) = + batch_split_by_key_val(&batch, key_val_plan, err_collector); + ensure!( + key_batch.row_count() == val_batch.row_count(), + InternalSnafu { + reason: format!( + "Key and val batch should have the same row count, found {} and {}", + key_batch.row_count(), + val_batch.row_count() + ) + } + ); + + let mut distinct_keys = BTreeSet::new(); + for row_idx in 0..key_batch.row_count() { + let key_row = key_batch.get_row(row_idx)?; + let key_row = Row::new(key_row); + + if distinct_keys.contains(&key_row) { + continue; + } else { + distinct_keys.insert(key_row.clone()); + } + } + + // TODO: here reduce numbers of eq to minimal by keeping slicing key/val batch + for key_row in distinct_keys { + let key_scalar_value = { + let mut key_scalar_value = Vec::with_capacity(key_row.len()); + for key in key_row.iter() { + let v = + key.try_to_scalar_value(&key.data_type()) + .context(DataTypeSnafu { + msg: "can't convert key values to datafusion value", + })?; + let arrow_value = + v.to_scalar().context(crate::expr::error::DatafusionSnafu { + context: "can't convert key values to arrow value", + })?; + key_scalar_value.push(arrow_value); + } + key_scalar_value + }; + + // first compute equal from separate columns + let eq_results = key_scalar_value + .into_iter() + .zip(key_batch.batch().iter()) + .map(|(key, col)| { + // TODO(discord9): this takes half of the cpu! And this is redundant amount of `eq`! + arrow::compute::kernels::cmp::eq(&key, &col.to_arrow_array().as_ref() as _) + }) + .try_collect::<_, Vec<_>, _>() + .context(ArrowSnafu { + context: "Failed to compare key values", + })?; + + // then combine all equal results to finally found equal key rows + let opt_eq_mask = eq_results + .into_iter() + .fold(None, |acc, v| match acc { + Some(Ok(acc)) => Some(arrow::compute::kernels::boolean::and(&acc, &v)), + Some(Err(_)) => acc, + None => Some(Ok(v)), + }) + .transpose() + .context(ArrowSnafu { + context: "Failed to combine key comparison results", + })?; + + let key_eq_mask = if let Some(eq_mask) = opt_eq_mask { + BooleanVector::from(eq_mask) + } else { + // if None, meaning key_batch's column number is zero, which means + // the key is empty, so we just return a mask of all true + // meaning taking all values + BooleanVector::from(vec![true; key_batch.row_count()]) + }; + // TODO: both slice and mutate remaining batch + + let cur_val_batch = val_batch.filter(&key_eq_mask)?; + + key_to_many_vals + .entry(key_row) + .or_default() + .push(cur_val_batch); + } + + Ok(()) + }); + } + + trace!( + "Reduce take {} batches, {} rows", + input_batch_count, + input_row_count + ); + + // write lock the arrange for the rest of the function body + // to prevent wired race condition + let mut arrange = arrange.write(); + let mut all_arrange_updates = Vec::with_capacity(key_to_many_vals.len()); + + let mut all_output_dict = BTreeMap::new(); + + for (key, val_batches) in key_to_many_vals { + err_collector.run(|| -> Result<(), _> { + let (accums, _, _) = arrange.get(now, &key).unwrap_or_default(); + let accum_list = + from_accum_values_to_live_accums(accums.unpack(), accum_plan.simple_aggrs.len())?; + + let mut accum_output = AccumOutput::new(); + for AggrWithIndex { + expr, + input_idx, + output_idx, + } in accum_plan.simple_aggrs.iter() + { + let cur_accum_value = accum_list.get(*output_idx).cloned().unwrap_or_default(); + let mut cur_accum = if cur_accum_value.is_empty() { + Accum::new_accum(&expr.func.clone())? + } else { + Accum::try_into_accum(&expr.func, cur_accum_value)? + }; + + for val_batch in val_batches.iter() { + // if batch is empty, input null instead + let cur_input = val_batch + .batch() + .get(*input_idx) + .cloned() + .unwrap_or_else(|| Arc::new(NullVector::new(val_batch.row_count()))); + let len = cur_input.len(); + cur_accum.update_batch(&expr.func, VectorDiff::from(cur_input))?; + + trace!("Reduce accum after take {} rows: {:?}", len, cur_accum); + } + let final_output = cur_accum.eval(&expr.func)?; + trace!("Reduce accum final output: {:?}", final_output); + accum_output.insert_output(*output_idx, final_output); + + let cur_accum_value = cur_accum.into_state(); + accum_output.insert_accum(*output_idx, cur_accum_value); + } + + let (new_accums, res_val_row) = accum_output.into_accum_output()?; + + let arrange_update = ((key.clone(), Row::new(new_accums)), now, 1); + all_arrange_updates.push(arrange_update); + + all_output_dict.insert(key, Row::from(res_val_row)); + + Ok(()) + }); + } + + err_collector.run(|| { + arrange.apply_updates(now, all_arrange_updates)?; + arrange.compact_to(now) + }); + // release the lock + drop(arrange); + + // this output part is not supposed to be resource intensive + // (because for every batch there wouldn't usually be as many output row?), + // so we can do some costly operation here + let output_types = all_output_dict.first_entry().map(|entry| { + entry + .key() + .iter() + .chain(entry.get().iter()) + .map(|v| v.data_type()) + .collect::>() + }); + + if let Some(output_types) = output_types { + err_collector.run(|| { + let column_cnt = output_types.len(); + let row_cnt = all_output_dict.len(); + + let mut output_builder = output_types + .into_iter() + .map(|t| t.create_mutable_vector(row_cnt)) + .collect_vec(); + + for (key, val) in all_output_dict { + for (i, v) in key.into_iter().chain(val.into_iter()).enumerate() { + output_builder + .get_mut(i) + .context(InternalSnafu{ + reason: format!( + "Output builder should have the same length as the row, expected at most {} but got {}", + column_cnt - 1, + i + ) + })? + .try_push_value_ref(v.as_value_ref()) + .context(DataTypeSnafu { + msg: "Failed to push value", + })?; + } + } + + let output_columns = output_builder + .into_iter() + .map(|mut b| b.to_vector()) + .collect_vec(); + + let output_batch = Batch::try_new(output_columns, row_cnt)?; + + trace!("Reduce output batch: {:?}", output_batch); + + send.give(vec![output_batch]); + + Ok(()) + }); + } +} + /// reduce subgraph, reduce the input data into a single row /// output is concat from key and val fn reduce_subgraph( @@ -856,6 +983,9 @@ impl AccumOutput { /// return (accums, output) fn into_accum_output(self) -> Result<(Vec, Vec), EvalError> { + if self.accum.is_empty() && self.output.is_empty() { + return Ok((vec![], vec![])); + } ensure!( !self.accum.is_empty() && self.accum.len() == self.output.len(), InternalSnafu { diff --git a/src/flow/src/compute/render/src_sink.rs b/src/flow/src/compute/render/src_sink.rs index 53141b8cc0..3f5bfaf345 100644 --- a/src/flow/src/compute/render/src_sink.rs +++ b/src/flow/src/compute/render/src_sink.rs @@ -16,7 +16,7 @@ use std::collections::{BTreeMap, VecDeque}; -use common_telemetry::debug; +use common_telemetry::{debug, trace}; use hydroflow::scheduled::graph_ext::GraphExt; use itertools::Itertools; use snafu::OptionExt; @@ -48,10 +48,13 @@ impl<'referred, 'df> Context<'referred, 'df> { let sub = self .df .add_subgraph_source("source_batch", send_port, move |_ctx, send| { + let mut total_batches = vec![]; + let mut total_row_count = 0; loop { match src_recv.try_recv() { Ok(batch) => { - send.give(vec![batch]); + total_row_count += batch.row_count(); + total_batches.push(batch); } Err(TryRecvError::Empty) => { break; @@ -78,6 +81,13 @@ impl<'referred, 'df> Context<'referred, 'df> { } } + trace!( + "Send {} rows in {} batches", + total_row_count, + total_batches.len() + ); + send.give(total_batches); + let now = *now.borrow(); // always schedule source to run at now so we can // repeatedly run source if needed @@ -185,13 +195,18 @@ impl<'referred, 'df> Context<'referred, 'df> { collection.into_inner(), move |_ctx, recv| { let data = recv.take_inner(); + let mut row_count = 0; + let mut batch_count = 0; for batch in data.into_iter().flat_map(|i| i.into_iter()) { + row_count += batch.row_count(); + batch_count += 1; // if the sender is closed unexpectedly, stop sending if sender.is_closed() || sender.send(batch).is_err() { common_telemetry::error!("UnboundedSinkBatch is closed"); break; } } + trace!("sink send {} rows in {} batches", row_count, batch_count); }, ); } diff --git a/src/flow/src/compute/types.rs b/src/flow/src/compute/types.rs index f7cbebae27..00ed660a6e 100644 --- a/src/flow/src/compute/types.rs +++ b/src/flow/src/compute/types.rs @@ -24,7 +24,7 @@ use hydroflow::scheduled::SubgraphId; use itertools::Itertools; use tokio::sync::Mutex; -use crate::expr::{EvalError, ScalarExpr}; +use crate::expr::{Batch, EvalError, ScalarExpr}; use crate::repr::DiffRow; use crate::utils::ArrangeHandler; @@ -123,6 +123,38 @@ pub struct CollectionBundle { pub arranged: BTreeMap, Arranged>, } +pub trait GenericBundle { + fn is_batch(&self) -> bool; + + fn try_as_batch(&self) -> Option<&CollectionBundle> { + None + } + + fn try_as_row(&self) -> Option<&CollectionBundle> { + None + } +} + +impl GenericBundle for CollectionBundle { + fn is_batch(&self) -> bool { + true + } + + fn try_as_batch(&self) -> Option<&CollectionBundle> { + Some(self) + } +} + +impl GenericBundle for CollectionBundle { + fn is_batch(&self) -> bool { + false + } + + fn try_as_row(&self) -> Option<&CollectionBundle> { + Some(self) + } +} + impl CollectionBundle { pub fn from_collection(collection: Collection) -> Self { Self { diff --git a/src/flow/src/expr.rs b/src/flow/src/expr.rs index 1f8160716a..2e6019ba4c 100644 --- a/src/flow/src/expr.rs +++ b/src/flow/src/expr.rs @@ -16,27 +16,29 @@ mod df_func; pub(crate) mod error; -mod func; +pub(crate) mod func; mod id; mod linear; -mod relation; +pub(crate) mod relation; mod scalar; mod signature; +use arrow::compute::FilterBuilder; use datatypes::prelude::DataType; use datatypes::value::Value; -use datatypes::vectors::VectorRef; +use datatypes::vectors::{BooleanVector, Helper, VectorRef}; pub(crate) use df_func::{DfScalarFunction, RawDfScalarFn}; pub(crate) use error::{EvalError, InvalidArgumentSnafu}; pub(crate) use func::{BinaryFunc, UnaryFunc, UnmaterializableFunc, VariadicFunc}; pub(crate) use id::{GlobalId, Id, LocalId}; use itertools::Itertools; pub(crate) use linear::{MapFilterProject, MfpPlan, SafeMfpPlan}; -pub(crate) use relation::{AggregateExpr, AggregateFunc}; +pub(crate) use relation::{Accum, Accumulator, AggregateExpr, AggregateFunc}; pub(crate) use scalar::{ScalarExpr, TypedExpr}; use snafu::{ensure, ResultExt}; -use crate::expr::error::DataTypeSnafu; +use crate::expr::error::{ArrowSnafu, DataTypeSnafu}; +use crate::repr::Diff; pub const TUMBLE_START: &str = "tumble_start"; pub const TUMBLE_END: &str = "tumble_end"; @@ -179,7 +181,9 @@ impl Batch { ) } ); - Ok(self.batch.iter().map(|v| v.get(idx)).collect_vec()) + let mut ret = Vec::with_capacity(self.column_count()); + ret.extend(self.batch.iter().map(|v| v.get(idx))); + Ok(ret) } /// Slices the `Batch`, returning a new `Batch`. @@ -248,4 +252,97 @@ impl Batch { self.row_count = self_row_count + other_row_count; Ok(()) } + + /// filter the batch with given predicate + pub fn filter(&self, predicate: &BooleanVector) -> Result { + let len = predicate.as_boolean_array().true_count(); + let filter_builder = FilterBuilder::new(predicate.as_boolean_array()).optimize(); + let filter_pred = filter_builder.build(); + let filtered = self + .batch() + .iter() + .map(|col| filter_pred.filter(col.to_arrow_array().as_ref())) + .try_collect::<_, Vec<_>, _>() + .context(ArrowSnafu { + context: "Failed to filter val batches", + })?; + let res_vector = Helper::try_into_vectors(&filtered).context(DataTypeSnafu { + msg: "can't convert arrow array to vector", + })?; + Self::try_new(res_vector, len) + } +} + +/// Vector with diff to note the insert and delete +pub(crate) struct VectorDiff { + vector: VectorRef, + diff: Option, +} + +impl From for VectorDiff { + fn from(vector: VectorRef) -> Self { + Self { vector, diff: None } + } +} + +impl VectorDiff { + fn len(&self) -> usize { + self.vector.len() + } + + fn try_new(vector: VectorRef, diff: Option) -> Result { + ensure!( + diff.as_ref() + .map_or(true, |diff| diff.len() == vector.len()), + InvalidArgumentSnafu { + reason: "Length of vector and diff should be the same" + } + ); + Ok(Self { vector, diff }) + } +} + +impl IntoIterator for VectorDiff { + type Item = (Value, Diff); + type IntoIter = VectorDiffIter; + + fn into_iter(self) -> Self::IntoIter { + VectorDiffIter { + vector: self.vector, + diff: self.diff, + idx: 0, + } + } +} + +/// iterator for VectorDiff +pub(crate) struct VectorDiffIter { + vector: VectorRef, + diff: Option, + idx: usize, +} + +impl std::iter::Iterator for VectorDiffIter { + type Item = (Value, Diff); + + fn next(&mut self) -> Option { + if self.idx >= self.vector.len() { + return None; + } + let value = self.vector.get(self.idx); + // +1 means insert, -1 means delete, and default to +1 insert when diff is not provided + let diff = if let Some(diff) = self.diff.as_ref() { + if let Ok(diff_at) = diff.get(self.idx).try_into() { + diff_at + } else { + common_telemetry::warn!("Invalid diff value at index {}", self.idx); + return None; + } + } else { + 1 + }; + + self.idx += 1; + Some((value, diff)) + } } diff --git a/src/flow/src/expr/df_func.rs b/src/flow/src/expr/df_func.rs index b0a2648dd1..620615c9eb 100644 --- a/src/flow/src/expr/df_func.rs +++ b/src/flow/src/expr/df_func.rs @@ -92,12 +92,8 @@ impl DfScalarFunction { let len = rb.num_rows(); - let res = self.fn_impl.evaluate(&rb).map_err(|err| { - EvalDatafusionSnafu { - raw: err, - context: "Failed to evaluate datafusion scalar function", - } - .build() + let res = self.fn_impl.evaluate(&rb).context(EvalDatafusionSnafu { + context: "Failed to evaluate datafusion scalar function", })?; let res = common_query::columnar_value::ColumnarValue::try_from(&res) .map_err(BoxedError::new) @@ -157,12 +153,8 @@ impl DfScalarFunction { .into_error(err) })?; - let res = self.fn_impl.evaluate(&rb).map_err(|err| { - EvalDatafusionSnafu { - raw: err, - context: "Failed to evaluate datafusion scalar function", - } - .build() + let res = self.fn_impl.evaluate(&rb).context(EvalDatafusionSnafu { + context: "Failed to evaluate datafusion scalar function", })?; let res = common_query::columnar_value::ColumnarValue::try_from(&res) .map_err(BoxedError::new) diff --git a/src/flow/src/expr/error.rs b/src/flow/src/expr/error.rs index 6703ce2404..4b69b3df23 100644 --- a/src/flow/src/expr/error.rs +++ b/src/flow/src/expr/error.rs @@ -106,18 +106,19 @@ pub enum EvalError { location: Location, }, - #[snafu(display("Arrow error: {raw:?}, context: {context}"))] + #[snafu(display("Arrow error: {error:?}, context: {context}"))] Arrow { #[snafu(source)] - raw: ArrowError, + error: ArrowError, context: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("DataFusion error: {raw:?}, context: {context}"))] + #[snafu(display("DataFusion error: {error:?}, context: {context}"))] Datafusion { - raw: DataFusionError, + #[snafu(source)] + error: DataFusionError, context: String, #[snafu(implicit)] location: Location, diff --git a/src/flow/src/expr/func.rs b/src/flow/src/expr/func.rs index 65da763e27..36a1029725 100644 --- a/src/flow/src/expr/func.rs +++ b/src/flow/src/expr/func.rs @@ -967,7 +967,7 @@ impl BinaryFunc { | Self::DivUInt32 | Self::DivUInt64 | Self::DivFloat32 - | Self::DivFloat64 => arrow::compute::kernels::numeric::mul(&left, &right) + | Self::DivFloat64 => arrow::compute::kernels::numeric::div(&left, &right) .context(ArrowSnafu { context: "div" })?, Self::ModInt16 @@ -1280,119 +1280,183 @@ where Ok(Value::from(left % right)) } -#[test] -fn test_num_ops() { - let left = Value::from(10); - let right = Value::from(3); - let res = add::(left.clone(), right.clone()).unwrap(); - assert_eq!(res, Value::from(13)); - let res = sub::(left.clone(), right.clone()).unwrap(); - assert_eq!(res, Value::from(7)); - let res = mul::(left.clone(), right.clone()).unwrap(); - assert_eq!(res, Value::from(30)); - let res = div::(left.clone(), right.clone()).unwrap(); - assert_eq!(res, Value::from(3)); - let res = rem::(left, right).unwrap(); - assert_eq!(res, Value::from(1)); +#[cfg(test)] +mod test { + use std::sync::Arc; - let values = vec![Value::from(true), Value::from(false)]; - let exprs = vec![ScalarExpr::Column(0), ScalarExpr::Column(1)]; - let res = and(&values, &exprs).unwrap(); - assert_eq!(res, Value::from(false)); - let res = or(&values, &exprs).unwrap(); - assert_eq!(res, Value::from(true)); -} + use common_time::Interval; + use datatypes::vectors::Vector; + use pretty_assertions::assert_eq; -/// test if the binary function specialization works -/// whether from direct type or from the expression that is literal -#[test] -fn test_binary_func_spec() { - assert_eq!( - BinaryFunc::from_str_expr_and_type( - "add", - &[ScalarExpr::Column(0), ScalarExpr::Column(0)], - &[ - Some(ConcreteDataType::int32_datatype()), - Some(ConcreteDataType::int32_datatype()) - ] - ) - .unwrap(), - (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) - ); + use super::*; - assert_eq!( - BinaryFunc::from_str_expr_and_type( - "add", - &[ScalarExpr::Column(0), ScalarExpr::Column(0)], - &[Some(ConcreteDataType::int32_datatype()), None] - ) - .unwrap(), - (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) - ); + #[test] + fn test_tumble_batch() { + let datetime_vector = DateTimeVector::from_vec(vec![1, 2, 10, 13, 14, 20, 25]); + let tumble_start = UnaryFunc::TumbleWindowFloor { + window_size: Interval::from_day_time(0, 10), + start_time: None, + }; + let tumble_end = UnaryFunc::TumbleWindowCeiling { + window_size: Interval::from_day_time(0, 10), + start_time: None, + }; - assert_eq!( - BinaryFunc::from_str_expr_and_type( - "add", - &[ScalarExpr::Column(0), ScalarExpr::Column(0)], - &[Some(ConcreteDataType::int32_datatype()), None] - ) - .unwrap(), - (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) - ); + let len = datetime_vector.len(); + let batch = Batch::try_new(vec![Arc::new(datetime_vector)], len).unwrap(); + let arg = ScalarExpr::Column(0); - assert_eq!( - BinaryFunc::from_str_expr_and_type( - "add", - &[ScalarExpr::Column(0), ScalarExpr::Column(0)], - &[Some(ConcreteDataType::int32_datatype()), None] - ) - .unwrap(), - (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) - ); + let start = tumble_start.eval_batch(&batch, &arg).unwrap(); + let end = tumble_end.eval_batch(&batch, &arg).unwrap(); + assert_eq!( + start.to_arrow_array().as_ref(), + TimestampMillisecondVector::from_vec(vec![0, 0, 10, 10, 10, 20, 20]) + .to_arrow_array() + .as_ref() + ); - assert_eq!( - BinaryFunc::from_str_expr_and_type( - "add", - &[ - ScalarExpr::Literal(Value::from(1i32), ConcreteDataType::int32_datatype()), - ScalarExpr::Column(0) - ], - &[None, None] - ) - .unwrap(), - (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) - ); + assert_eq!( + end.to_arrow_array().as_ref(), + TimestampMillisecondVector::from_vec(vec![10, 10, 20, 20, 20, 30, 30]) + .to_arrow_array() + .as_ref() + ); - // this testcase make sure the specialization can find actual type from expression and fill in signature - assert_eq!( - BinaryFunc::from_str_expr_and_type( - "equal", - &[ - ScalarExpr::Literal(Value::from(1i32), ConcreteDataType::int32_datatype()), - ScalarExpr::Column(0) - ], - &[None, None] - ) - .unwrap(), - ( - BinaryFunc::Eq, - Signature { - input: smallvec![ - ConcreteDataType::int32_datatype(), - ConcreteDataType::int32_datatype() + let ts_ms_vector = TimestampMillisecondVector::from_vec(vec![1, 2, 10, 13, 14, 20, 25]); + let batch = Batch::try_new(vec![Arc::new(ts_ms_vector)], len).unwrap(); + + let start = tumble_start.eval_batch(&batch, &arg).unwrap(); + let end = tumble_end.eval_batch(&batch, &arg).unwrap(); + + assert_eq!( + start.to_arrow_array().as_ref(), + TimestampMillisecondVector::from_vec(vec![0, 0, 10, 10, 10, 20, 20]) + .to_arrow_array() + .as_ref() + ); + + assert_eq!( + end.to_arrow_array().as_ref(), + TimestampMillisecondVector::from_vec(vec![10, 10, 20, 20, 20, 30, 30]) + .to_arrow_array() + .as_ref() + ); + } + + #[test] + fn test_num_ops() { + let left = Value::from(10); + let right = Value::from(3); + let res = add::(left.clone(), right.clone()).unwrap(); + assert_eq!(res, Value::from(13)); + let res = sub::(left.clone(), right.clone()).unwrap(); + assert_eq!(res, Value::from(7)); + let res = mul::(left.clone(), right.clone()).unwrap(); + assert_eq!(res, Value::from(30)); + let res = div::(left.clone(), right.clone()).unwrap(); + assert_eq!(res, Value::from(3)); + let res = rem::(left, right).unwrap(); + assert_eq!(res, Value::from(1)); + + let values = vec![Value::from(true), Value::from(false)]; + let exprs = vec![ScalarExpr::Column(0), ScalarExpr::Column(1)]; + let res = and(&values, &exprs).unwrap(); + assert_eq!(res, Value::from(false)); + let res = or(&values, &exprs).unwrap(); + assert_eq!(res, Value::from(true)); + } + + /// test if the binary function specialization works + /// whether from direct type or from the expression that is literal + #[test] + fn test_binary_func_spec() { + assert_eq!( + BinaryFunc::from_str_expr_and_type( + "add", + &[ScalarExpr::Column(0), ScalarExpr::Column(0)], + &[ + Some(ConcreteDataType::int32_datatype()), + Some(ConcreteDataType::int32_datatype()) + ] + ) + .unwrap(), + (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) + ); + + assert_eq!( + BinaryFunc::from_str_expr_and_type( + "add", + &[ScalarExpr::Column(0), ScalarExpr::Column(0)], + &[Some(ConcreteDataType::int32_datatype()), None] + ) + .unwrap(), + (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) + ); + + assert_eq!( + BinaryFunc::from_str_expr_and_type( + "add", + &[ScalarExpr::Column(0), ScalarExpr::Column(0)], + &[Some(ConcreteDataType::int32_datatype()), None] + ) + .unwrap(), + (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) + ); + + assert_eq!( + BinaryFunc::from_str_expr_and_type( + "add", + &[ScalarExpr::Column(0), ScalarExpr::Column(0)], + &[Some(ConcreteDataType::int32_datatype()), None] + ) + .unwrap(), + (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) + ); + + assert_eq!( + BinaryFunc::from_str_expr_and_type( + "add", + &[ + ScalarExpr::Literal(Value::from(1i32), ConcreteDataType::int32_datatype()), + ScalarExpr::Column(0) ], - output: ConcreteDataType::boolean_datatype(), - generic_fn: GenericFn::Eq - } - ) - ); + &[None, None] + ) + .unwrap(), + (BinaryFunc::AddInt32, BinaryFunc::AddInt32.signature()) + ); - matches!( - BinaryFunc::from_str_expr_and_type( - "add", - &[ScalarExpr::Column(0), ScalarExpr::Column(0)], - &[None, None] - ), - Err(Error::InvalidQuery { .. }) - ); + // this testcase make sure the specialization can find actual type from expression and fill in signature + assert_eq!( + BinaryFunc::from_str_expr_and_type( + "equal", + &[ + ScalarExpr::Literal(Value::from(1i32), ConcreteDataType::int32_datatype()), + ScalarExpr::Column(0) + ], + &[None, None] + ) + .unwrap(), + ( + BinaryFunc::Eq, + Signature { + input: smallvec![ + ConcreteDataType::int32_datatype(), + ConcreteDataType::int32_datatype() + ], + output: ConcreteDataType::boolean_datatype(), + generic_fn: GenericFn::Eq + } + ) + ); + + matches!( + BinaryFunc::from_str_expr_and_type( + "add", + &[ScalarExpr::Column(0), ScalarExpr::Column(0)], + &[None, None] + ), + Err(Error::InvalidQuery { .. }) + ); + } } diff --git a/src/flow/src/expr/linear.rs b/src/flow/src/expr/linear.rs index 3185bdffcf..8e220f7d86 100644 --- a/src/flow/src/expr/linear.rs +++ b/src/flow/src/expr/linear.rs @@ -17,8 +17,9 @@ use std::collections::{BTreeMap, BTreeSet}; use arrow::array::BooleanArray; +use arrow::buffer::BooleanBuffer; use arrow::compute::FilterBuilder; -use common_telemetry::debug; +use common_telemetry::trace; use datatypes::prelude::ConcreteDataType; use datatypes::value::Value; use datatypes::vectors::{BooleanVector, Helper}; @@ -500,7 +501,7 @@ impl SafeMfpPlan { for col in batch.batch() { let filtered = pred .filter(col.to_arrow_array().as_ref()) - .context(ArrowSnafu { + .with_context(|_| ArrowSnafu { context: format!("failed to filter column for mfp operator {:?}", self), })?; result.push(Helper::try_into_vector(filtered).context(DataTypeSnafu { @@ -523,7 +524,9 @@ impl SafeMfpPlan { // mark the columns that have been evaluated and appended to the `batch` let mut expression = 0; // preds default to true and will be updated as we evaluate each predicate - let mut all_preds = BooleanVector::from(vec![Some(true); batch.row_count()]); + let buf = BooleanBuffer::new_set(batch.row_count()); + let arr = BooleanArray::new(buf, None); + let mut all_preds = BooleanVector::from(arr); // to compute predicate, need to first compute all expressions used in predicates for (support, predicate) in self.mfp.predicates.iter() { @@ -793,7 +796,7 @@ impl MfpPlan { if Some(lower_bound) != upper_bound && !null_eval { if self.mfp.mfp.projection.iter().any(|c| values.len() <= *c) { - debug!("values={:?}, mfp={:?}", &values, &self.mfp.mfp); + trace!("values={:?}, mfp={:?}", &values, &self.mfp.mfp); let err = InternalSnafu { reason: format!( "Index out of bound for mfp={:?} and values={:?}", diff --git a/src/flow/src/expr/relation.rs b/src/flow/src/expr/relation.rs index 3661db4ff0..b5d7e4ef20 100644 --- a/src/flow/src/expr/relation.rs +++ b/src/flow/src/expr/relation.rs @@ -14,6 +14,7 @@ //! Describes an aggregation function and it's input expression. +pub(crate) use accum::{Accum, Accumulator}; pub(crate) use func::AggregateFunc; use crate::expr::ScalarExpr; diff --git a/src/flow/src/expr/relation/func.rs b/src/flow/src/expr/relation/func.rs index afcdb7ddd1..fd72b58bb1 100644 --- a/src/flow/src/expr/relation/func.rs +++ b/src/flow/src/expr/relation/func.rs @@ -21,14 +21,14 @@ use datatypes::value::Value; use datatypes::vectors::VectorRef; use serde::{Deserialize, Serialize}; use smallvec::smallvec; -use snafu::{ensure, IntoError, OptionExt}; +use snafu::{IntoError, OptionExt}; use strum::{EnumIter, IntoEnumIterator}; use crate::error::{DatafusionSnafu, Error, InvalidQuerySnafu}; use crate::expr::error::EvalError; use crate::expr::relation::accum::{Accum, Accumulator}; use crate::expr::signature::{GenericFn, Signature}; -use crate::expr::InvalidArgumentSnafu; +use crate::expr::VectorDiff; use crate::repr::Diff; /// Aggregate functions that can be applied to a group of rows. @@ -161,72 +161,6 @@ impl AggregateFunc { } } -struct VectorDiff { - vector: VectorRef, - diff: Option, -} - -impl VectorDiff { - fn len(&self) -> usize { - self.vector.len() - } - - fn try_new(vector: VectorRef, diff: Option) -> Result { - ensure!( - diff.as_ref() - .map_or(true, |diff| diff.len() == vector.len()), - InvalidArgumentSnafu { - reason: "Length of vector and diff should be the same" - } - ); - Ok(Self { vector, diff }) - } -} - -impl IntoIterator for VectorDiff { - type Item = (Value, Diff); - type IntoIter = VectorDiffIter; - - fn into_iter(self) -> Self::IntoIter { - VectorDiffIter { - vector: self.vector, - diff: self.diff, - idx: 0, - } - } -} - -struct VectorDiffIter { - vector: VectorRef, - diff: Option, - idx: usize, -} - -impl std::iter::Iterator for VectorDiffIter { - type Item = (Value, Diff); - - fn next(&mut self) -> Option { - if self.idx >= self.vector.len() { - return None; - } - let value = self.vector.get(self.idx); - // +1 means insert, -1 means delete, and default to +1 insert when diff is not provided - let diff = if let Some(diff) = self.diff.as_ref() { - if let Ok(diff_at) = diff.get(self.idx).try_into() { - diff_at - } else { - common_telemetry::warn!("Invalid diff value at index {}", self.idx); - return None; - } - } else { - 1 - }; - - self.idx += 1; - Some((value, diff)) - } -} - /// Generate signature for each aggregate function macro_rules! generate_signature { ($value:ident, diff --git a/src/flow/src/expr/scalar.rs b/src/flow/src/expr/scalar.rs index 6a9cf30d95..a6e00cce5b 100644 --- a/src/flow/src/expr/scalar.rs +++ b/src/flow/src/expr/scalar.rs @@ -16,17 +16,20 @@ use std::collections::{BTreeMap, BTreeSet}; +use arrow::array::{make_array, ArrayData, ArrayRef}; use common_error::ext::BoxedError; use datatypes::prelude::{ConcreteDataType, DataType}; use datatypes::value::Value; -use datatypes::vectors::{BooleanVector, Helper, NullVector, Vector, VectorRef}; +use datatypes::vectors::{BooleanVector, Helper, VectorRef}; +use hydroflow::lattices::cc_traits::Iter; +use itertools::Itertools; use snafu::{ensure, OptionExt, ResultExt}; use crate::error::{ DatafusionSnafu, Error, InvalidQuerySnafu, UnexpectedSnafu, UnsupportedTemporalFilterSnafu, }; use crate::expr::error::{ - DataTypeSnafu, EvalError, InternalSnafu, InvalidArgumentSnafu, OptimizeSnafu, TypeMismatchSnafu, + ArrowSnafu, DataTypeSnafu, EvalError, InvalidArgumentSnafu, OptimizeSnafu, TypeMismatchSnafu, }; use crate::expr::func::{BinaryFunc, UnaryFunc, UnmaterializableFunc, VariadicFunc}; use crate::expr::{Batch, DfScalarFunction}; @@ -222,6 +225,8 @@ impl ScalarExpr { } } + /// NOTE: this if then eval impl assume all given expr are pure, and will not change the state of the world + /// since it will evaluate both then and else branch and filter the result fn eval_if_then( batch: &Batch, cond: &ScalarExpr, @@ -240,130 +245,69 @@ impl ScalarExpr { })? .as_boolean_array(); - let mut then_input_batch = None; - let mut else_input_batch = None; - let mut null_input_batch = None; + let indices = bool_conds + .into_iter() + .enumerate() + .map(|(idx, b)| { + ( + match b { + Some(true) => 0, // then branch vector + Some(false) => 1, // else branch vector + None => 2, // null vector + }, + idx, + ) + }) + .collect_vec(); - // instructions for how to reassembly result vector, - // iterate over (type of vec, offset, length) and append to resulting vec - let mut assembly_idx = vec![]; + let then_input_vec = then.eval_batch(batch)?; + let else_input_vec = els.eval_batch(batch)?; - // append batch, returning appended batch's slice in (offset, length) - fn append_batch( - batch: &mut Option, - to_be_append: Batch, - ) -> Result<(usize, usize), EvalError> { - let len = to_be_append.row_count(); - if let Some(batch) = batch { - let offset = batch.row_count(); - batch.append_batch(to_be_append)?; - Ok((offset, len)) - } else { - *batch = Some(to_be_append); - Ok((0, len)) + ensure!( + then_input_vec.data_type() == else_input_vec.data_type(), + TypeMismatchSnafu { + expected: then_input_vec.data_type(), + actual: else_input_vec.data_type(), } + ); + + ensure!( + then_input_vec.len() == else_input_vec.len() && then_input_vec.len() == batch.row_count(), + InvalidArgumentSnafu { + reason: format!( + "then and else branch must have the same length(found {} and {}) which equals input batch's row count(which is {})", + then_input_vec.len(), + else_input_vec.len(), + batch.row_count() + ) + } + ); + + fn new_nulls(dt: &arrow_schema::DataType, len: usize) -> ArrayRef { + let data = ArrayData::new_null(dt, len); + make_array(data) } - let mut prev_cond: Option> = None; - let mut prev_start_idx: Option = None; - // first put different conds' vector into different batches - for (idx, cond) in bool_conds.iter().enumerate() { - // if belong to same slice and not last one continue - if prev_cond == Some(cond) { - continue; - } else if let Some(prev_cond_idx) = prev_start_idx { - let prev_cond = prev_cond.unwrap(); + let null_input_vec = new_nulls( + &then_input_vec.data_type().as_arrow_type(), + batch.row_count(), + ); - // put a slice to corresponding batch - let slice_offset = prev_cond_idx; - let slice_length = idx - prev_cond_idx; - let to_be_append = batch.slice(slice_offset, slice_length)?; + let interleave_values = vec![ + then_input_vec.to_arrow_array(), + else_input_vec.to_arrow_array(), + null_input_vec, + ]; + let int_ref: Vec<_> = interleave_values.iter().map(|x| x.as_ref()).collect(); - let to_put_back = match prev_cond { - Some(true) => ( - Some(true), - append_batch(&mut then_input_batch, to_be_append)?, - ), - Some(false) => ( - Some(false), - append_batch(&mut else_input_batch, to_be_append)?, - ), - None => (None, append_batch(&mut null_input_batch, to_be_append)?), - }; - assembly_idx.push(to_put_back); - } - prev_cond = Some(cond); - prev_start_idx = Some(idx); - } - - // deal with empty and last slice case - if let Some(slice_offset) = prev_start_idx { - let prev_cond = prev_cond.unwrap(); - let slice_length = bool_conds.len() - slice_offset; - let to_be_append = batch.slice(slice_offset, slice_length)?; - let to_put_back = match prev_cond { - Some(true) => ( - Some(true), - append_batch(&mut then_input_batch, to_be_append)?, - ), - Some(false) => ( - Some(false), - append_batch(&mut else_input_batch, to_be_append)?, - ), - None => (None, append_batch(&mut null_input_batch, to_be_append)?), - }; - assembly_idx.push(to_put_back); - } - - let then_output_vec = then_input_batch - .map(|batch| then.eval_batch(&batch)) - .transpose()?; - let else_output_vec = else_input_batch - .map(|batch| els.eval_batch(&batch)) - .transpose()?; - let null_output_vec = null_input_batch - .map(|null| NullVector::new(null.row_count()).slice(0, null.row_count())); - - let dt = then_output_vec - .as_ref() - .map(|v| v.data_type()) - .or(else_output_vec.as_ref().map(|v| v.data_type())) - .unwrap_or(ConcreteDataType::null_datatype()); - let mut builder = dt.create_mutable_vector(conds.len()); - for (cond, (offset, length)) in assembly_idx { - let slice = match cond { - Some(true) => then_output_vec.as_ref(), - Some(false) => else_output_vec.as_ref(), - None => null_output_vec.as_ref(), - } - .context(InternalSnafu { - reason: "Expect corresponding output vector to exist", + let interleave_res_arr = + arrow::compute::interleave(&int_ref, &indices).context(ArrowSnafu { + context: "Failed to interleave output arrays", })?; - // TODO(discord9): seems `extend_slice_of` doesn't support NullVector or ConstantVector - // consider adding it maybe? - if slice.data_type().is_null() { - builder.push_nulls(length); - } else if slice.is_const() { - let arr = slice.slice(offset, length).to_arrow_array(); - let vector = Helper::try_into_vector(arr).context(DataTypeSnafu { - msg: "Failed to convert arrow array to vector", - })?; - builder - .extend_slice_of(vector.as_ref(), 0, vector.len()) - .context(DataTypeSnafu { - msg: "Failed to build result vector for if-then expression", - })?; - } else { - builder - .extend_slice_of(slice.as_ref(), offset, length) - .context(DataTypeSnafu { - msg: "Failed to build result vector for if-then expression", - })?; - } - } - let result_vec = builder.to_vector(); - - Ok(result_vec) + let res_vec = Helper::try_into_vector(interleave_res_arr).context(DataTypeSnafu { + msg: "Failed to convert arrow array to vector", + })?; + Ok(res_vec) } /// Eval this expression with the given values. @@ -685,7 +629,7 @@ impl ScalarExpr { #[cfg(test)] mod test { - use datatypes::vectors::Int32Vector; + use datatypes::vectors::{Int32Vector, Vector}; use pretty_assertions::assert_eq; use super::*; @@ -781,7 +725,7 @@ mod test { } #[test] - fn test_eval_batch() { + fn test_eval_batch_if_then() { // TODO(discord9): add more tests { let expr = ScalarExpr::If { @@ -840,7 +784,7 @@ mod test { let vectors = vec![Int32Vector::from(raw).slice(0, raw_len)]; let batch = Batch::try_new(vectors, raw_len).unwrap(); - let expected = NullVector::new(raw_len).slice(0, raw_len); + let expected = Int32Vector::from(vec![]).slice(0, raw_len); assert_eq!(expr.eval_batch(&batch).unwrap(), expected); } } diff --git a/src/flow/src/repr.rs b/src/flow/src/repr.rs index 7b57fc3ed2..acec71ebcf 100644 --- a/src/flow/src/repr.rs +++ b/src/flow/src/repr.rs @@ -50,12 +50,13 @@ pub type KeyValDiffRow = ((Row, Row), Timestamp, Diff); /// broadcast channel capacity, can be important to memory consumption, since this influence how many /// updates can be buffered in memory in the entire dataflow /// TODO(discord9): add config for this, so cpu&mem usage can be balanced and configured by this -pub const BROADCAST_CAP: usize = 65535; +pub const BROADCAST_CAP: usize = 1024; /// The maximum capacity of the send buffer, to prevent the buffer from growing too large pub const SEND_BUF_CAP: usize = BROADCAST_CAP * 2; -pub const BATCH_SIZE: usize = BROADCAST_CAP / 2; +/// Flow worker will try to at least accumulate this many rows before processing them(if one second havn't passed) +pub const BATCH_SIZE: usize = 32 * 16384; /// Convert a value that is or can be converted to Datetime to internal timestamp /// diff --git a/src/flow/src/utils.rs b/src/flow/src/utils.rs index 269c53fa84..1cd5b3ba5c 100644 --- a/src/flow/src/utils.rs +++ b/src/flow/src/utils.rs @@ -18,7 +18,7 @@ use std::collections::{BTreeMap, BTreeSet}; use std::ops::Bound; use std::sync::Arc; -use common_telemetry::debug; +use common_telemetry::trace; use smallvec::{smallvec, SmallVec}; use tokio::sync::RwLock; @@ -235,9 +235,11 @@ impl Arrangement { if let Some(s) = &mut self.expire_state { if let Some(expired_by) = s.get_expire_duration_and_update_event_ts(now, &key)? { max_expired_by = max_expired_by.max(Some(expired_by)); - debug!( + trace!( "Expired key: {:?}, expired by: {:?} with time being now={}", - key, expired_by, now + key, + expired_by, + now ); continue; } diff --git a/tests/cases/standalone/common/flow/flow_basic.result b/tests/cases/standalone/common/flow/flow_basic.result index db3d3c8c3b..6aaa2c74be 100644 --- a/tests/cases/standalone/common/flow/flow_basic.result +++ b/tests/cases/standalone/common/flow/flow_basic.result @@ -7,10 +7,13 @@ CREATE TABLE numbers_input_basic ( Affected Rows: 0 -CREATE FLOW test_numbers_basic -SINK TO out_num_cnt_basic -AS -SELECT sum(number) FROM numbers_input_basic GROUP BY tumble(ts, '1 second', '2021-07-01 00:00:00'); +CREATE FLOW test_numbers_basic SINK TO out_num_cnt_basic AS +SELECT + sum(number) +FROM + numbers_input_basic +GROUP BY + tumble(ts, '1 second', '2021-07-01 00:00:00'); Affected Rows: 0 @@ -25,7 +28,8 @@ admin flush_flow('test_numbers_basic'); +----------------------------------------+ -- SQLNESS ARG restart=true -INSERT INTO numbers_input_basic +INSERT INTO + numbers_input_basic VALUES (20, "2021-07-01 00:00:00.200"), (22, "2021-07-01 00:00:00.600"); @@ -40,7 +44,12 @@ admin flush_flow('test_numbers_basic'); | 1 | +----------------------------------------+ -SELECT "SUM(numbers_input_basic.number)", window_start, window_end FROM out_num_cnt_basic; +SELECT + "SUM(numbers_input_basic.number)", + window_start, + window_end +FROM + out_num_cnt_basic; +---------------------------------+---------------------+---------------------+ | SUM(numbers_input_basic.number) | window_start | window_end | @@ -56,10 +65,11 @@ admin flush_flow('test_numbers_basic'); | 0 | +----------------------------------------+ -INSERT INTO numbers_input_basic +INSERT INTO + numbers_input_basic VALUES - (23,"2021-07-01 00:00:01.000"), - (24,"2021-07-01 00:00:01.500"); + (23, "2021-07-01 00:00:01.000"), + (24, "2021-07-01 00:00:01.500"); Affected Rows: 2 @@ -72,7 +82,12 @@ admin flush_flow('test_numbers_basic'); +----------------------------------------+ -- note that this quote-unquote column is a column-name, **not** a aggregation expr, generated by datafusion -SELECT "SUM(numbers_input_basic.number)", window_start, window_end FROM out_num_cnt_basic; +SELECT + "SUM(numbers_input_basic.number)", + window_start, + window_end +FROM + out_num_cnt_basic; +---------------------------------+---------------------+---------------------+ | SUM(numbers_input_basic.number) | window_start | window_end | @@ -93,6 +108,114 @@ DROP TABLE out_num_cnt_basic; Affected Rows: 0 +-- test distinct +CREATE TABLE distinct_basic ( + number INT, + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + PRIMARY KEY(number), + TIME INDEX(ts) +); + +Affected Rows: 0 + +CREATE FLOW test_distinct_basic SINK TO out_distinct_basic AS +SELECT + DISTINCT number as dis +FROM + distinct_basic; + +Affected Rows: 0 + +-- TODO(discord9): confirm if it's necessary to flush flow here? +-- because flush_flow result is at most 1 +admin flush_flow('test_distinct_basic'); + ++-----------------------------------------+ +| ADMIN flush_flow('test_distinct_basic') | ++-----------------------------------------+ +| 0 | ++-----------------------------------------+ + +-- SQLNESS ARG restart=true +INSERT INTO + distinct_basic +VALUES + (20, "2021-07-01 00:00:00.200"), + (22, "2021-07-01 00:00:00.600"); + +Affected Rows: 2 + +admin flush_flow('test_distinct_basic'); + ++-----------------------------------------+ +| ADMIN flush_flow('test_distinct_basic') | ++-----------------------------------------+ +| 1 | ++-----------------------------------------+ + +SELECT + dis +FROM + out_distinct_basic; + ++-----+ +| dis | ++-----+ +| 20 | +| 22 | ++-----+ + +admin flush_flow('test_distinct_basic'); + ++-----------------------------------------+ +| ADMIN flush_flow('test_distinct_basic') | ++-----------------------------------------+ +| 0 | ++-----------------------------------------+ + +INSERT INTO + distinct_basic +VALUES + (23, "2021-07-01 00:00:01.000"), + (24, "2021-07-01 00:00:01.500"); + +Affected Rows: 2 + +admin flush_flow('test_distinct_basic'); + ++-----------------------------------------+ +| ADMIN flush_flow('test_distinct_basic') | ++-----------------------------------------+ +| 1 | ++-----------------------------------------+ + +-- note that this quote-unquote column is a column-name, **not** a aggregation expr, generated by datafusion +SELECT + dis +FROM + out_distinct_basic; + ++-----+ +| dis | ++-----+ +| 20 | +| 22 | +| 23 | +| 24 | ++-----+ + +DROP FLOW test_distinct_basic; + +Affected Rows: 0 + +DROP TABLE distinct_basic; + +Affected Rows: 0 + +DROP TABLE out_distinct_basic; + +Affected Rows: 0 + -- test interprete interval CREATE TABLE numbers_input_basic ( number INT, @@ -105,11 +228,20 @@ Affected Rows: 0 create table out_num_cnt_basic ( number INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP TIME INDEX); + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP TIME INDEX +); Affected Rows: 0 -CREATE FLOW filter_numbers_basic SINK TO out_num_cnt_basic AS SELECT INTERVAL '1 day 1 second', INTERVAL '1 month 1 day 1 second', INTERVAL '1 year 1 month' FROM numbers_input_basic where number > 10; +CREATE FLOW filter_numbers_basic SINK TO out_num_cnt_basic AS +SELECT + INTERVAL '1 day 1 second', + INTERVAL '1 month 1 day 1 second', + INTERVAL '1 year 1 month' +FROM + numbers_input_basic +where + number > 10; Affected Rows: 0 @@ -137,7 +269,8 @@ Affected Rows: 0 CREATE TABLE bytes_log ( byte INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, -- event time + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + -- event time TIME INDEX(ts) ); @@ -153,16 +286,22 @@ CREATE TABLE approx_rate ( Affected Rows: 0 -CREATE FLOW find_approx_rate -SINK TO approx_rate -AS -SELECT (max(byte) - min(byte))/30.0 as rate, date_bin(INTERVAL '30 second', ts) as time_window from bytes_log GROUP BY time_window; +CREATE FLOW find_approx_rate SINK TO approx_rate AS +SELECT + (max(byte) - min(byte)) / 30.0 as rate, + date_bin(INTERVAL '30 second', ts) as time_window +from + bytes_log +GROUP BY + time_window; Affected Rows: 0 -INSERT INTO bytes_log VALUES -(101, '2025-01-01 00:00:01'), -(300, '2025-01-01 00:00:29'); +INSERT INTO + bytes_log +VALUES + (101, '2025-01-01 00:00:01'), + (300, '2025-01-01 00:00:29'); Affected Rows: 2 @@ -174,7 +313,11 @@ admin flush_flow('find_approx_rate'); | 1 | +--------------------------------------+ -SELECT rate, time_window FROM approx_rate; +SELECT + rate, + time_window +FROM + approx_rate; +-------------------+---------------------+ | rate | time_window | @@ -182,9 +325,11 @@ SELECT rate, time_window FROM approx_rate; | 6.633333333333334 | 2025-01-01T00:00:00 | +-------------------+---------------------+ -INSERT INTO bytes_log VALUES -(450, '2025-01-01 00:00:32'), -(500, '2025-01-01 00:00:37'); +INSERT INTO + bytes_log +VALUES + (450, '2025-01-01 00:00:32'), + (500, '2025-01-01 00:00:37'); Affected Rows: 2 @@ -196,7 +341,11 @@ admin flush_flow('find_approx_rate'); | 1 | +--------------------------------------+ -SELECT rate, time_window FROM approx_rate; +SELECT + rate, + time_window +FROM + approx_rate; +--------------------+---------------------+ | rate | time_window | @@ -217,3 +366,474 @@ DROP TABLE approx_rate; Affected Rows: 0 +-- input table +CREATE TABLE ngx_access_log ( + client STRING, + country STRING, + access_time TIMESTAMP TIME INDEX +); + +Affected Rows: 0 + +-- create flow task to calculate the distinct country +CREATE FLOW calc_ngx_country SINK TO ngx_country AS +SELECT + DISTINCT country, +FROM + ngx_access_log; + +Affected Rows: 0 + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 0); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('calc_ngx_country'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_country') | ++--------------------------------------+ +| 1 | ++--------------------------------------+ + +SELECT + "ngx_access_log.country" +FROM + ngx_country; + ++------------------------+ +| ngx_access_log.country | ++------------------------+ +| b | ++------------------------+ + +-- making sure distinct is working +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 1); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('calc_ngx_country'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_country') | ++--------------------------------------+ +| 1 | ++--------------------------------------+ + +SELECT + "ngx_access_log.country" +FROM + ngx_country; + ++------------------------+ +| ngx_access_log.country | ++------------------------+ +| b | ++------------------------+ + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "c", 2); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('calc_ngx_country'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_country') | ++--------------------------------------+ +| 1 | ++--------------------------------------+ + +SELECT + "ngx_access_log.country" +FROM + ngx_country; + ++------------------------+ +| ngx_access_log.country | ++------------------------+ +| b | +| c | ++------------------------+ + +DROP FLOW calc_ngx_country; + +Affected Rows: 0 + +DROP TABLE ngx_access_log; + +Affected Rows: 0 + +DROP TABLE ngx_country; + +Affected Rows: 0 + +CREATE TABLE ngx_access_log ( + client STRING, + country STRING, + access_time TIMESTAMP TIME INDEX +); + +Affected Rows: 0 + +CREATE FLOW calc_ngx_country SINK TO ngx_country AS +SELECT + DISTINCT country, + -- this distinct is not necessary, but it's a good test to see if it works + date_bin(INTERVAL '1 hour', access_time) as time_window, +FROM + ngx_access_log +GROUP BY + country, + time_window; + +Affected Rows: 0 + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 0); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('calc_ngx_country'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_country') | ++--------------------------------------+ +| 1 | ++--------------------------------------+ + +SELECT + "ngx_access_log.country", + time_window +FROM + ngx_country; + ++------------------------+---------------------+ +| ngx_access_log.country | time_window | ++------------------------+---------------------+ +| b | 1970-01-01T00:00:00 | ++------------------------+---------------------+ + +-- making sure distinct is working +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 1); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('calc_ngx_country'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_country') | ++--------------------------------------+ +| 1 | ++--------------------------------------+ + +SELECT + "ngx_access_log.country", + time_window +FROM + ngx_country; + ++------------------------+---------------------+ +| ngx_access_log.country | time_window | ++------------------------+---------------------+ +| b | 1970-01-01T00:00:00 | ++------------------------+---------------------+ + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "c", 2); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('calc_ngx_country'); + ++--------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_country') | ++--------------------------------------+ +| 1 | ++--------------------------------------+ + +SELECT + "ngx_access_log.country", + time_window +FROM + ngx_country; + ++------------------------+---------------------+ +| ngx_access_log.country | time_window | ++------------------------+---------------------+ +| b | 1970-01-01T00:00:00 | +| c | 1970-01-01T00:00:00 | ++------------------------+---------------------+ + +DROP FLOW calc_ngx_country; + +Affected Rows: 0 + +DROP TABLE ngx_access_log; + +Affected Rows: 0 + +DROP TABLE ngx_country; + +Affected Rows: 0 + +CREATE TABLE temp_sensor_data ( + sensor_id INT, + loc STRING, + temperature DOUBLE, + ts TIMESTAMP TIME INDEX +); + +Affected Rows: 0 + +CREATE TABLE temp_alerts ( + sensor_id INT, + loc STRING, + max_temp DOUBLE, + ts TIMESTAMP TIME INDEX +); + +Affected Rows: 0 + +CREATE FLOW temp_monitoring SINK TO temp_alerts AS +SELECT + sensor_id, + loc, + max(temperature) as max_temp, +FROM + temp_sensor_data +GROUP BY + sensor_id, + loc +HAVING + max_temp > 100; + +Affected Rows: 0 + +INSERT INTO + temp_sensor_data +VALUES + (1, "room1", 50, 0); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('temp_monitoring'); + ++-------------------------------------+ +| ADMIN FLUSH_FLOW('temp_monitoring') | ++-------------------------------------+ +| 1 | ++-------------------------------------+ + +-- This table should not exist yet +SHOW TABLES LIKE 'temp_alerts'; + ++-------------+ +| Tables | ++-------------+ +| temp_alerts | ++-------------+ + +INSERT INTO + temp_sensor_data +VALUES + (1, "room1", 150, 1); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('temp_monitoring'); + ++-------------------------------------+ +| ADMIN FLUSH_FLOW('temp_monitoring') | ++-------------------------------------+ +| 1 | ++-------------------------------------+ + +SHOW TABLES LIKE 'temp_alerts'; + ++-------------+ +| Tables | ++-------------+ +| temp_alerts | ++-------------+ + +SELECT + sensor_id, + loc, + max_temp +FROM + temp_alerts; + ++-----------+-------+----------+ +| sensor_id | loc | max_temp | ++-----------+-------+----------+ +| 1 | room1 | 150.0 | ++-----------+-------+----------+ + +INSERT INTO + temp_sensor_data +VALUES + (2, "room1", 0, 2); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('temp_monitoring'); + ++-------------------------------------+ +| ADMIN FLUSH_FLOW('temp_monitoring') | ++-------------------------------------+ +| 1 | ++-------------------------------------+ + +SELECT + sensor_id, + loc, + max_temp +FROM + temp_alerts; + ++-----------+-------+----------+ +| sensor_id | loc | max_temp | ++-----------+-------+----------+ +| 1 | room1 | 150.0 | ++-----------+-------+----------+ + +DROP FLOW temp_monitoring; + +Affected Rows: 0 + +DROP TABLE temp_sensor_data; + +Affected Rows: 0 + +DROP TABLE temp_alerts; + +Affected Rows: 0 + +CREATE TABLE ngx_access_log ( + client STRING, + stat INT, + size INT, + access_time TIMESTAMP TIME INDEX +); + +Affected Rows: 0 + +CREATE TABLE ngx_distribution ( + stat INT, + bucket_size INT, + total_logs BIGINT, + time_window TIMESTAMP TIME INDEX, + update_at TIMESTAMP, -- auto generated column by flow engine + PRIMARY KEY(stat, bucket_size) +); + +Affected Rows: 0 + +CREATE FLOW calc_ngx_distribution SINK TO ngx_distribution AS +SELECT + stat, + trunc(size, -1)::INT as bucket_size, + count(client) AS total_logs, + date_bin(INTERVAL '1 minutes', access_time) as time_window, +FROM + ngx_access_log +GROUP BY + stat, + time_window, + bucket_size; + +Affected Rows: 0 + +INSERT INTO + ngx_access_log +VALUES + ("cli1", 200, 100, 0); + +Affected Rows: 1 + +ADMIN FLUSH_FLOW('calc_ngx_distribution'); + ++-------------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_distribution') | ++-------------------------------------------+ +| 1 | ++-------------------------------------------+ + +SELECT + stat, + bucket_size, + total_logs, + time_window +FROM + ngx_distribution; + ++------+-------------+------------+---------------------+ +| stat | bucket_size | total_logs | time_window | ++------+-------------+------------+---------------------+ +| 200 | 100 | 1 | 1970-01-01T00:00:00 | ++------+-------------+------------+---------------------+ + +INSERT INTO + ngx_access_log +VALUES + ("cli1", 200, 200, 1), + ("cli1", 200, 205, 1), + ("cli1", 200, 209, 1), + ("cli1", 200, 210, 1), + ("cli2", 200, 300, 1); + +Affected Rows: 5 + +ADMIN FLUSH_FLOW('calc_ngx_distribution'); + ++-------------------------------------------+ +| ADMIN FLUSH_FLOW('calc_ngx_distribution') | ++-------------------------------------------+ +| 1 | ++-------------------------------------------+ + +SELECT + stat, + bucket_size, + total_logs, + time_window +FROM + ngx_distribution; + ++------+-------------+------------+---------------------+ +| stat | bucket_size | total_logs | time_window | ++------+-------------+------------+---------------------+ +| 200 | 100 | 1 | 1970-01-01T00:00:00 | +| 200 | 200 | 1 | 1970-01-01T00:00:00 | +| 200 | 210 | 3 | 1970-01-01T00:00:00 | +| 200 | 300 | 1 | 1970-01-01T00:00:00 | ++------+-------------+------------+---------------------+ + +DROP FLOW calc_ngx_distribution; + +Affected Rows: 0 + +DROP TABLE ngx_access_log; + +Affected Rows: 0 + +DROP TABLE ngx_distribution; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/flow/flow_basic.sql b/tests/cases/standalone/common/flow/flow_basic.sql index b9ccc81058..9356508aba 100644 --- a/tests/cases/standalone/common/flow/flow_basic.sql +++ b/tests/cases/standalone/common/flow/flow_basic.sql @@ -5,54 +5,134 @@ CREATE TABLE numbers_input_basic ( TIME INDEX(ts) ); -CREATE FLOW test_numbers_basic -SINK TO out_num_cnt_basic -AS -SELECT sum(number) FROM numbers_input_basic GROUP BY tumble(ts, '1 second', '2021-07-01 00:00:00'); +CREATE FLOW test_numbers_basic SINK TO out_num_cnt_basic AS +SELECT + sum(number) +FROM + numbers_input_basic +GROUP BY + tumble(ts, '1 second', '2021-07-01 00:00:00'); -- TODO(discord9): confirm if it's necessary to flush flow here? -- because flush_flow result is at most 1 admin flush_flow('test_numbers_basic'); -- SQLNESS ARG restart=true -INSERT INTO numbers_input_basic +INSERT INTO + numbers_input_basic VALUES (20, "2021-07-01 00:00:00.200"), (22, "2021-07-01 00:00:00.600"); admin flush_flow('test_numbers_basic'); -SELECT "SUM(numbers_input_basic.number)", window_start, window_end FROM out_num_cnt_basic; +SELECT + "SUM(numbers_input_basic.number)", + window_start, + window_end +FROM + out_num_cnt_basic; admin flush_flow('test_numbers_basic'); -INSERT INTO numbers_input_basic +INSERT INTO + numbers_input_basic VALUES - (23,"2021-07-01 00:00:01.000"), - (24,"2021-07-01 00:00:01.500"); + (23, "2021-07-01 00:00:01.000"), + (24, "2021-07-01 00:00:01.500"); admin flush_flow('test_numbers_basic'); -- note that this quote-unquote column is a column-name, **not** a aggregation expr, generated by datafusion -SELECT "SUM(numbers_input_basic.number)", window_start, window_end FROM out_num_cnt_basic; +SELECT + "SUM(numbers_input_basic.number)", + window_start, + window_end +FROM + out_num_cnt_basic; DROP FLOW test_numbers_basic; + DROP TABLE numbers_input_basic; + DROP TABLE out_num_cnt_basic; --- test interprete interval +-- test distinct +CREATE TABLE distinct_basic ( + number INT, + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + PRIMARY KEY(number), + TIME INDEX(ts) +); +CREATE FLOW test_distinct_basic SINK TO out_distinct_basic AS +SELECT + DISTINCT number as dis +FROM + distinct_basic; + +-- TODO(discord9): confirm if it's necessary to flush flow here? +-- because flush_flow result is at most 1 +admin flush_flow('test_distinct_basic'); + +-- SQLNESS ARG restart=true +INSERT INTO + distinct_basic +VALUES + (20, "2021-07-01 00:00:00.200"), + (22, "2021-07-01 00:00:00.600"); + +admin flush_flow('test_distinct_basic'); + +SELECT + dis +FROM + out_distinct_basic; + +admin flush_flow('test_distinct_basic'); + +INSERT INTO + distinct_basic +VALUES + (23, "2021-07-01 00:00:01.000"), + (24, "2021-07-01 00:00:01.500"); + +admin flush_flow('test_distinct_basic'); + +-- note that this quote-unquote column is a column-name, **not** a aggregation expr, generated by datafusion +SELECT + dis +FROM + out_distinct_basic; + +DROP FLOW test_distinct_basic; + +DROP TABLE distinct_basic; + +DROP TABLE out_distinct_basic; + +-- test interprete interval CREATE TABLE numbers_input_basic ( number INT, ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, PRIMARY KEY(number), TIME INDEX(ts) ); + create table out_num_cnt_basic ( number INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP TIME INDEX); + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP TIME INDEX +); -CREATE FLOW filter_numbers_basic SINK TO out_num_cnt_basic AS SELECT INTERVAL '1 day 1 second', INTERVAL '1 month 1 day 1 second', INTERVAL '1 year 1 month' FROM numbers_input_basic where number > 10; +CREATE FLOW filter_numbers_basic SINK TO out_num_cnt_basic AS +SELECT + INTERVAL '1 day 1 second', + INTERVAL '1 month 1 day 1 second', + INTERVAL '1 year 1 month' +FROM + numbers_input_basic +where + number > 10; SHOW CREATE FLOW filter_numbers_basic; @@ -64,7 +144,8 @@ drop table numbers_input_basic; CREATE TABLE bytes_log ( byte INT, - ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, -- event time + ts TIMESTAMP DEFAULT CURRENT_TIMESTAMP, + -- event time TIME INDEX(ts) ); @@ -76,27 +157,307 @@ CREATE TABLE approx_rate ( TIME INDEX(time_window) ); -CREATE FLOW find_approx_rate -SINK TO approx_rate -AS -SELECT (max(byte) - min(byte))/30.0 as rate, date_bin(INTERVAL '30 second', ts) as time_window from bytes_log GROUP BY time_window; +CREATE FLOW find_approx_rate SINK TO approx_rate AS +SELECT + (max(byte) - min(byte)) / 30.0 as rate, + date_bin(INTERVAL '30 second', ts) as time_window +from + bytes_log +GROUP BY + time_window; -INSERT INTO bytes_log VALUES -(101, '2025-01-01 00:00:01'), -(300, '2025-01-01 00:00:29'); +INSERT INTO + bytes_log +VALUES + (101, '2025-01-01 00:00:01'), + (300, '2025-01-01 00:00:29'); admin flush_flow('find_approx_rate'); -SELECT rate, time_window FROM approx_rate; +SELECT + rate, + time_window +FROM + approx_rate; -INSERT INTO bytes_log VALUES -(450, '2025-01-01 00:00:32'), -(500, '2025-01-01 00:00:37'); +INSERT INTO + bytes_log +VALUES + (450, '2025-01-01 00:00:32'), + (500, '2025-01-01 00:00:37'); admin flush_flow('find_approx_rate'); -SELECT rate, time_window FROM approx_rate; +SELECT + rate, + time_window +FROM + approx_rate; DROP TABLE bytes_log; + DROP FLOW find_approx_rate; + DROP TABLE approx_rate; + +-- input table +CREATE TABLE ngx_access_log ( + client STRING, + country STRING, + access_time TIMESTAMP TIME INDEX +); + +-- create flow task to calculate the distinct country +CREATE FLOW calc_ngx_country SINK TO ngx_country AS +SELECT + DISTINCT country, +FROM + ngx_access_log; + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 0); + +ADMIN FLUSH_FLOW('calc_ngx_country'); + +SELECT + "ngx_access_log.country" +FROM + ngx_country; + +-- making sure distinct is working +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 1); + +ADMIN FLUSH_FLOW('calc_ngx_country'); + +SELECT + "ngx_access_log.country" +FROM + ngx_country; + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "c", 2); + +ADMIN FLUSH_FLOW('calc_ngx_country'); + +SELECT + "ngx_access_log.country" +FROM + ngx_country; + +DROP FLOW calc_ngx_country; + +DROP TABLE ngx_access_log; + +DROP TABLE ngx_country; + +CREATE TABLE ngx_access_log ( + client STRING, + country STRING, + access_time TIMESTAMP TIME INDEX +); + +CREATE FLOW calc_ngx_country SINK TO ngx_country AS +SELECT + DISTINCT country, + -- this distinct is not necessary, but it's a good test to see if it works + date_bin(INTERVAL '1 hour', access_time) as time_window, +FROM + ngx_access_log +GROUP BY + country, + time_window; + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 0); + +ADMIN FLUSH_FLOW('calc_ngx_country'); + +SELECT + "ngx_access_log.country", + time_window +FROM + ngx_country; + +-- making sure distinct is working +INSERT INTO + ngx_access_log +VALUES + ("cli1", "b", 1); + +ADMIN FLUSH_FLOW('calc_ngx_country'); + +SELECT + "ngx_access_log.country", + time_window +FROM + ngx_country; + +INSERT INTO + ngx_access_log +VALUES + ("cli1", "c", 2); + +ADMIN FLUSH_FLOW('calc_ngx_country'); + +SELECT + "ngx_access_log.country", + time_window +FROM + ngx_country; + +DROP FLOW calc_ngx_country; + +DROP TABLE ngx_access_log; + +DROP TABLE ngx_country; + +CREATE TABLE temp_sensor_data ( + sensor_id INT, + loc STRING, + temperature DOUBLE, + ts TIMESTAMP TIME INDEX +); + +CREATE TABLE temp_alerts ( + sensor_id INT, + loc STRING, + max_temp DOUBLE, + ts TIMESTAMP TIME INDEX +); + +CREATE FLOW temp_monitoring SINK TO temp_alerts AS +SELECT + sensor_id, + loc, + max(temperature) as max_temp, +FROM + temp_sensor_data +GROUP BY + sensor_id, + loc +HAVING + max_temp > 100; + +INSERT INTO + temp_sensor_data +VALUES + (1, "room1", 50, 0); + +ADMIN FLUSH_FLOW('temp_monitoring'); + +-- This table should not exist yet +SHOW TABLES LIKE 'temp_alerts'; + +INSERT INTO + temp_sensor_data +VALUES + (1, "room1", 150, 1); + +ADMIN FLUSH_FLOW('temp_monitoring'); + +SHOW TABLES LIKE 'temp_alerts'; + +SELECT + sensor_id, + loc, + max_temp +FROM + temp_alerts; + +INSERT INTO + temp_sensor_data +VALUES + (2, "room1", 0, 2); + +ADMIN FLUSH_FLOW('temp_monitoring'); + +SELECT + sensor_id, + loc, + max_temp +FROM + temp_alerts; + +DROP FLOW temp_monitoring; + +DROP TABLE temp_sensor_data; + +DROP TABLE temp_alerts; + +CREATE TABLE ngx_access_log ( + client STRING, + stat INT, + size INT, + access_time TIMESTAMP TIME INDEX +); + +CREATE TABLE ngx_distribution ( + stat INT, + bucket_size INT, + total_logs BIGINT, + time_window TIMESTAMP TIME INDEX, + update_at TIMESTAMP, -- auto generated column by flow engine + PRIMARY KEY(stat, bucket_size) +); + +CREATE FLOW calc_ngx_distribution SINK TO ngx_distribution AS +SELECT + stat, + trunc(size, -1)::INT as bucket_size, + count(client) AS total_logs, + date_bin(INTERVAL '1 minutes', access_time) as time_window, +FROM + ngx_access_log +GROUP BY + stat, + time_window, + bucket_size; + +INSERT INTO + ngx_access_log +VALUES + ("cli1", 200, 100, 0); + +ADMIN FLUSH_FLOW('calc_ngx_distribution'); + +SELECT + stat, + bucket_size, + total_logs, + time_window +FROM + ngx_distribution; + +INSERT INTO + ngx_access_log +VALUES + ("cli1", 200, 200, 1), + ("cli1", 200, 205, 1), + ("cli1", 200, 209, 1), + ("cli1", 200, 210, 1), + ("cli2", 200, 300, 1); + +ADMIN FLUSH_FLOW('calc_ngx_distribution'); + +SELECT + stat, + bucket_size, + total_logs, + time_window +FROM + ngx_distribution; + +DROP FLOW calc_ngx_distribution; + +DROP TABLE ngx_access_log; + +DROP TABLE ngx_distribution; From aa03d3b11ccff692ad348cd113da46a7e758b527 Mon Sep 17 00:00:00 2001 From: zyy17 Date: Wed, 11 Sep 2024 15:49:23 +0800 Subject: [PATCH 002/128] docs: use docs comment prefix and bump toml2docs version (#4711) --- Makefile | 2 +- config/config.md | 172 +++++++++++++++--------------- config/datanode.example.toml | 84 ++++++++------- config/flownode.example.toml | 6 +- config/frontend.example.toml | 26 ++--- config/metasrv.example.toml | 12 +-- config/standalone.example.toml | 86 ++++++++------- src/cmd/tests/load_config_test.rs | 11 -- 8 files changed, 200 insertions(+), 199 deletions(-) diff --git a/Makefile b/Makefile index acc2a73126..697a31aed6 100644 --- a/Makefile +++ b/Makefile @@ -221,7 +221,7 @@ config-docs: ## Generate configuration documentation from toml files. docker run --rm \ -v ${PWD}:/greptimedb \ -w /greptimedb/config \ - toml2docs/toml2docs:v0.1.1 \ + toml2docs/toml2docs:v0.1.3 \ -p '##' \ -t ./config-docs-template.md \ -o ./config.md diff --git a/config/config.md b/config/config.md index a792be5de5..235b1f86fd 100644 --- a/config/config.md +++ b/config/config.md @@ -14,7 +14,7 @@ | --- | -----| ------- | ----------- | | `mode` | String | `standalone` | The running mode of the datanode. It can be `standalone` or `distributed`. | | `enable_telemetry` | Bool | `true` | Enable telemetry to collect anonymous usage data. | -| `default_timezone` | String | `None` | The default timezone of the server. | +| `default_timezone` | String | Unset | The default timezone of the server. | | `init_regions_in_background` | Bool | `false` | Initialize all regions in the background during the startup.
By default, it provides services after all regions have been initialized. | | `init_regions_parallelism` | Integer | `16` | Parallelism of initializing regions. | | `runtime` | -- | -- | The runtime options. | @@ -29,8 +29,8 @@ | `grpc.runtime_size` | Integer | `8` | The number of server worker threads. | | `grpc.tls` | -- | -- | gRPC server TLS options, see `mysql.tls` section. | | `grpc.tls.mode` | String | `disable` | TLS mode. | -| `grpc.tls.cert_path` | String | `None` | Certificate file path. | -| `grpc.tls.key_path` | String | `None` | Private key file path. | +| `grpc.tls.cert_path` | String | Unset | Certificate file path. | +| `grpc.tls.key_path` | String | Unset | Private key file path. | | `grpc.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload.
For now, gRPC tls config does not support auto reload. | | `mysql` | -- | -- | MySQL server options. | | `mysql.enable` | Bool | `true` | Whether to enable. | @@ -38,8 +38,8 @@ | `mysql.runtime_size` | Integer | `2` | The number of server worker threads. | | `mysql.tls` | -- | -- | -- | | `mysql.tls.mode` | String | `disable` | TLS mode, refer to https://www.postgresql.org/docs/current/libpq-ssl.html
- `disable` (default value)
- `prefer`
- `require`
- `verify-ca`
- `verify-full` | -| `mysql.tls.cert_path` | String | `None` | Certificate file path. | -| `mysql.tls.key_path` | String | `None` | Private key file path. | +| `mysql.tls.cert_path` | String | Unset | Certificate file path. | +| `mysql.tls.key_path` | String | Unset | Private key file path. | | `mysql.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload | | `postgres` | -- | -- | PostgresSQL server options. | | `postgres.enable` | Bool | `true` | Whether to enable | @@ -47,8 +47,8 @@ | `postgres.runtime_size` | Integer | `2` | The number of server worker threads. | | `postgres.tls` | -- | -- | PostgresSQL server TLS options, see `mysql.tls` section. | | `postgres.tls.mode` | String | `disable` | TLS mode. | -| `postgres.tls.cert_path` | String | `None` | Certificate file path. | -| `postgres.tls.key_path` | String | `None` | Private key file path. | +| `postgres.tls.cert_path` | String | Unset | Certificate file path. | +| `postgres.tls.key_path` | String | Unset | Private key file path. | | `postgres.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload | | `opentsdb` | -- | -- | OpenTSDB protocol options. | | `opentsdb.enable` | Bool | `true` | Whether to enable OpenTSDB put in HTTP API. | @@ -59,7 +59,7 @@ | `prom_store.with_metric_engine` | Bool | `true` | Whether to store the data from Prometheus remote write in metric engine. | | `wal` | -- | -- | The WAL options. | | `wal.provider` | String | `raft_engine` | The provider of the WAL.
- `raft_engine`: the wal is stored in the local file system by raft-engine.
- `kafka`: it's remote wal that data is stored in Kafka. | -| `wal.dir` | String | `None` | The directory to store the WAL files.
**It's only used when the provider is `raft_engine`**. | +| `wal.dir` | String | Unset | The directory to store the WAL files.
**It's only used when the provider is `raft_engine`**. | | `wal.file_size` | String | `256MB` | The size of the WAL segment file.
**It's only used when the provider is `raft_engine`**. | | `wal.purge_threshold` | String | `4GB` | The threshold of the WAL size to trigger a flush.
**It's only used when the provider is `raft_engine`**. | | `wal.purge_interval` | String | `10m` | The interval to trigger a flush.
**It's only used when the provider is `raft_engine`**. | @@ -91,22 +91,22 @@ | `storage` | -- | -- | The data storage options. | | `storage.data_home` | String | `/tmp/greptimedb/` | The working home directory. | | `storage.type` | String | `File` | The storage type used to store the data.
- `File`: the data is stored in the local file system.
- `S3`: the data is stored in the S3 object storage.
- `Gcs`: the data is stored in the Google Cloud Storage.
- `Azblob`: the data is stored in the Azure Blob Storage.
- `Oss`: the data is stored in the Aliyun OSS. | -| `storage.cache_path` | String | `None` | Cache configuration for object storage such as 'S3' etc.
The local file cache directory. | -| `storage.cache_capacity` | String | `None` | The local file cache capacity in bytes. | -| `storage.bucket` | String | `None` | The S3 bucket name.
**It's only used when the storage type is `S3`, `Oss` and `Gcs`**. | -| `storage.root` | String | `None` | The S3 data will be stored in the specified prefix, for example, `s3://${bucket}/${root}`.
**It's only used when the storage type is `S3`, `Oss` and `Azblob`**. | -| `storage.access_key_id` | String | `None` | The access key id of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3` and `Oss`**. | -| `storage.secret_access_key` | String | `None` | The secret access key of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3`**. | -| `storage.access_key_secret` | String | `None` | The secret access key of the aliyun account.
**It's only used when the storage type is `Oss`**. | -| `storage.account_name` | String | `None` | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.account_key` | String | `None` | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.scope` | String | `None` | The scope of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | -| `storage.credential_path` | String | `None` | The credential path of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | -| `storage.credential` | String | `None` | The credential of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | -| `storage.container` | String | `None` | The container of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.sas_token` | String | `None` | The sas token of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.endpoint` | String | `None` | The endpoint of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | -| `storage.region` | String | `None` | The region of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | +| `storage.cache_path` | String | Unset | Cache configuration for object storage such as 'S3' etc.
The local file cache directory. | +| `storage.cache_capacity` | String | Unset | The local file cache capacity in bytes. | +| `storage.bucket` | String | Unset | The S3 bucket name.
**It's only used when the storage type is `S3`, `Oss` and `Gcs`**. | +| `storage.root` | String | Unset | The S3 data will be stored in the specified prefix, for example, `s3://${bucket}/${root}`.
**It's only used when the storage type is `S3`, `Oss` and `Azblob`**. | +| `storage.access_key_id` | String | Unset | The access key id of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3` and `Oss`**. | +| `storage.secret_access_key` | String | Unset | The secret access key of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3`**. | +| `storage.access_key_secret` | String | Unset | The secret access key of the aliyun account.
**It's only used when the storage type is `Oss`**. | +| `storage.account_name` | String | Unset | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.account_key` | String | Unset | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.scope` | String | Unset | The scope of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | +| `storage.credential_path` | String | Unset | The credential path of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | +| `storage.credential` | String | Unset | The credential of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | +| `storage.container` | String | Unset | The container of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.sas_token` | String | Unset | The sas token of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.endpoint` | String | Unset | The endpoint of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | +| `storage.region` | String | Unset | The region of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | | `[[region_engine]]` | -- | -- | The region engine options. You can configure multiple region engines. | | `region_engine.mito` | -- | -- | The Mito engine options. | | `region_engine.mito.num_workers` | Integer | `8` | Number of region workers. | @@ -116,16 +116,16 @@ | `region_engine.mito.compress_manifest` | Bool | `false` | Whether to compress manifest and checkpoint file by gzip (default false). | | `region_engine.mito.max_background_jobs` | Integer | `4` | Max number of running background jobs | | `region_engine.mito.auto_flush_interval` | String | `1h` | Interval to auto flush a region if it has not flushed yet. | -| `region_engine.mito.global_write_buffer_size` | String | `1GB` | Global write buffer size for all regions. If not set, it's default to 1/8 of OS memory with a max limitation of 1GB. | -| `region_engine.mito.global_write_buffer_reject_size` | String | `2GB` | Global write buffer size threshold to reject write requests. If not set, it's default to 2 times of `global_write_buffer_size` | -| `region_engine.mito.sst_meta_cache_size` | String | `128MB` | Cache size for SST metadata. Setting it to 0 to disable the cache.
If not set, it's default to 1/32 of OS memory with a max limitation of 128MB. | -| `region_engine.mito.vector_cache_size` | String | `512MB` | Cache size for vectors and arrow arrays. Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | -| `region_engine.mito.page_cache_size` | String | `512MB` | Cache size for pages of SST row groups. Setting it to 0 to disable the cache.
If not set, it's default to 1/8 of OS memory. | -| `region_engine.mito.selector_result_cache_size` | String | `512MB` | Cache size for time series selector (e.g. `last_value()`). Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | +| `region_engine.mito.global_write_buffer_size` | String | Auto | Global write buffer size for all regions. If not set, it's default to 1/8 of OS memory with a max limitation of 1GB. | +| `region_engine.mito.global_write_buffer_reject_size` | String | Auto | Global write buffer size threshold to reject write requests. If not set, it's default to 2 times of `global_write_buffer_size`. | +| `region_engine.mito.sst_meta_cache_size` | String | Auto | Cache size for SST metadata. Setting it to 0 to disable the cache.
If not set, it's default to 1/32 of OS memory with a max limitation of 128MB. | +| `region_engine.mito.vector_cache_size` | String | Auto | Cache size for vectors and arrow arrays. Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | +| `region_engine.mito.page_cache_size` | String | Auto | Cache size for pages of SST row groups. Setting it to 0 to disable the cache.
If not set, it's default to 1/8 of OS memory. | +| `region_engine.mito.selector_result_cache_size` | String | Auto | Cache size for time series selector (e.g. `last_value()`). Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | | `region_engine.mito.enable_experimental_write_cache` | Bool | `false` | Whether to enable the experimental write cache. | | `region_engine.mito.experimental_write_cache_path` | String | `""` | File system path for write cache, defaults to `{data_home}/write_cache`. | | `region_engine.mito.experimental_write_cache_size` | String | `512MB` | Capacity for write cache. | -| `region_engine.mito.experimental_write_cache_ttl` | String | `None` | TTL for write cache. | +| `region_engine.mito.experimental_write_cache_ttl` | String | Unset | TTL for write cache. | | `region_engine.mito.sst_write_buffer_size` | String | `8MB` | Buffer size for SST writing. | | `region_engine.mito.scan_parallelism` | Integer | `0` | Parallelism to scan a region (default: 1/4 of cpu cores).
- `0`: using the default value (1/4 of cpu cores).
- `1`: scan in current thread.
- `n`: scan in parallelism n. | | `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. | @@ -155,7 +155,7 @@ | `region_engine.file` | -- | -- | Enable the file engine. | | `logging` | -- | -- | The logging options. | | `logging.dir` | String | `/tmp/greptimedb/logs` | The directory to store the log files. If set to empty, logs will not be written to files. | -| `logging.level` | String | `None` | The log level. Can be `info`/`debug`/`warn`/`error`. | +| `logging.level` | String | Unset | The log level. Can be `info`/`debug`/`warn`/`error`. | | `logging.enable_otlp_tracing` | Bool | `false` | Enable OTLP tracing. | | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | @@ -165,13 +165,13 @@ | `export_metrics` | -- | -- | The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API.
This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. | | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | -| `export_metrics.self_import` | -- | -- | For `standalone` mode, `self_import` is recommend to collect metrics generated by itself
You must create the database before enabling it. | -| `export_metrics.self_import.db` | String | `None` | -- | +| `export_metrics.self_import` | -- | -- | For `standalone` mode, `self_import` is recommended to collect metrics generated by itself
You must create the database before enabling it. | +| `export_metrics.self_import.db` | String | Unset | -- | | `export_metrics.remote_write` | -- | -- | -- | | `export_metrics.remote_write.url` | String | `""` | The url the metrics send to. The url example can be: `http://127.0.0.1:4000/v1/prometheus/write?db=greptime_metrics`. | | `export_metrics.remote_write.headers` | InlineTable | -- | HTTP headers of Prometheus remote-write carry. | | `tracing` | -- | -- | The tracing options. Only effect when compiled with `tokio-console` feature. | -| `tracing.tokio_console_addr` | String | `None` | The tokio console address. | +| `tracing.tokio_console_addr` | String | Unset | The tokio console address. | ## Distributed Mode @@ -180,7 +180,7 @@ | Key | Type | Default | Descriptions | | --- | -----| ------- | ----------- | -| `default_timezone` | String | `None` | The default timezone of the server. | +| `default_timezone` | String | Unset | The default timezone of the server. | | `runtime` | -- | -- | The runtime options. | | `runtime.global_rt_size` | Integer | `8` | The number of threads to execute the runtime for global read operations. | | `runtime.compact_rt_size` | Integer | `4` | The number of threads to execute the runtime for global write operations. | @@ -197,8 +197,8 @@ | `grpc.runtime_size` | Integer | `8` | The number of server worker threads. | | `grpc.tls` | -- | -- | gRPC server TLS options, see `mysql.tls` section. | | `grpc.tls.mode` | String | `disable` | TLS mode. | -| `grpc.tls.cert_path` | String | `None` | Certificate file path. | -| `grpc.tls.key_path` | String | `None` | Private key file path. | +| `grpc.tls.cert_path` | String | Unset | Certificate file path. | +| `grpc.tls.key_path` | String | Unset | Private key file path. | | `grpc.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload.
For now, gRPC tls config does not support auto reload. | | `mysql` | -- | -- | MySQL server options. | | `mysql.enable` | Bool | `true` | Whether to enable. | @@ -206,8 +206,8 @@ | `mysql.runtime_size` | Integer | `2` | The number of server worker threads. | | `mysql.tls` | -- | -- | -- | | `mysql.tls.mode` | String | `disable` | TLS mode, refer to https://www.postgresql.org/docs/current/libpq-ssl.html
- `disable` (default value)
- `prefer`
- `require`
- `verify-ca`
- `verify-full` | -| `mysql.tls.cert_path` | String | `None` | Certificate file path. | -| `mysql.tls.key_path` | String | `None` | Private key file path. | +| `mysql.tls.cert_path` | String | Unset | Certificate file path. | +| `mysql.tls.key_path` | String | Unset | Private key file path. | | `mysql.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload | | `postgres` | -- | -- | PostgresSQL server options. | | `postgres.enable` | Bool | `true` | Whether to enable | @@ -215,8 +215,8 @@ | `postgres.runtime_size` | Integer | `2` | The number of server worker threads. | | `postgres.tls` | -- | -- | PostgresSQL server TLS options, see `mysql.tls` section. | | `postgres.tls.mode` | String | `disable` | TLS mode. | -| `postgres.tls.cert_path` | String | `None` | Certificate file path. | -| `postgres.tls.key_path` | String | `None` | Private key file path. | +| `postgres.tls.cert_path` | String | Unset | Certificate file path. | +| `postgres.tls.key_path` | String | Unset | Private key file path. | | `postgres.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload | | `opentsdb` | -- | -- | OpenTSDB protocol options. | | `opentsdb.enable` | Bool | `true` | Whether to enable OpenTSDB put in HTTP API. | @@ -241,7 +241,7 @@ | `datanode.client.tcp_nodelay` | Bool | `true` | -- | | `logging` | -- | -- | The logging options. | | `logging.dir` | String | `/tmp/greptimedb/logs` | The directory to store the log files. If set to empty, logs will not be written to files. | -| `logging.level` | String | `None` | The log level. Can be `info`/`debug`/`warn`/`error`. | +| `logging.level` | String | Unset | The log level. Can be `info`/`debug`/`warn`/`error`. | | `logging.enable_otlp_tracing` | Bool | `false` | Enable OTLP tracing. | | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | @@ -252,12 +252,12 @@ | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | | `export_metrics.self_import` | -- | -- | For `standalone` mode, `self_import` is recommend to collect metrics generated by itself
You must create the database before enabling it. | -| `export_metrics.self_import.db` | String | `None` | -- | +| `export_metrics.self_import.db` | String | Unset | -- | | `export_metrics.remote_write` | -- | -- | -- | | `export_metrics.remote_write.url` | String | `""` | The url the metrics send to. The url example can be: `http://127.0.0.1:4000/v1/prometheus/write?db=greptime_metrics`. | | `export_metrics.remote_write.headers` | InlineTable | -- | HTTP headers of Prometheus remote-write carry. | | `tracing` | -- | -- | The tracing options. Only effect when compiled with `tokio-console` feature. | -| `tracing.tokio_console_addr` | String | `None` | The tokio console address. | +| `tracing.tokio_console_addr` | String | Unset | The tokio console address. | ### Metasrv @@ -306,7 +306,7 @@ | `wal.backoff_deadline` | String | `5mins` | Stop reconnecting if the total wait time reaches the deadline. If this config is missing, the reconnecting won't terminate. | | `logging` | -- | -- | The logging options. | | `logging.dir` | String | `/tmp/greptimedb/logs` | The directory to store the log files. If set to empty, logs will not be written to files. | -| `logging.level` | String | `None` | The log level. Can be `info`/`debug`/`warn`/`error`. | +| `logging.level` | String | Unset | The log level. Can be `info`/`debug`/`warn`/`error`. | | `logging.enable_otlp_tracing` | Bool | `false` | Enable OTLP tracing. | | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | @@ -317,12 +317,12 @@ | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | | `export_metrics.self_import` | -- | -- | For `standalone` mode, `self_import` is recommend to collect metrics generated by itself
You must create the database before enabling it. | -| `export_metrics.self_import.db` | String | `None` | -- | +| `export_metrics.self_import.db` | String | Unset | -- | | `export_metrics.remote_write` | -- | -- | -- | | `export_metrics.remote_write.url` | String | `""` | The url the metrics send to. The url example can be: `http://127.0.0.1:4000/v1/prometheus/write?db=greptime_metrics`. | | `export_metrics.remote_write.headers` | InlineTable | -- | HTTP headers of Prometheus remote-write carry. | | `tracing` | -- | -- | The tracing options. Only effect when compiled with `tokio-console` feature. | -| `tracing.tokio_console_addr` | String | `None` | The tokio console address. | +| `tracing.tokio_console_addr` | String | Unset | The tokio console address. | ### Datanode @@ -330,16 +330,16 @@ | Key | Type | Default | Descriptions | | --- | -----| ------- | ----------- | | `mode` | String | `standalone` | The running mode of the datanode. It can be `standalone` or `distributed`. | -| `node_id` | Integer | `None` | The datanode identifier and should be unique in the cluster. | +| `node_id` | Integer | Unset | The datanode identifier and should be unique in the cluster. | | `require_lease_before_startup` | Bool | `false` | Start services after regions have obtained leases.
It will block the datanode start if it can't receive leases in the heartbeat from metasrv. | | `init_regions_in_background` | Bool | `false` | Initialize all regions in the background during the startup.
By default, it provides services after all regions have been initialized. | | `enable_telemetry` | Bool | `true` | Enable telemetry to collect anonymous usage data. | | `init_regions_parallelism` | Integer | `16` | Parallelism of initializing regions. | -| `rpc_addr` | String | `None` | Deprecated, use `grpc.addr` instead. | -| `rpc_hostname` | String | `None` | Deprecated, use `grpc.hostname` instead. | -| `rpc_runtime_size` | Integer | `None` | Deprecated, use `grpc.runtime_size` instead. | -| `rpc_max_recv_message_size` | String | `None` | Deprecated, use `grpc.rpc_max_recv_message_size` instead. | -| `rpc_max_send_message_size` | String | `None` | Deprecated, use `grpc.rpc_max_send_message_size` instead. | +| `rpc_addr` | String | Unset | Deprecated, use `grpc.addr` instead. | +| `rpc_hostname` | String | Unset | Deprecated, use `grpc.hostname` instead. | +| `rpc_runtime_size` | Integer | Unset | Deprecated, use `grpc.runtime_size` instead. | +| `rpc_max_recv_message_size` | String | Unset | Deprecated, use `grpc.rpc_max_recv_message_size` instead. | +| `rpc_max_send_message_size` | String | Unset | Deprecated, use `grpc.rpc_max_send_message_size` instead. | | `http` | -- | -- | The HTTP server options. | | `http.addr` | String | `127.0.0.1:4000` | The address to bind the HTTP server. | | `http.timeout` | String | `30s` | HTTP request timeout. Set to 0 to disable timeout. | @@ -352,8 +352,8 @@ | `grpc.max_send_message_size` | String | `512MB` | The maximum send message size for gRPC server. | | `grpc.tls` | -- | -- | gRPC server TLS options, see `mysql.tls` section. | | `grpc.tls.mode` | String | `disable` | TLS mode. | -| `grpc.tls.cert_path` | String | `None` | Certificate file path. | -| `grpc.tls.key_path` | String | `None` | Private key file path. | +| `grpc.tls.cert_path` | String | Unset | Certificate file path. | +| `grpc.tls.key_path` | String | Unset | Private key file path. | | `grpc.tls.watch` | Bool | `false` | Watch for Certificate and key file change and auto reload.
For now, gRPC tls config does not support auto reload. | | `runtime` | -- | -- | The runtime options. | | `runtime.global_rt_size` | Integer | `8` | The number of threads to execute the runtime for global read operations. | @@ -373,7 +373,7 @@ | `meta_client.metadata_cache_tti` | String | `5m` | -- | | `wal` | -- | -- | The WAL options. | | `wal.provider` | String | `raft_engine` | The provider of the WAL.
- `raft_engine`: the wal is stored in the local file system by raft-engine.
- `kafka`: it's remote wal that data is stored in Kafka. | -| `wal.dir` | String | `None` | The directory to store the WAL files.
**It's only used when the provider is `raft_engine`**. | +| `wal.dir` | String | Unset | The directory to store the WAL files.
**It's only used when the provider is `raft_engine`**. | | `wal.file_size` | String | `256MB` | The size of the WAL segment file.
**It's only used when the provider is `raft_engine`**. | | `wal.purge_threshold` | String | `4GB` | The threshold of the WAL size to trigger a flush.
**It's only used when the provider is `raft_engine`**. | | `wal.purge_interval` | String | `10m` | The interval to trigger a flush.
**It's only used when the provider is `raft_engine`**. | @@ -395,22 +395,22 @@ | `storage` | -- | -- | The data storage options. | | `storage.data_home` | String | `/tmp/greptimedb/` | The working home directory. | | `storage.type` | String | `File` | The storage type used to store the data.
- `File`: the data is stored in the local file system.
- `S3`: the data is stored in the S3 object storage.
- `Gcs`: the data is stored in the Google Cloud Storage.
- `Azblob`: the data is stored in the Azure Blob Storage.
- `Oss`: the data is stored in the Aliyun OSS. | -| `storage.cache_path` | String | `None` | Cache configuration for object storage such as 'S3' etc.
The local file cache directory. | -| `storage.cache_capacity` | String | `None` | The local file cache capacity in bytes. | -| `storage.bucket` | String | `None` | The S3 bucket name.
**It's only used when the storage type is `S3`, `Oss` and `Gcs`**. | -| `storage.root` | String | `None` | The S3 data will be stored in the specified prefix, for example, `s3://${bucket}/${root}`.
**It's only used when the storage type is `S3`, `Oss` and `Azblob`**. | -| `storage.access_key_id` | String | `None` | The access key id of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3` and `Oss`**. | -| `storage.secret_access_key` | String | `None` | The secret access key of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3`**. | -| `storage.access_key_secret` | String | `None` | The secret access key of the aliyun account.
**It's only used when the storage type is `Oss`**. | -| `storage.account_name` | String | `None` | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.account_key` | String | `None` | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.scope` | String | `None` | The scope of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | -| `storage.credential_path` | String | `None` | The credential path of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | -| `storage.credential` | String | `None` | The credential of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | -| `storage.container` | String | `None` | The container of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.sas_token` | String | `None` | The sas token of the azure account.
**It's only used when the storage type is `Azblob`**. | -| `storage.endpoint` | String | `None` | The endpoint of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | -| `storage.region` | String | `None` | The region of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | +| `storage.cache_path` | String | Unset | Cache configuration for object storage such as 'S3' etc.
The local file cache directory. | +| `storage.cache_capacity` | String | Unset | The local file cache capacity in bytes. | +| `storage.bucket` | String | Unset | The S3 bucket name.
**It's only used when the storage type is `S3`, `Oss` and `Gcs`**. | +| `storage.root` | String | Unset | The S3 data will be stored in the specified prefix, for example, `s3://${bucket}/${root}`.
**It's only used when the storage type is `S3`, `Oss` and `Azblob`**. | +| `storage.access_key_id` | String | Unset | The access key id of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3` and `Oss`**. | +| `storage.secret_access_key` | String | Unset | The secret access key of the aws account.
It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key.
**It's only used when the storage type is `S3`**. | +| `storage.access_key_secret` | String | Unset | The secret access key of the aliyun account.
**It's only used when the storage type is `Oss`**. | +| `storage.account_name` | String | Unset | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.account_key` | String | Unset | The account key of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.scope` | String | Unset | The scope of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | +| `storage.credential_path` | String | Unset | The credential path of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | +| `storage.credential` | String | Unset | The credential of the google cloud storage.
**It's only used when the storage type is `Gcs`**. | +| `storage.container` | String | Unset | The container of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.sas_token` | String | Unset | The sas token of the azure account.
**It's only used when the storage type is `Azblob`**. | +| `storage.endpoint` | String | Unset | The endpoint of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | +| `storage.region` | String | Unset | The region of the S3 service.
**It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. | | `[[region_engine]]` | -- | -- | The region engine options. You can configure multiple region engines. | | `region_engine.mito` | -- | -- | The Mito engine options. | | `region_engine.mito.num_workers` | Integer | `8` | Number of region workers. | @@ -420,16 +420,16 @@ | `region_engine.mito.compress_manifest` | Bool | `false` | Whether to compress manifest and checkpoint file by gzip (default false). | | `region_engine.mito.max_background_jobs` | Integer | `4` | Max number of running background jobs | | `region_engine.mito.auto_flush_interval` | String | `1h` | Interval to auto flush a region if it has not flushed yet. | -| `region_engine.mito.global_write_buffer_size` | String | `1GB` | Global write buffer size for all regions. If not set, it's default to 1/8 of OS memory with a max limitation of 1GB. | -| `region_engine.mito.global_write_buffer_reject_size` | String | `2GB` | Global write buffer size threshold to reject write requests. If not set, it's default to 2 times of `global_write_buffer_size` | -| `region_engine.mito.sst_meta_cache_size` | String | `128MB` | Cache size for SST metadata. Setting it to 0 to disable the cache.
If not set, it's default to 1/32 of OS memory with a max limitation of 128MB. | -| `region_engine.mito.vector_cache_size` | String | `512MB` | Cache size for vectors and arrow arrays. Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | -| `region_engine.mito.page_cache_size` | String | `512MB` | Cache size for pages of SST row groups. Setting it to 0 to disable the cache.
If not set, it's default to 1/8 of OS memory. | -| `region_engine.mito.selector_result_cache_size` | String | `512MB` | Cache size for time series selector (e.g. `last_value()`). Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | +| `region_engine.mito.global_write_buffer_size` | String | Auto | Global write buffer size for all regions. If not set, it's default to 1/8 of OS memory with a max limitation of 1GB. | +| `region_engine.mito.global_write_buffer_reject_size` | String | Auto | Global write buffer size threshold to reject write requests. If not set, it's default to 2 times of `global_write_buffer_size` | +| `region_engine.mito.sst_meta_cache_size` | String | Auto | Cache size for SST metadata. Setting it to 0 to disable the cache.
If not set, it's default to 1/32 of OS memory with a max limitation of 128MB. | +| `region_engine.mito.vector_cache_size` | String | Auto | Cache size for vectors and arrow arrays. Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | +| `region_engine.mito.page_cache_size` | String | Auto | Cache size for pages of SST row groups. Setting it to 0 to disable the cache.
If not set, it's default to 1/8 of OS memory. | +| `region_engine.mito.selector_result_cache_size` | String | Auto | Cache size for time series selector (e.g. `last_value()`). Setting it to 0 to disable the cache.
If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. | | `region_engine.mito.enable_experimental_write_cache` | Bool | `false` | Whether to enable the experimental write cache. | | `region_engine.mito.experimental_write_cache_path` | String | `""` | File system path for write cache, defaults to `{data_home}/write_cache`. | | `region_engine.mito.experimental_write_cache_size` | String | `512MB` | Capacity for write cache. | -| `region_engine.mito.experimental_write_cache_ttl` | String | `None` | TTL for write cache. | +| `region_engine.mito.experimental_write_cache_ttl` | String | Unset | TTL for write cache. | | `region_engine.mito.sst_write_buffer_size` | String | `8MB` | Buffer size for SST writing. | | `region_engine.mito.scan_parallelism` | Integer | `0` | Parallelism to scan a region (default: 1/4 of cpu cores).
- `0`: using the default value (1/4 of cpu cores).
- `1`: scan in current thread.
- `n`: scan in parallelism n. | | `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. | @@ -457,7 +457,7 @@ | `region_engine.file` | -- | -- | Enable the file engine. | | `logging` | -- | -- | The logging options. | | `logging.dir` | String | `/tmp/greptimedb/logs` | The directory to store the log files. If set to empty, logs will not be written to files. | -| `logging.level` | String | `None` | The log level. Can be `info`/`debug`/`warn`/`error`. | +| `logging.level` | String | Unset | The log level. Can be `info`/`debug`/`warn`/`error`. | | `logging.enable_otlp_tracing` | Bool | `false` | Enable OTLP tracing. | | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | @@ -468,12 +468,12 @@ | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | | `export_metrics.self_import` | -- | -- | For `standalone` mode, `self_import` is recommend to collect metrics generated by itself
You must create the database before enabling it. | -| `export_metrics.self_import.db` | String | `None` | -- | +| `export_metrics.self_import.db` | String | Unset | -- | | `export_metrics.remote_write` | -- | -- | -- | | `export_metrics.remote_write.url` | String | `""` | The url the metrics send to. The url example can be: `http://127.0.0.1:4000/v1/prometheus/write?db=greptime_metrics`. | | `export_metrics.remote_write.headers` | InlineTable | -- | HTTP headers of Prometheus remote-write carry. | | `tracing` | -- | -- | The tracing options. Only effect when compiled with `tokio-console` feature. | -| `tracing.tokio_console_addr` | String | `None` | The tokio console address. | +| `tracing.tokio_console_addr` | String | Unset | The tokio console address. | ### Flownode @@ -481,7 +481,7 @@ | Key | Type | Default | Descriptions | | --- | -----| ------- | ----------- | | `mode` | String | `distributed` | The running mode of the flownode. It can be `standalone` or `distributed`. | -| `node_id` | Integer | `None` | The flownode identifier and should be unique in the cluster. | +| `node_id` | Integer | Unset | The flownode identifier and should be unique in the cluster. | | `grpc` | -- | -- | The gRPC server options. | | `grpc.addr` | String | `127.0.0.1:6800` | The address to bind the gRPC server. | | `grpc.hostname` | String | `127.0.0.1` | The hostname advertised to the metasrv,
and used for connections from outside the host | @@ -503,7 +503,7 @@ | `heartbeat.retry_interval` | String | `3s` | Interval for retrying to send heartbeat messages to the metasrv. | | `logging` | -- | -- | The logging options. | | `logging.dir` | String | `/tmp/greptimedb/logs` | The directory to store the log files. If set to empty, logs will not be written to files. | -| `logging.level` | String | `None` | The log level. Can be `info`/`debug`/`warn`/`error`. | +| `logging.level` | String | Unset | The log level. Can be `info`/`debug`/`warn`/`error`. | | `logging.enable_otlp_tracing` | Bool | `false` | Enable OTLP tracing. | | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | @@ -511,4 +511,4 @@ | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | | `tracing` | -- | -- | The tracing options. Only effect when compiled with `tokio-console` feature. | -| `tracing.tokio_console_addr` | String | `None` | The tokio console address. | +| `tracing.tokio_console_addr` | String | Unset | The tokio console address. | diff --git a/config/datanode.example.toml b/config/datanode.example.toml index 14fbf914e7..4388c4420f 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -2,7 +2,7 @@ mode = "standalone" ## The datanode identifier and should be unique in the cluster. -## +toml2docs:none-default +## @toml2docs:none-default node_id = 42 ## Start services after regions have obtained leases. @@ -20,23 +20,23 @@ enable_telemetry = true init_regions_parallelism = 16 ## Deprecated, use `grpc.addr` instead. -## +toml2docs:none-default +## @toml2docs:none-default rpc_addr = "127.0.0.1:3001" ## Deprecated, use `grpc.hostname` instead. -## +toml2docs:none-default +## @toml2docs:none-default rpc_hostname = "127.0.0.1" ## Deprecated, use `grpc.runtime_size` instead. -## +toml2docs:none-default +## @toml2docs:none-default rpc_runtime_size = 8 ## Deprecated, use `grpc.rpc_max_recv_message_size` instead. -## +toml2docs:none-default +## @toml2docs:none-default rpc_max_recv_message_size = "512MB" ## Deprecated, use `grpc.rpc_max_send_message_size` instead. -## +toml2docs:none-default +## @toml2docs:none-default rpc_max_send_message_size = "512MB" @@ -71,11 +71,11 @@ max_send_message_size = "512MB" mode = "disable" ## Certificate file path. -## +toml2docs:none-default +## @toml2docs:none-default cert_path = "" ## Private key file path. -## +toml2docs:none-default +## @toml2docs:none-default key_path = "" ## Watch for Certificate and key file change and auto reload. @@ -83,11 +83,11 @@ key_path = "" watch = false ## The runtime options. -[runtime] +#+ [runtime] ## The number of threads to execute the runtime for global read operations. -global_rt_size = 8 +#+ global_rt_size = 8 ## The number of threads to execute the runtime for global write operations. -compact_rt_size = 4 +#+ compact_rt_size = 4 ## The heartbeat options. [heartbeat] @@ -135,7 +135,7 @@ provider = "raft_engine" ## The directory to store the WAL files. ## **It's only used when the provider is `raft_engine`**. -## +toml2docs:none-default +## @toml2docs:none-default dir = "/tmp/greptimedb/wal" ## The size of the WAL segment file. @@ -282,83 +282,83 @@ type = "File" ## Cache configuration for object storage such as 'S3' etc. ## The local file cache directory. -## +toml2docs:none-default +## @toml2docs:none-default cache_path = "/path/local_cache" ## The local file cache capacity in bytes. -## +toml2docs:none-default +## @toml2docs:none-default cache_capacity = "256MB" ## The S3 bucket name. ## **It's only used when the storage type is `S3`, `Oss` and `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default bucket = "greptimedb" ## The S3 data will be stored in the specified prefix, for example, `s3://${bucket}/${root}`. ## **It's only used when the storage type is `S3`, `Oss` and `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default root = "greptimedb" ## The access key id of the aws account. ## It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key. ## **It's only used when the storage type is `S3` and `Oss`**. -## +toml2docs:none-default +## @toml2docs:none-default access_key_id = "test" ## The secret access key of the aws account. ## It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key. ## **It's only used when the storage type is `S3`**. -## +toml2docs:none-default +## @toml2docs:none-default secret_access_key = "test" ## The secret access key of the aliyun account. ## **It's only used when the storage type is `Oss`**. -## +toml2docs:none-default +## @toml2docs:none-default access_key_secret = "test" ## The account key of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default account_name = "test" ## The account key of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default account_key = "test" ## The scope of the google cloud storage. ## **It's only used when the storage type is `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default scope = "test" ## The credential path of the google cloud storage. ## **It's only used when the storage type is `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default credential_path = "test" ## The credential of the google cloud storage. ## **It's only used when the storage type is `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default credential = "base64-credential" ## The container of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default container = "greptimedb" ## The sas token of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default sas_token = "" ## The endpoint of the S3 service. ## **It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default endpoint = "https://s3.amazonaws.com" ## The region of the S3 service. ## **It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default region = "us-west-2" # Custom storage options @@ -388,7 +388,7 @@ region = "us-west-2" [region_engine.mito] ## Number of region workers. -num_workers = 8 +#+ num_workers = 8 ## Request channel size of each worker. worker_channel_size = 128 @@ -409,26 +409,32 @@ max_background_jobs = 4 auto_flush_interval = "1h" ## Global write buffer size for all regions. If not set, it's default to 1/8 of OS memory with a max limitation of 1GB. -global_write_buffer_size = "1GB" +## @toml2docs:none-default="Auto" +#+ global_write_buffer_size = "1GB" ## Global write buffer size threshold to reject write requests. If not set, it's default to 2 times of `global_write_buffer_size` -global_write_buffer_reject_size = "2GB" +## @toml2docs:none-default="Auto" +#+ global_write_buffer_reject_size = "2GB" ## Cache size for SST metadata. Setting it to 0 to disable the cache. ## If not set, it's default to 1/32 of OS memory with a max limitation of 128MB. -sst_meta_cache_size = "128MB" +## @toml2docs:none-default="Auto" +#+ sst_meta_cache_size = "128MB" ## Cache size for vectors and arrow arrays. Setting it to 0 to disable the cache. ## If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. -vector_cache_size = "512MB" +## @toml2docs:none-default="Auto" +#+ vector_cache_size = "512MB" ## Cache size for pages of SST row groups. Setting it to 0 to disable the cache. ## If not set, it's default to 1/8 of OS memory. -page_cache_size = "512MB" +## @toml2docs:none-default="Auto" +#+ page_cache_size = "512MB" ## Cache size for time series selector (e.g. `last_value()`). Setting it to 0 to disable the cache. ## If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. -selector_result_cache_size = "512MB" +## @toml2docs:none-default="Auto" +#+ selector_result_cache_size = "512MB" ## Whether to enable the experimental write cache. enable_experimental_write_cache = false @@ -440,7 +446,7 @@ experimental_write_cache_path = "" experimental_write_cache_size = "512MB" ## TTL for write cache. -## +toml2docs:none-default +## @toml2docs:none-default experimental_write_cache_ttl = "8h" ## Buffer size for SST writing. @@ -556,7 +562,7 @@ fork_dictionary_bytes = "1GiB" dir = "/tmp/greptimedb/logs" ## The log level. Can be `info`/`debug`/`warn`/`error`. -## +toml2docs:none-default +## @toml2docs:none-default level = "info" ## Enable OTLP tracing. @@ -590,7 +596,7 @@ write_interval = "30s" ## For `standalone` mode, `self_import` is recommend to collect metrics generated by itself ## You must create the database before enabling it. [export_metrics.self_import] -## +toml2docs:none-default +## @toml2docs:none-default db = "greptime_metrics" [export_metrics.remote_write] @@ -603,5 +609,5 @@ headers = { } ## The tracing options. Only effect when compiled with `tokio-console` feature. [tracing] ## The tokio console address. -## +toml2docs:none-default +## @toml2docs:none-default tokio_console_addr = "127.0.0.1" diff --git a/config/flownode.example.toml b/config/flownode.example.toml index d5640062f7..a3a414fb98 100644 --- a/config/flownode.example.toml +++ b/config/flownode.example.toml @@ -2,7 +2,7 @@ mode = "distributed" ## The flownode identifier and should be unique in the cluster. -## +toml2docs:none-default +## @toml2docs:none-default node_id = 14 ## The gRPC server options. @@ -63,7 +63,7 @@ retry_interval = "3s" dir = "/tmp/greptimedb/logs" ## The log level. Can be `info`/`debug`/`warn`/`error`. -## +toml2docs:none-default +## @toml2docs:none-default level = "info" ## Enable OTLP tracing. @@ -87,6 +87,6 @@ default_ratio = 1.0 ## The tracing options. Only effect when compiled with `tokio-console` feature. [tracing] ## The tokio console address. -## +toml2docs:none-default +## @toml2docs:none-default tokio_console_addr = "127.0.0.1" diff --git a/config/frontend.example.toml b/config/frontend.example.toml index e5a7f5af89..eae001a2db 100644 --- a/config/frontend.example.toml +++ b/config/frontend.example.toml @@ -1,13 +1,13 @@ ## The default timezone of the server. -## +toml2docs:none-default +## @toml2docs:none-default default_timezone = "UTC" ## The runtime options. -[runtime] +#+ [runtime] ## The number of threads to execute the runtime for global read operations. -global_rt_size = 8 +#+ global_rt_size = 8 ## The number of threads to execute the runtime for global write operations. -compact_rt_size = 4 +#+ compact_rt_size = 4 ## The heartbeat options. [heartbeat] @@ -44,11 +44,11 @@ runtime_size = 8 mode = "disable" ## Certificate file path. -## +toml2docs:none-default +## @toml2docs:none-default cert_path = "" ## Private key file path. -## +toml2docs:none-default +## @toml2docs:none-default key_path = "" ## Watch for Certificate and key file change and auto reload. @@ -76,11 +76,11 @@ runtime_size = 2 mode = "disable" ## Certificate file path. -## +toml2docs:none-default +## @toml2docs:none-default cert_path = "" ## Private key file path. -## +toml2docs:none-default +## @toml2docs:none-default key_path = "" ## Watch for Certificate and key file change and auto reload @@ -101,11 +101,11 @@ runtime_size = 2 mode = "disable" ## Certificate file path. -## +toml2docs:none-default +## @toml2docs:none-default cert_path = "" ## Private key file path. -## +toml2docs:none-default +## @toml2docs:none-default key_path = "" ## Watch for Certificate and key file change and auto reload @@ -170,7 +170,7 @@ tcp_nodelay = true dir = "/tmp/greptimedb/logs" ## The log level. Can be `info`/`debug`/`warn`/`error`. -## +toml2docs:none-default +## @toml2docs:none-default level = "info" ## Enable OTLP tracing. @@ -204,7 +204,7 @@ write_interval = "30s" ## For `standalone` mode, `self_import` is recommend to collect metrics generated by itself ## You must create the database before enabling it. [export_metrics.self_import] -## +toml2docs:none-default +## @toml2docs:none-default db = "greptime_metrics" [export_metrics.remote_write] @@ -217,5 +217,5 @@ headers = { } ## The tracing options. Only effect when compiled with `tokio-console` feature. [tracing] ## The tokio console address. -## +toml2docs:none-default +## @toml2docs:none-default tokio_console_addr = "127.0.0.1" diff --git a/config/metasrv.example.toml b/config/metasrv.example.toml index dc5f091166..e95a9fa7f2 100644 --- a/config/metasrv.example.toml +++ b/config/metasrv.example.toml @@ -36,11 +36,11 @@ enable_region_failover = false backend = "EtcdStore" ## The runtime options. -[runtime] +#+ [runtime] ## The number of threads to execute the runtime for global read operations. -global_rt_size = 8 +#+ global_rt_size = 8 ## The number of threads to execute the runtime for global write operations. -compact_rt_size = 4 +#+ compact_rt_size = 4 ## Procedure storage options. [procedure] @@ -157,7 +157,7 @@ backoff_deadline = "5mins" dir = "/tmp/greptimedb/logs" ## The log level. Can be `info`/`debug`/`warn`/`error`. -## +toml2docs:none-default +## @toml2docs:none-default level = "info" ## Enable OTLP tracing. @@ -191,7 +191,7 @@ write_interval = "30s" ## For `standalone` mode, `self_import` is recommend to collect metrics generated by itself ## You must create the database before enabling it. [export_metrics.self_import] -## +toml2docs:none-default +## @toml2docs:none-default db = "greptime_metrics" [export_metrics.remote_write] @@ -204,5 +204,5 @@ headers = { } ## The tracing options. Only effect when compiled with `tokio-console` feature. [tracing] ## The tokio console address. -## +toml2docs:none-default +## @toml2docs:none-default tokio_console_addr = "127.0.0.1" diff --git a/config/standalone.example.toml b/config/standalone.example.toml index f7c7b2af29..040e1e62c2 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -5,7 +5,7 @@ mode = "standalone" enable_telemetry = true ## The default timezone of the server. -## +toml2docs:none-default +## @toml2docs:none-default default_timezone = "UTC" ## Initialize all regions in the background during the startup. @@ -16,11 +16,11 @@ init_regions_in_background = false init_regions_parallelism = 16 ## The runtime options. -[runtime] +#+ [runtime] ## The number of threads to execute the runtime for global read operations. -global_rt_size = 8 +#+ global_rt_size = 8 ## The number of threads to execute the runtime for global write operations. -compact_rt_size = 4 +#+ compact_rt_size = 4 ## The HTTP server options. [http] @@ -46,11 +46,11 @@ runtime_size = 8 mode = "disable" ## Certificate file path. -## +toml2docs:none-default +## @toml2docs:none-default cert_path = "" ## Private key file path. -## +toml2docs:none-default +## @toml2docs:none-default key_path = "" ## Watch for Certificate and key file change and auto reload. @@ -78,11 +78,11 @@ runtime_size = 2 mode = "disable" ## Certificate file path. -## +toml2docs:none-default +## @toml2docs:none-default cert_path = "" ## Private key file path. -## +toml2docs:none-default +## @toml2docs:none-default key_path = "" ## Watch for Certificate and key file change and auto reload @@ -103,11 +103,11 @@ runtime_size = 2 mode = "disable" ## Certificate file path. -## +toml2docs:none-default +## @toml2docs:none-default cert_path = "" ## Private key file path. -## +toml2docs:none-default +## @toml2docs:none-default key_path = "" ## Watch for Certificate and key file change and auto reload @@ -139,7 +139,7 @@ provider = "raft_engine" ## The directory to store the WAL files. ## **It's only used when the provider is `raft_engine`**. -## +toml2docs:none-default +## @toml2docs:none-default dir = "/tmp/greptimedb/wal" ## The size of the WAL segment file. @@ -320,83 +320,83 @@ type = "File" ## Cache configuration for object storage such as 'S3' etc. ## The local file cache directory. -## +toml2docs:none-default +## @toml2docs:none-default cache_path = "/path/local_cache" ## The local file cache capacity in bytes. -## +toml2docs:none-default +## @toml2docs:none-default cache_capacity = "256MB" ## The S3 bucket name. ## **It's only used when the storage type is `S3`, `Oss` and `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default bucket = "greptimedb" ## The S3 data will be stored in the specified prefix, for example, `s3://${bucket}/${root}`. ## **It's only used when the storage type is `S3`, `Oss` and `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default root = "greptimedb" ## The access key id of the aws account. ## It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key. ## **It's only used when the storage type is `S3` and `Oss`**. -## +toml2docs:none-default +## @toml2docs:none-default access_key_id = "test" ## The secret access key of the aws account. ## It's **highly recommended** to use AWS IAM roles instead of hardcoding the access key id and secret key. ## **It's only used when the storage type is `S3`**. -## +toml2docs:none-default +## @toml2docs:none-default secret_access_key = "test" ## The secret access key of the aliyun account. ## **It's only used when the storage type is `Oss`**. -## +toml2docs:none-default +## @toml2docs:none-default access_key_secret = "test" ## The account key of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default account_name = "test" ## The account key of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default account_key = "test" ## The scope of the google cloud storage. ## **It's only used when the storage type is `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default scope = "test" ## The credential path of the google cloud storage. ## **It's only used when the storage type is `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default credential_path = "test" ## The credential of the google cloud storage. ## **It's only used when the storage type is `Gcs`**. -## +toml2docs:none-default +## @toml2docs:none-default credential = "base64-credential" ## The container of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default container = "greptimedb" ## The sas token of the azure account. ## **It's only used when the storage type is `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default sas_token = "" ## The endpoint of the S3 service. ## **It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default endpoint = "https://s3.amazonaws.com" ## The region of the S3 service. ## **It's only used when the storage type is `S3`, `Oss`, `Gcs` and `Azblob`**. -## +toml2docs:none-default +## @toml2docs:none-default region = "us-west-2" # Custom storage options @@ -426,7 +426,7 @@ region = "us-west-2" [region_engine.mito] ## Number of region workers. -num_workers = 8 +#+ num_workers = 8 ## Request channel size of each worker. worker_channel_size = 128 @@ -447,26 +447,32 @@ max_background_jobs = 4 auto_flush_interval = "1h" ## Global write buffer size for all regions. If not set, it's default to 1/8 of OS memory with a max limitation of 1GB. -global_write_buffer_size = "1GB" +## @toml2docs:none-default="Auto" +#+ global_write_buffer_size = "1GB" -## Global write buffer size threshold to reject write requests. If not set, it's default to 2 times of `global_write_buffer_size` -global_write_buffer_reject_size = "2GB" +## Global write buffer size threshold to reject write requests. If not set, it's default to 2 times of `global_write_buffer_size`. +## @toml2docs:none-default="Auto" +#+ global_write_buffer_reject_size = "2GB" ## Cache size for SST metadata. Setting it to 0 to disable the cache. ## If not set, it's default to 1/32 of OS memory with a max limitation of 128MB. -sst_meta_cache_size = "128MB" +## @toml2docs:none-default="Auto" +#+ sst_meta_cache_size = "128MB" ## Cache size for vectors and arrow arrays. Setting it to 0 to disable the cache. ## If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. -vector_cache_size = "512MB" +## @toml2docs:none-default="Auto" +#+ vector_cache_size = "512MB" ## Cache size for pages of SST row groups. Setting it to 0 to disable the cache. ## If not set, it's default to 1/8 of OS memory. -page_cache_size = "512MB" +## @toml2docs:none-default="Auto" +#+ page_cache_size = "512MB" ## Cache size for time series selector (e.g. `last_value()`). Setting it to 0 to disable the cache. ## If not set, it's default to 1/16 of OS memory with a max limitation of 512MB. -selector_result_cache_size = "512MB" +## @toml2docs:none-default="Auto" +#+ selector_result_cache_size = "512MB" ## Whether to enable the experimental write cache. enable_experimental_write_cache = false @@ -478,7 +484,7 @@ experimental_write_cache_path = "" experimental_write_cache_size = "512MB" ## TTL for write cache. -## +toml2docs:none-default +## @toml2docs:none-default experimental_write_cache_ttl = "8h" ## Buffer size for SST writing. @@ -600,7 +606,7 @@ fork_dictionary_bytes = "1GiB" dir = "/tmp/greptimedb/logs" ## The log level. Can be `info`/`debug`/`warn`/`error`. -## +toml2docs:none-default +## @toml2docs:none-default level = "info" ## Enable OTLP tracing. @@ -631,10 +637,10 @@ enable = false ## The interval of export metrics. write_interval = "30s" -## For `standalone` mode, `self_import` is recommend to collect metrics generated by itself +## For `standalone` mode, `self_import` is recommended to collect metrics generated by itself ## You must create the database before enabling it. [export_metrics.self_import] -## +toml2docs:none-default +## @toml2docs:none-default db = "greptime_metrics" [export_metrics.remote_write] @@ -647,5 +653,5 @@ headers = { } ## The tracing options. Only effect when compiled with `tokio-console` feature. [tracing] ## The tokio console address. -## +toml2docs:none-default +## @toml2docs:none-default tokio_console_addr = "127.0.0.1" diff --git a/src/cmd/tests/load_config_test.rs b/src/cmd/tests/load_config_test.rs index 78d0786f7c..68d67a53a4 100644 --- a/src/cmd/tests/load_config_test.rs +++ b/src/cmd/tests/load_config_test.rs @@ -74,15 +74,9 @@ fn test_load_datanode_example_config() { }, region_engine: vec![ RegionEngineConfig::Mito(MitoConfig { - num_workers: 8, auto_flush_interval: Duration::from_secs(3600), scan_parallelism: 0, - global_write_buffer_size: ReadableSize::gb(1), global_write_buffer_reject_size: ReadableSize::gb(2), - sst_meta_cache_size: ReadableSize::mb(128), - vector_cache_size: ReadableSize::mb(512), - page_cache_size: ReadableSize::mb(512), - selector_result_cache_size: ReadableSize::mb(512), max_background_jobs: 4, experimental_write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), ..Default::default() @@ -213,15 +207,10 @@ fn test_load_standalone_example_config() { }), region_engine: vec![ RegionEngineConfig::Mito(MitoConfig { - num_workers: 8, auto_flush_interval: Duration::from_secs(3600), scan_parallelism: 0, - global_write_buffer_size: ReadableSize::gb(1), global_write_buffer_reject_size: ReadableSize::gb(2), sst_meta_cache_size: ReadableSize::mb(128), - vector_cache_size: ReadableSize::mb(512), - page_cache_size: ReadableSize::mb(512), - selector_result_cache_size: ReadableSize::mb(512), max_background_jobs: 4, experimental_write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), ..Default::default() From 67fb3d003e3c06bfc6a98eb83f13e0cdeb8cdf2d Mon Sep 17 00:00:00 2001 From: Yohan Wal <59358312+CookiePieWw@users.noreply.github.com> Date: Wed, 11 Sep 2024 16:17:57 +0800 Subject: [PATCH 003/128] feat: add respective get_by_path UDFs for JSON type (#4720) * feat: add respectiv get_by_path udf for json type * Apply review comments Co-authored-by: Weny Xu * fix: fix compile error * refactor: change name of UDFs, add some tests --------- Co-authored-by: Weny Xu --- src/common/function/src/scalars/json.rs | 7 + .../function/src/scalars/json/json_get.rs | 454 ++++++++++++++++++ .../standalone/common/function/json.result | 198 ++++++++ .../cases/standalone/common/function/json.sql | 56 +++ 4 files changed, 715 insertions(+) create mode 100644 src/common/function/src/scalars/json/json_get.rs create mode 100644 tests/cases/standalone/common/function/json.result create mode 100644 tests/cases/standalone/common/function/json.sql diff --git a/src/common/function/src/scalars/json.rs b/src/common/function/src/scalars/json.rs index 3812b33f23..26d63d3b45 100644 --- a/src/common/function/src/scalars/json.rs +++ b/src/common/function/src/scalars/json.rs @@ -13,9 +13,11 @@ // limitations under the License. use std::sync::Arc; +mod json_get; mod json_to_string; mod to_json; +use json_get::{JsonGetBool, JsonGetFloat, JsonGetInt, JsonGetString}; use json_to_string::JsonToStringFunction; use to_json::ToJsonFunction; @@ -27,5 +29,10 @@ impl JsonFunction { pub fn register(registry: &FunctionRegistry) { registry.register(Arc::new(JsonToStringFunction)); registry.register(Arc::new(ToJsonFunction)); + + registry.register(Arc::new(JsonGetInt)); + registry.register(Arc::new(JsonGetFloat)); + registry.register(Arc::new(JsonGetString)); + registry.register(Arc::new(JsonGetBool)); } } diff --git a/src/common/function/src/scalars/json/json_get.rs b/src/common/function/src/scalars/json/json_get.rs new file mode 100644 index 0000000000..78ddc1d264 --- /dev/null +++ b/src/common/function/src/scalars/json/json_get.rs @@ -0,0 +1,454 @@ +// 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::fmt::{self, Display}; + +use common_query::error::{InvalidFuncArgsSnafu, Result, UnsupportedInputDataTypeSnafu}; +use common_query::prelude::Signature; +use datafusion::logical_expr::Volatility; +use datatypes::data_type::ConcreteDataType; +use datatypes::prelude::VectorRef; +use datatypes::scalars::ScalarVectorBuilder; +use datatypes::vectors::{ + BooleanVectorBuilder, Float64VectorBuilder, Int64VectorBuilder, MutableVector, + StringVectorBuilder, +}; +use snafu::ensure; + +use crate::function::{Function, FunctionContext}; + +fn get_json_by_path(json: &[u8], path: &str) -> Option> { + let json_path = jsonb::jsonpath::parse_json_path(path.as_bytes()); + match json_path { + Ok(json_path) => { + let mut sub_jsonb = Vec::new(); + let mut sub_offsets = Vec::new(); + match jsonb::get_by_path(json, json_path, &mut sub_jsonb, &mut sub_offsets) { + Ok(_) => Some(sub_jsonb), + Err(_) => None, + } + } + _ => None, + } +} + +/// Get the value from the JSONB by the given path and return it as specified type. +/// If the path does not exist or the value is not the type specified, return `NULL`. +macro_rules! json_get { + // e.g. name = JsonGetInt, type = Int64, rust_type = i64, doc = "Get the value from the JSONB by the given path and return it as an integer." + ($name: ident, $type: ident, $rust_type: ident, $doc:expr) => { + paste::paste! { + #[doc = $doc] + #[derive(Clone, Debug, Default)] + pub struct $name; + + impl Function for $name { + fn name(&self) -> &str { + stringify!([<$name:snake>]) + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::[<$type:snake _datatype>]()) + } + + fn signature(&self) -> Signature { + Signature::exact( + vec![ + ConcreteDataType::json_datatype(), + ConcreteDataType::string_datatype(), + ], + Volatility::Immutable, + ) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 2, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect exactly two, have: {}", + columns.len() + ), + } + ); + let jsons = &columns[0]; + let paths = &columns[1]; + + let size = jsons.len(); + let datatype = jsons.data_type(); + let mut results = [<$type VectorBuilder>]::with_capacity(size); + + match datatype { + // JSON data type uses binary vector + ConcreteDataType::Binary(_) => { + for i in 0..size { + let json = jsons.get_ref(i); + let path = paths.get_ref(i); + + let json = json.as_binary(); + let path = path.as_string(); + let result = match (json, path) { + (Ok(Some(json)), Ok(Some(path))) => { + get_json_by_path(json, path) + .and_then(|json| { jsonb::[](&json).ok() }) + } + _ => None, + }; + + results.push(result); + } + } + _ => { + return UnsupportedInputDataTypeSnafu { + function: stringify!([<$name:snake>]), + datatypes: columns.iter().map(|c| c.data_type()).collect::>(), + } + .fail(); + } + } + + Ok(results.to_vector()) + } + } + + impl Display for $name { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", stringify!([<$name:snake>]).to_ascii_uppercase()) + } + } + } + }; +} + +json_get!( + JsonGetInt, + Int64, + i64, + "Get the value from the JSONB by the given path and return it as an integer." +); + +json_get!( + JsonGetFloat, + Float64, + f64, + "Get the value from the JSONB by the given path and return it as a float." +); + +json_get!( + JsonGetBool, + Boolean, + bool, + "Get the value from the JSONB by the given path and return it as a boolean." +); + +/// Get the value from the JSONB by the given path and return it as a string. +#[derive(Clone, Debug, Default)] +pub struct JsonGetString; + +impl Function for JsonGetString { + fn name(&self) -> &str { + "json_get_string" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::string_datatype()) + } + + fn signature(&self) -> Signature { + Signature::exact( + vec![ + ConcreteDataType::json_datatype(), + ConcreteDataType::string_datatype(), + ], + Volatility::Immutable, + ) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 2, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect exactly two, have: {}", + columns.len() + ), + } + ); + let jsons = &columns[0]; + let paths = &columns[1]; + + let size = jsons.len(); + let datatype = jsons.data_type(); + let mut results = StringVectorBuilder::with_capacity(size); + + match datatype { + // JSON data type uses binary vector + ConcreteDataType::Binary(_) => { + for i in 0..size { + let json = jsons.get_ref(i); + let path = paths.get_ref(i); + + let json = json.as_binary(); + let path = path.as_string(); + let result = match (json, path) { + (Ok(Some(json)), Ok(Some(path))) => { + get_json_by_path(json, path).and_then(|json| jsonb::to_str(&json).ok()) + } + _ => None, + }; + + results.push(result.as_deref()); + } + } + _ => { + return UnsupportedInputDataTypeSnafu { + function: "json_get_string", + datatypes: columns.iter().map(|c| c.data_type()).collect::>(), + } + .fail(); + } + } + + Ok(results.to_vector()) + } +} + +impl Display for JsonGetString { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", "json_get_string".to_ascii_uppercase()) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use common_query::prelude::TypeSignature; + use datatypes::scalars::ScalarVector; + use datatypes::vectors::{BinaryVector, StringVector}; + + use super::*; + + #[test] + fn test_json_get_int() { + let json_get_int = JsonGetInt; + + assert_eq!("json_get_int", json_get_int.name()); + assert_eq!( + ConcreteDataType::int64_datatype(), + json_get_int + .return_type(&[ + ConcreteDataType::json_datatype(), + ConcreteDataType::string_datatype() + ]) + .unwrap() + ); + + assert!(matches!(json_get_int.signature(), + Signature { + type_signature: TypeSignature::Exact(valid_types), + volatility: Volatility::Immutable + } if valid_types == vec![ConcreteDataType::json_datatype(), ConcreteDataType::string_datatype()] + )); + + let json_strings = [ + r#"{"a": {"b": 2}, "b": 2, "c": 3}"#, + r#"{"a": 4, "b": {"c": 6}, "c": 6}"#, + r#"{"a": 7, "b": 8, "c": {"a": 7}}"#, + ]; + let paths = vec!["$.a.b", "$.a", "$.c"]; + let results = [Some(2), Some(4), None]; + + let jsonbs = json_strings + .iter() + .map(|s| { + let value = jsonb::parse_value(s.as_bytes()).unwrap(); + value.to_vec() + }) + .collect::>(); + + let json_vector = BinaryVector::from_vec(jsonbs); + let path_vector = StringVector::from_vec(paths); + let args: Vec = vec![Arc::new(json_vector), Arc::new(path_vector)]; + let vector = json_get_int + .eval(FunctionContext::default(), &args) + .unwrap(); + + assert_eq!(3, vector.len()); + for (i, gt) in results.iter().enumerate() { + let result = vector.get_ref(i); + let result = result.as_i64().unwrap(); + assert_eq!(*gt, result); + } + } + + #[test] + fn test_json_get_float() { + let json_get_float = JsonGetFloat; + + assert_eq!("json_get_float", json_get_float.name()); + assert_eq!( + ConcreteDataType::float64_datatype(), + json_get_float + .return_type(&[ + ConcreteDataType::json_datatype(), + ConcreteDataType::string_datatype() + ]) + .unwrap() + ); + + assert!(matches!(json_get_float.signature(), + Signature { + type_signature: TypeSignature::Exact(valid_types), + volatility: Volatility::Immutable + } if valid_types == vec![ConcreteDataType::json_datatype(), ConcreteDataType::string_datatype()] + )); + + let json_strings = [ + r#"{"a": {"b": 2.1}, "b": 2.2, "c": 3.3}"#, + r#"{"a": 4.4, "b": {"c": 6.6}, "c": 6.6}"#, + r#"{"a": 7.7, "b": 8.8, "c": {"a": 7.7}}"#, + ]; + let paths = vec!["$.a.b", "$.a", "$.c"]; + let results = [Some(2.1), Some(4.4), None]; + + let jsonbs = json_strings + .iter() + .map(|s| { + let value = jsonb::parse_value(s.as_bytes()).unwrap(); + value.to_vec() + }) + .collect::>(); + + let json_vector = BinaryVector::from_vec(jsonbs); + let path_vector = StringVector::from_vec(paths); + let args: Vec = vec![Arc::new(json_vector), Arc::new(path_vector)]; + let vector = json_get_float + .eval(FunctionContext::default(), &args) + .unwrap(); + + assert_eq!(3, vector.len()); + for (i, gt) in results.iter().enumerate() { + let result = vector.get_ref(i); + let result = result.as_f64().unwrap(); + assert_eq!(*gt, result); + } + } + + #[test] + fn test_json_get_bool() { + let json_get_bool = JsonGetBool; + + assert_eq!("json_get_bool", json_get_bool.name()); + assert_eq!( + ConcreteDataType::boolean_datatype(), + json_get_bool + .return_type(&[ + ConcreteDataType::json_datatype(), + ConcreteDataType::string_datatype() + ]) + .unwrap() + ); + + assert!(matches!(json_get_bool.signature(), + Signature { + type_signature: TypeSignature::Exact(valid_types), + volatility: Volatility::Immutable + } if valid_types == vec![ConcreteDataType::json_datatype(), ConcreteDataType::string_datatype()] + )); + + let json_strings = [ + r#"{"a": {"b": true}, "b": false, "c": true}"#, + r#"{"a": false, "b": {"c": true}, "c": false}"#, + r#"{"a": true, "b": false, "c": {"a": true}}"#, + ]; + let paths = vec!["$.a.b", "$.a", "$.c"]; + let results = [Some(true), Some(false), None]; + + let jsonbs = json_strings + .iter() + .map(|s| { + let value = jsonb::parse_value(s.as_bytes()).unwrap(); + value.to_vec() + }) + .collect::>(); + + let json_vector = BinaryVector::from_vec(jsonbs); + let path_vector = StringVector::from_vec(paths); + let args: Vec = vec![Arc::new(json_vector), Arc::new(path_vector)]; + let vector = json_get_bool + .eval(FunctionContext::default(), &args) + .unwrap(); + + assert_eq!(3, vector.len()); + for (i, gt) in results.iter().enumerate() { + let result = vector.get_ref(i); + let result = result.as_boolean().unwrap(); + assert_eq!(*gt, result); + } + } + + #[test] + fn test_json_get_string() { + let json_get_string = JsonGetString; + + assert_eq!("json_get_string", json_get_string.name()); + assert_eq!( + ConcreteDataType::string_datatype(), + json_get_string + .return_type(&[ + ConcreteDataType::json_datatype(), + ConcreteDataType::string_datatype() + ]) + .unwrap() + ); + + assert!(matches!(json_get_string.signature(), + Signature { + type_signature: TypeSignature::Exact(valid_types), + volatility: Volatility::Immutable + } if valid_types == vec![ConcreteDataType::json_datatype(), ConcreteDataType::string_datatype()] + )); + + let json_strings = [ + r#"{"a": {"b": "a"}, "b": "b", "c": "c"}"#, + r#"{"a": "d", "b": {"c": "e"}, "c": "f"}"#, + r#"{"a": "g", "b": "h", "c": {"a": "g"}}"#, + ]; + let paths = vec!["$.a.b", "$.a", ""]; + let results = [Some("a"), Some("d"), None]; + + let jsonbs = json_strings + .iter() + .map(|s| { + let value = jsonb::parse_value(s.as_bytes()).unwrap(); + value.to_vec() + }) + .collect::>(); + + let json_vector = BinaryVector::from_vec(jsonbs); + let path_vector = StringVector::from_vec(paths); + let args: Vec = vec![Arc::new(json_vector), Arc::new(path_vector)]; + let vector = json_get_string + .eval(FunctionContext::default(), &args) + .unwrap(); + + assert_eq!(3, vector.len()); + for (i, gt) in results.iter().enumerate() { + let result = vector.get_ref(i); + let result = result.as_string().unwrap(); + assert_eq!(*gt, result); + } + } +} diff --git a/tests/cases/standalone/common/function/json.result b/tests/cases/standalone/common/function/json.result new file mode 100644 index 0000000000..f2a59b9d70 --- /dev/null +++ b/tests/cases/standalone/common/function/json.result @@ -0,0 +1,198 @@ +-- json_get functions -- +SELECT json_get_int(to_json('{"a": {"b": {"c": 1}}}'), 'a.b.c'); + ++---------------------------------------------------------------------+ +| json_get_int(to_json(Utf8("{"a": {"b": {"c": 1}}}")),Utf8("a.b.c")) | ++---------------------------------------------------------------------+ +| 1 | ++---------------------------------------------------------------------+ + +SELECT json_get_float(to_json('{"a": {"b": {"c": 1.234}}}'), 'a:b.c'); + ++---------------------------------------------------------------------------+ +| json_get_float(to_json(Utf8("{"a": {"b": {"c": 1.234}}}")),Utf8("a:b.c")) | ++---------------------------------------------------------------------------+ +| 1.234 | ++---------------------------------------------------------------------------+ + +SELECT json_get_string(to_json('{"a": {"b": {"c": "foo"}}}'), 'a.b:c'); + ++----------------------------------------------------------------------------+ +| json_get_string(to_json(Utf8("{"a": {"b": {"c": "foo"}}}")),Utf8("a.b:c")) | ++----------------------------------------------------------------------------+ +| foo | ++----------------------------------------------------------------------------+ + +SELECT json_get_bool(to_json('{"a": {"b": {"c": true}}}'), 'a.b["c"]'); + ++----------------------------------------------------------------------------+ +| json_get_bool(to_json(Utf8("{"a": {"b": {"c": true}}}")),Utf8("a.b["c"]")) | ++----------------------------------------------------------------------------+ +| true | ++----------------------------------------------------------------------------+ + +SELECT json_get_int(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); + ++--------------------------------------------------------------------------+ +| json_get_int(to_json(Utf8("{"a": {"b": {"c": {"d": 1}}}}")),Utf8("a.b")) | ++--------------------------------------------------------------------------+ +| | ++--------------------------------------------------------------------------+ + +SELECT json_get_string(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); + ++-----------------------------------------------------------------------------+ +| json_get_string(to_json(Utf8("{"a": {"b": {"c": {"d": 1}}}}")),Utf8("a.b")) | ++-----------------------------------------------------------------------------+ +| | ++-----------------------------------------------------------------------------+ + +-- test functions with table rows -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +Affected Rows: 0 + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1}}}'), 1); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1.234}}}'), 2); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": "foo"}}}'), 3); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": true}}}'), 4); + +Affected Rows: 1 + +SELECT json_get_int(j, 'a.b.c') FROM jsons; + ++-------------------------------------+ +| json_get_int(jsons.j,Utf8("a.b.c")) | ++-------------------------------------+ +| 1 | +| | +| | +| 1 | ++-------------------------------------+ + +SELECT json_get_float(j, 'a["b"].c') FROM jsons; + ++------------------------------------------+ +| json_get_float(jsons.j,Utf8("a["b"].c")) | ++------------------------------------------+ +| 1.0 | +| 1.234 | +| | +| 1.0 | ++------------------------------------------+ + +SELECT json_get_string(j, 'a.b.c?(@ == 1)') FROM jsons; + ++-------------------------------------------------+ +| json_get_string(jsons.j,Utf8("a.b.c?(@ == 1)")) | ++-------------------------------------------------+ +| 1 | +| | +| | +| | ++-------------------------------------------------+ + +SELECT json_get_bool(j, 'a.b.c') FROM jsons; + ++--------------------------------------+ +| json_get_bool(jsons.j,Utf8("a.b.c")) | ++--------------------------------------+ +| | +| | +| | +| true | ++--------------------------------------+ + +SELECT json_get_int(j, 'a.b["c"]') FROM jsons; + ++----------------------------------------+ +| json_get_int(jsons.j,Utf8("a.b["c"]")) | ++----------------------------------------+ +| 1 | +| | +| | +| 1 | ++----------------------------------------+ + +DROP TABLE jsons; + +Affected Rows: 0 + +-- test functions with arrays -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +Affected Rows: 0 + +INSERT INTO jsons VALUES(to_json('["a", "bcde", "", "Long time ago, there is a little pig flying in the sky"]'), 1); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(to_json('[true, false, false, false]'), 2); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(to_json('[1, 0, -2147483649, 2147483648]'), 3); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(to_json('[1.2, 3.1415926535897932384626, -3e123, 1e100]'), 4); + +Affected Rows: 1 + +SELECT json_get_int(j, '[0]') FROM jsons; + ++-----------------------------------+ +| json_get_int(jsons.j,Utf8("[0]")) | ++-----------------------------------+ +| | +| 1 | +| 1 | +| | ++-----------------------------------+ + +SELECT json_get_float(j, '[1]') FROM jsons; + ++-------------------------------------+ +| json_get_float(jsons.j,Utf8("[1]")) | ++-------------------------------------+ +| | +| 0.0 | +| 0.0 | +| 3.141592653589793 | ++-------------------------------------+ + +SELECT json_get_bool(j, '[2]') FROM jsons; + ++------------------------------------+ +| json_get_bool(jsons.j,Utf8("[2]")) | ++------------------------------------+ +| | +| false | +| | +| | ++------------------------------------+ + +SELECT json_get_string(j, '[3]') FROM jsons; + ++--------------------------------------------------------+ +| json_get_string(jsons.j,Utf8("[3]")) | ++--------------------------------------------------------+ +| Long time ago, there is a little pig flying in the sky | +| false | +| 2147483648 | +| 1e100 | ++--------------------------------------------------------+ + +DROP TABLE jsons; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/function/json.sql b/tests/cases/standalone/common/function/json.sql new file mode 100644 index 0000000000..c6214ae0f8 --- /dev/null +++ b/tests/cases/standalone/common/function/json.sql @@ -0,0 +1,56 @@ +-- json_get functions -- +SELECT json_get_int(to_json('{"a": {"b": {"c": 1}}}'), 'a.b.c'); + +SELECT json_get_float(to_json('{"a": {"b": {"c": 1.234}}}'), 'a:b.c'); + +SELECT json_get_string(to_json('{"a": {"b": {"c": "foo"}}}'), 'a.b:c'); + +SELECT json_get_bool(to_json('{"a": {"b": {"c": true}}}'), 'a.b["c"]'); + +SELECT json_get_int(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); + +SELECT json_get_string(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); + +-- test functions with table rows -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1}}}'), 1); + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1.234}}}'), 2); + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": "foo"}}}'), 3); + +INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": true}}}'), 4); + +SELECT json_get_int(j, 'a.b.c') FROM jsons; + +SELECT json_get_float(j, 'a["b"].c') FROM jsons; + +SELECT json_get_string(j, 'a.b.c?(@ == 1)') FROM jsons; + +SELECT json_get_bool(j, 'a.b.c') FROM jsons; + +SELECT json_get_int(j, 'a.b["c"]') FROM jsons; + +DROP TABLE jsons; + +-- test functions with arrays -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +INSERT INTO jsons VALUES(to_json('["a", "bcde", "", "Long time ago, there is a little pig flying in the sky"]'), 1); + +INSERT INTO jsons VALUES(to_json('[true, false, false, false]'), 2); + +INSERT INTO jsons VALUES(to_json('[1, 0, -2147483649, 2147483648]'), 3); + +INSERT INTO jsons VALUES(to_json('[1.2, 3.1415926535897932384626, -3e123, 1e100]'), 4); + +SELECT json_get_int(j, '[0]') FROM jsons; + +SELECT json_get_float(j, '[1]') FROM jsons; + +SELECT json_get_bool(j, '[2]') FROM jsons; + +SELECT json_get_string(j, '[3]') FROM jsons; + +DROP TABLE jsons; From 36b1bafbf00152cc89c8d4362ace432bbb1932a3 Mon Sep 17 00:00:00 2001 From: localhost Date: Fri, 13 Sep 2024 02:21:05 +0800 Subject: [PATCH 004/128] fix: pipeline dissert error is returned directly to the user, instead of printing a warn log (#4709) * fix: pipeline dissert error is returned directly to the user, instead of printing a warn log * chore: add more test for pipeline --- src/pipeline/src/etl.rs | 21 +++++++---- src/pipeline/src/etl/processor/dissect.rs | 14 +++---- src/pipeline/tests/dissect.rs | 34 +++++++++++++++++ src/pipeline/tests/regex.rs | 46 +++++++++++++++++++++++ 4 files changed, 98 insertions(+), 17 deletions(-) diff --git a/src/pipeline/src/etl.rs b/src/pipeline/src/etl.rs index de4c544a01..f6b3efd6e6 100644 --- a/src/pipeline/src/etl.rs +++ b/src/pipeline/src/etl.rs @@ -21,7 +21,7 @@ pub mod value; use ahash::HashSet; use common_telemetry::debug; -use itertools::{merge, Itertools}; +use itertools::Itertools; use processor::{Processor, ProcessorBuilder, Processors}; use transform::{TransformBuilders, Transformer, Transforms}; use value::Value; @@ -91,13 +91,18 @@ where debug!("required_keys: {:?}", required_keys); // intermediate keys are the keys that all processor and transformer required - let ordered_intermediate_keys: Vec = - merge(processors_required_keys, transforms_required_keys) - .cloned() - .collect::>() - .into_iter() - .sorted() - .collect(); + let ordered_intermediate_keys: Vec = [ + processors_required_keys, + transforms_required_keys, + processors_output_keys, + ] + .iter() + .flat_map(|l| l.iter()) + .collect::>() + .into_iter() + .sorted() + .cloned() + .collect_vec(); let mut final_intermediate_keys = Vec::with_capacity(ordered_intermediate_keys.len()); let mut intermediate_keys_exclude_original = diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs index 9a4b8a966e..dca88d3843 100644 --- a/src/pipeline/src/etl/processor/dissect.rs +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -817,16 +817,12 @@ impl Processor for DissectProcessor { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { - Some(Value::String(val_str)) => match self.process(val_str) { - Ok(r) => { - for (k, v) in r { - val[k] = v; - } + Some(Value::String(val_str)) => { + let r = self.process(val_str)?; + for (k, v) in r { + val[k] = v; } - Err(e) => { - warn!("dissect processor: {}", e); - } - }, + } Some(Value::Null) | None => { if !self.ignore_missing { return Err(format!( diff --git a/src/pipeline/tests/dissect.rs b/src/pipeline/tests/dissect.rs index 82ce63399c..22cf14c46b 100644 --- a/src/pipeline/tests/dissect.rs +++ b/src/pipeline/tests/dissect.rs @@ -247,3 +247,37 @@ transform: Some(StringValue("key1_key2".to_string())) ); } + +#[test] +fn test_parse_failure() { + let input_str = r#" +{ + "str": "key1 key2" +}"#; + + let pipeline_yaml = r#" +processors: + - dissect: + field: str + patterns: + - "%{key1} %{key2} %{key3}" + +transform: + - fields: + - key1 + type: string +"#; + + let input_value = serde_json::from_str::(input_str).unwrap(); + + let yaml_content = pipeline::Content::Yaml(pipeline_yaml.into()); + let pipeline: pipeline::Pipeline = + pipeline::parse(&yaml_content).expect("failed to parse pipeline"); + let mut result = pipeline.init_intermediate_state(); + + pipeline.prepare(input_value, &mut result).unwrap(); + let row = pipeline.exec_mut(&mut result); + + assert!(row.is_err()); + assert_eq!(row.err().unwrap(), "No matching pattern found"); +} diff --git a/src/pipeline/tests/regex.rs b/src/pipeline/tests/regex.rs index 5be60c9875..a8a7daaf5c 100644 --- a/src/pipeline/tests/regex.rs +++ b/src/pipeline/tests/regex.rs @@ -122,3 +122,49 @@ transform: assert_eq!(output.rows[0].values[0].value_data, None); } + +#[test] +fn test_unuse_regex_group() { + let input_value_str = r#" + [ + { + "str": "123 456" + } + ] +"#; + + let pipeline_yaml = r#" +processors: +- regex: + fields: + - str + pattern: "(?\\d+) (?\\d+)" + +transform: +- field: str_id1 + type: string +"#; + + let output = common::parse_and_exec(input_value_str, pipeline_yaml); + + assert_eq!( + output.schema, + vec![ + common::make_column_schema( + "str_id1".to_string(), + ColumnDataType::String, + SemanticType::Field, + ), + common::make_column_schema( + "greptime_timestamp".to_string(), + ColumnDataType::TimestampNanosecond, + SemanticType::Timestamp, + ), + ] + ); + + assert_eq!( + output.rows[0].values[0].value_data, + Some(StringValue("123".to_string())) + ); +} From d1dfffcdaff8be61032dddf2dfabd5060f2b0b0a Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Wed, 18 Sep 2024 12:01:30 +0900 Subject: [PATCH 005/128] chore: enable fuzz test for append table (#4702) * chore: enable fuzz test for append table * fix: fix mysql translator --- tests-fuzz/src/generator/create_expr.rs | 10 +++++++++- tests-fuzz/src/translator/mysql/create_expr.rs | 2 +- tests-fuzz/targets/fuzz_alter_logical_table.rs | 6 ++++++ tests-fuzz/targets/fuzz_alter_table.rs | 6 ++++++ tests-fuzz/targets/fuzz_create_logical_table.rs | 6 ++++++ tests-fuzz/targets/fuzz_create_table.rs | 7 +++++++ tests-fuzz/targets/fuzz_insert.rs | 7 +++++++ tests-fuzz/targets/fuzz_insert_logical_table.rs | 5 +++++ 8 files changed, 47 insertions(+), 2 deletions(-) diff --git a/tests-fuzz/src/generator/create_expr.rs b/tests-fuzz/src/generator/create_expr.rs index 93809c06d4..200485e266 100644 --- a/tests-fuzz/src/generator/create_expr.rs +++ b/tests-fuzz/src/generator/create_expr.rs @@ -243,12 +243,20 @@ pub struct CreatePhysicalTableExprGenerator { name_generator: Box>, #[builder(default = "false")] if_not_exists: bool, + #[builder(default, setter(into))] + with_clause: HashMap, } impl Generator for CreatePhysicalTableExprGenerator { type Error = Error; fn generate(&self, rng: &mut R) -> Result { + let mut options = HashMap::with_capacity(self.with_clause.len() + 1); + options.insert("physical_metric_table".to_string(), Value::from("")); + for (key, value) in &self.with_clause { + options.insert(key.to_string(), Value::from(value.to_string())); + } + Ok(CreateTableExpr { table_name: self.name_generator.gen(rng), columns: vec![ @@ -266,7 +274,7 @@ impl Generator for CreatePhysicalTableExpr if_not_exists: self.if_not_exists, partition: None, engine: "metric".to_string(), - options: [("physical_metric_table".to_string(), "".into())].into(), + options, primary_keys: vec![], }) } diff --git a/tests-fuzz/src/translator/mysql/create_expr.rs b/tests-fuzz/src/translator/mysql/create_expr.rs index 3ce659bf6e..073643b059 100644 --- a/tests-fuzz/src/translator/mysql/create_expr.rs +++ b/tests-fuzz/src/translator/mysql/create_expr.rs @@ -151,7 +151,7 @@ impl CreateTableExprTranslator { for (key, value) in &input.options { output.push(format!("\"{key}\" = \"{value}\"")); } - format!(" with ({})", output.join("\n")) + format!(" with ({})", output.join(",\n")) } } } diff --git a/tests-fuzz/targets/fuzz_alter_logical_table.rs b/tests-fuzz/targets/fuzz_alter_logical_table.rs index 3ceb5b8b45..80f017a353 100644 --- a/tests-fuzz/targets/fuzz_alter_logical_table.rs +++ b/tests-fuzz/targets/fuzz_alter_logical_table.rs @@ -14,6 +14,7 @@ #![no_main] +use std::collections::HashMap; use std::sync::Arc; use arbitrary::{Arbitrary, Unstructured}; @@ -76,12 +77,17 @@ impl Arbitrary<'_> for FuzzInput { fn generate_create_physical_table_expr(rng: &mut R) -> Result { let physical_table_if_not_exists = rng.gen_bool(0.5); + let mut with_clause = HashMap::new(); + if rng.gen_bool(0.5) { + with_clause.insert("append_mode".to_string(), "true".to_string()); + } let create_physical_table_expr = CreatePhysicalTableExprGeneratorBuilder::default() .name_generator(Box::new(MappedGenerator::new( WordGenerator, merge_two_word_map_fn(random_capitalize_map, uppercase_and_keyword_backtick_map), ))) .if_not_exists(physical_table_if_not_exists) + .with_clause(with_clause) .build() .unwrap(); create_physical_table_expr.generate(rng) diff --git a/tests-fuzz/targets/fuzz_alter_table.rs b/tests-fuzz/targets/fuzz_alter_table.rs index 8c17612d04..7f2a809c9e 100644 --- a/tests-fuzz/targets/fuzz_alter_table.rs +++ b/tests-fuzz/targets/fuzz_alter_table.rs @@ -14,6 +14,7 @@ #![no_main] +use std::collections::HashMap; use std::sync::Arc; use arbitrary::{Arbitrary, Unstructured}; @@ -71,6 +72,10 @@ enum AlterTableOption { fn generate_create_table_expr(rng: &mut R) -> Result { let max_columns = get_gt_fuzz_input_max_columns(); let columns = rng.gen_range(2..max_columns); + let mut with_clause = HashMap::new(); + if rng.gen_bool(0.5) { + with_clause.insert("append_mode".to_string(), "true".to_string()); + } let create_table_generator = CreateTableExprGeneratorBuilder::default() .name_generator(Box::new(MappedGenerator::new( WordGenerator, @@ -78,6 +83,7 @@ fn generate_create_table_expr(rng: &mut R) -> Result Resul // Create physical table let physical_table_if_not_exists = rng.gen_bool(0.5); + let mut with_clause = HashMap::new(); + if rng.gen_bool(0.5) { + with_clause.insert("append_mode".to_string(), "true".to_string()); + } let create_physical_table_expr = CreatePhysicalTableExprGeneratorBuilder::default() .name_generator(Box::new(MappedGenerator::new( WordGenerator, merge_two_word_map_fn(random_capitalize_map, uppercase_and_keyword_backtick_map), ))) .if_not_exists(physical_table_if_not_exists) + .with_clause(with_clause) .build() .unwrap() .generate(&mut rng)?; diff --git a/tests-fuzz/targets/fuzz_create_table.rs b/tests-fuzz/targets/fuzz_create_table.rs index 0d2458c15e..6d03b0dffa 100644 --- a/tests-fuzz/targets/fuzz_create_table.rs +++ b/tests-fuzz/targets/fuzz_create_table.rs @@ -14,6 +14,8 @@ #![no_main] +use std::collections::HashMap; + use common_telemetry::info; use libfuzzer_sys::arbitrary::{Arbitrary, Unstructured}; use libfuzzer_sys::fuzz_target; @@ -65,6 +67,10 @@ impl Arbitrary<'_> for FuzzInput { fn generate_expr(input: FuzzInput) -> Result { let mut rng = ChaChaRng::seed_from_u64(input.seed); let if_not_exists = rng.gen_bool(0.5); + let mut with_clause = HashMap::new(); + if rng.gen_bool(0.5) { + with_clause.insert("append_mode".to_string(), "true".to_string()); + } let create_table_generator = CreateTableExprGeneratorBuilder::default() .name_generator(Box::new(MappedGenerator::new( @@ -74,6 +80,7 @@ fn generate_expr(input: FuzzInput) -> Result { .columns(input.columns) .engine("mito") .if_not_exists(if_not_exists) + .with_clause(with_clause) .build() .unwrap(); create_table_generator.generate(&mut rng) diff --git a/tests-fuzz/targets/fuzz_insert.rs b/tests-fuzz/targets/fuzz_insert.rs index a5db2bfffe..739d6af7a3 100644 --- a/tests-fuzz/targets/fuzz_insert.rs +++ b/tests-fuzz/targets/fuzz_insert.rs @@ -14,6 +14,7 @@ #![no_main] +use std::collections::HashMap; use std::sync::Arc; use common_telemetry::info; @@ -83,6 +84,11 @@ fn generate_create_expr( input: FuzzInput, rng: &mut R, ) -> Result { + let mut with_clause = HashMap::new(); + if rng.gen_bool(0.5) { + with_clause.insert("append_mode".to_string(), "true".to_string()); + } + let create_table_generator = CreateTableExprGeneratorBuilder::default() .name_generator(Box::new(MappedGenerator::new( WordGenerator, @@ -90,6 +96,7 @@ fn generate_create_expr( ))) .columns(input.columns) .engine("mito") + .with_clause(with_clause) .ts_column_type_generator(Box::new(MySQLTsColumnTypeGenerator)) .build() .unwrap(); diff --git a/tests-fuzz/targets/fuzz_insert_logical_table.rs b/tests-fuzz/targets/fuzz_insert_logical_table.rs index 6efb6ea509..abafdef9a8 100644 --- a/tests-fuzz/targets/fuzz_insert_logical_table.rs +++ b/tests-fuzz/targets/fuzz_insert_logical_table.rs @@ -79,12 +79,17 @@ impl Arbitrary<'_> for FuzzInput { fn generate_create_physical_table_expr(rng: &mut R) -> Result { let physical_table_if_not_exists = rng.gen_bool(0.5); + let mut with_clause = HashMap::new(); + if rng.gen_bool(0.5) { + with_clause.insert("append_mode".to_string(), "true".to_string()); + } let create_physical_table_expr = CreatePhysicalTableExprGeneratorBuilder::default() .name_generator(Box::new(MappedGenerator::new( WordGenerator, merge_two_word_map_fn(random_capitalize_map, uppercase_and_keyword_backtick_map), ))) .if_not_exists(physical_table_if_not_exists) + .with_clause(with_clause) .build() .unwrap(); create_physical_table_expr.generate(rng) From 3b5b90654374cbba640d004d0871356f865efa0a Mon Sep 17 00:00:00 2001 From: Zhenchi Date: Wed, 18 Sep 2024 11:33:55 +0800 Subject: [PATCH 006/128] feat(index): add explicit adapter between `RangeReader` and `AsyncRead` (#4724) Signed-off-by: Zhenchi --- src/common/base/src/range_read.rs | 35 ++++++++++++++++--- .../src/inverted_index/format/reader/blob.rs | 10 ++++-- .../inverted_index/format/reader/footer.rs | 23 ++++++------ .../src/inverted_index/format/writer/blob.rs | 5 +-- .../src/sst/index/inverted_index/applier.rs | 2 ++ 5 files changed, 54 insertions(+), 21 deletions(-) diff --git a/src/common/base/src/range_read.rs b/src/common/base/src/range_read.rs index 920b2e1f8c..4e7a48a8b7 100644 --- a/src/common/base/src/range_read.rs +++ b/src/common/base/src/range_read.rs @@ -60,21 +60,46 @@ pub trait RangeReader: Send + Unpin { } } -/// Implement `RangeReader` for a type that implements `AsyncRead + AsyncSeek`. +#[async_trait] +impl RangeReader for &mut R { + async fn metadata(&mut self) -> io::Result { + (*self).metadata().await + } + async fn read(&mut self, range: Range) -> io::Result { + (*self).read(range).await + } + async fn read_into( + &mut self, + range: Range, + buf: &mut (impl BufMut + Send), + ) -> io::Result<()> { + (*self).read_into(range, buf).await + } + async fn read_vec(&mut self, ranges: &[Range]) -> io::Result> { + (*self).read_vec(ranges).await + } +} + +/// `RangeReaderAdapter` bridges `RangeReader` and `AsyncRead + AsyncSeek`. +pub struct RangeReaderAdapter(pub R); + +/// Implements `RangeReader` for a type that implements `AsyncRead + AsyncSeek`. /// /// TODO(zhongzc): It's a temporary solution for porting the codebase from `AsyncRead + AsyncSeek` to `RangeReader`. /// Until the codebase is fully ported to `RangeReader`, remove this implementation. #[async_trait] -impl RangeReader for R { +impl RangeReader + for RangeReaderAdapter +{ async fn metadata(&mut self) -> io::Result { - let content_length = self.seek(io::SeekFrom::End(0)).await?; + let content_length = self.0.seek(io::SeekFrom::End(0)).await?; Ok(Metadata { content_length }) } async fn read(&mut self, range: Range) -> io::Result { let mut buf = vec![0; (range.end - range.start) as usize]; - self.seek(io::SeekFrom::Start(range.start)).await?; - self.read_exact(&mut buf).await?; + self.0.seek(io::SeekFrom::Start(range.start)).await?; + self.0.read_exact(&mut buf).await?; Ok(Bytes::from(buf)) } } diff --git a/src/index/src/inverted_index/format/reader/blob.rs b/src/index/src/inverted_index/format/reader/blob.rs index 5da70e3748..f79d651e79 100644 --- a/src/index/src/inverted_index/format/reader/blob.rs +++ b/src/index/src/inverted_index/format/reader/blob.rs @@ -80,6 +80,7 @@ impl InvertedIndexReader for InvertedIndexBlobReader { #[cfg(test)] mod tests { use common_base::bit_vec::prelude::*; + use common_base::range_read::RangeReaderAdapter; use fst::MapBuilder; use futures::io::Cursor; use greptime_proto::v1::index::{InvertedIndexMeta, InvertedIndexMetas}; @@ -162,7 +163,8 @@ mod tests { #[tokio::test] async fn test_inverted_index_blob_reader_metadata() { let blob = create_inverted_index_blob(); - let mut blob_reader = InvertedIndexBlobReader::new(Cursor::new(blob)); + let cursor = RangeReaderAdapter(Cursor::new(blob)); + let mut blob_reader = InvertedIndexBlobReader::new(cursor); let metas = blob_reader.metadata().await.unwrap(); assert_eq!(metas.metas.len(), 2); @@ -189,7 +191,8 @@ mod tests { #[tokio::test] async fn test_inverted_index_blob_reader_fst() { let blob = create_inverted_index_blob(); - let mut blob_reader = InvertedIndexBlobReader::new(Cursor::new(blob)); + let cursor = RangeReaderAdapter(Cursor::new(blob)); + let mut blob_reader = InvertedIndexBlobReader::new(cursor); let metas = blob_reader.metadata().await.unwrap(); let meta = metas.metas.get("tag0").unwrap(); @@ -221,7 +224,8 @@ mod tests { #[tokio::test] async fn test_inverted_index_blob_reader_bitmap() { let blob = create_inverted_index_blob(); - let mut blob_reader = InvertedIndexBlobReader::new(Cursor::new(blob)); + let cursor = RangeReaderAdapter(Cursor::new(blob)); + let mut blob_reader = InvertedIndexBlobReader::new(cursor); let metas = blob_reader.metadata().await.unwrap(); let meta = metas.metas.get("tag0").unwrap(); diff --git a/src/index/src/inverted_index/format/reader/footer.rs b/src/index/src/inverted_index/format/reader/footer.rs index 244973669b..ffcaf9d921 100644 --- a/src/index/src/inverted_index/format/reader/footer.rs +++ b/src/index/src/inverted_index/format/reader/footer.rs @@ -24,18 +24,18 @@ use crate::inverted_index::error::{ use crate::inverted_index::format::FOOTER_PAYLOAD_SIZE_SIZE; /// InvertedIndeFooterReader is for reading the footer section of the blob. -pub struct InvertedIndeFooterReader<'a, R> { - source: &'a mut R, +pub struct InvertedIndeFooterReader { + source: R, blob_size: u64, } -impl<'a, R> InvertedIndeFooterReader<'a, R> { - pub fn new(source: &'a mut R, blob_size: u64) -> Self { +impl InvertedIndeFooterReader { + pub fn new(source: R, blob_size: u64) -> Self { Self { source, blob_size } } } -impl<'a, R: RangeReader> InvertedIndeFooterReader<'a, R> { +impl InvertedIndeFooterReader { pub async fn metadata(&mut self) -> Result { let payload_size = self.read_payload_size().await?; let metas = self.read_payload(payload_size).await?; @@ -113,6 +113,7 @@ impl<'a, R: RangeReader> InvertedIndeFooterReader<'a, R> { #[cfg(test)] mod tests { + use common_base::range_read::RangeReaderAdapter; use futures::io::Cursor; use prost::Message; @@ -142,8 +143,8 @@ mod tests { let payload_buf = create_test_payload(meta); let blob_size = payload_buf.len() as u64; - let mut cursor = Cursor::new(payload_buf); - let mut reader = InvertedIndeFooterReader::new(&mut cursor, blob_size); + let cursor = RangeReaderAdapter(Cursor::new(payload_buf)); + let mut reader = InvertedIndeFooterReader::new(cursor, blob_size); let payload_size = reader.read_payload_size().await.unwrap(); let metas = reader.read_payload(payload_size).await.unwrap(); @@ -163,8 +164,8 @@ mod tests { let mut payload_buf = create_test_payload(meta); payload_buf.push(0xff); // Add an extra byte to corrupt the footer let blob_size = payload_buf.len() as u64; - let mut cursor = Cursor::new(payload_buf); - let mut reader = InvertedIndeFooterReader::new(&mut cursor, blob_size); + let cursor = RangeReaderAdapter(Cursor::new(payload_buf)); + let mut reader = InvertedIndeFooterReader::new(cursor, blob_size); let payload_size_result = reader.read_payload_size().await; assert!(payload_size_result.is_err()); @@ -181,8 +182,8 @@ mod tests { let payload_buf = create_test_payload(meta); let blob_size = payload_buf.len() as u64; - let mut cursor = Cursor::new(payload_buf); - let mut reader = InvertedIndeFooterReader::new(&mut cursor, blob_size); + let cursor = RangeReaderAdapter(Cursor::new(payload_buf)); + let mut reader = InvertedIndeFooterReader::new(cursor, blob_size); let payload_size = reader.read_payload_size().await.unwrap(); let payload_result = reader.read_payload(payload_size).await; diff --git a/src/index/src/inverted_index/format/writer/blob.rs b/src/index/src/inverted_index/format/writer/blob.rs index 767a7a3412..26d3fb26d9 100644 --- a/src/index/src/inverted_index/format/writer/blob.rs +++ b/src/index/src/inverted_index/format/writer/blob.rs @@ -99,6 +99,7 @@ impl InvertedIndexBlobWriter { #[cfg(test)] mod tests { + use common_base::range_read::RangeReaderAdapter; use futures::io::Cursor; use futures::stream; @@ -119,7 +120,7 @@ mod tests { .await .unwrap(); - let cursor = Cursor::new(blob); + let cursor = RangeReaderAdapter(Cursor::new(blob)); let mut reader = InvertedIndexBlobReader::new(cursor); let metadata = reader.metadata().await.unwrap(); assert_eq!(metadata.total_row_count, 8); @@ -160,7 +161,7 @@ mod tests { .await .unwrap(); - let cursor = Cursor::new(blob); + let cursor = RangeReaderAdapter(Cursor::new(blob)); let mut reader = InvertedIndexBlobReader::new(cursor); let metadata = reader.metadata().await.unwrap(); assert_eq!(metadata.total_row_count, 8); diff --git a/src/mito2/src/sst/index/inverted_index/applier.rs b/src/mito2/src/sst/index/inverted_index/applier.rs index cac3ffedd7..a3482cc075 100644 --- a/src/mito2/src/sst/index/inverted_index/applier.rs +++ b/src/mito2/src/sst/index/inverted_index/applier.rs @@ -16,6 +16,7 @@ pub mod builder; use std::sync::Arc; +use common_base::range_read::RangeReaderAdapter; use common_telemetry::warn; use index::inverted_index::format::reader::InvertedIndexBlobReader; use index::inverted_index::search::index_apply::{ @@ -108,6 +109,7 @@ impl InvertedIndexApplier { self.remote_blob_reader(file_id).await? } }; + let blob = RangeReaderAdapter(blob); if let Some(index_cache) = &self.inverted_index_cache { let mut index_reader = CachedInvertedIndexBlobReader::new( From c014e875f36d1d8c5c91ed582583a3493837e986 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Wed, 18 Sep 2024 12:32:00 +0800 Subject: [PATCH 007/128] chore: add auto-decompression layer for otlp http request (#4723) * chore: add auto-decompression for http request * test: otlp --- Cargo.lock | 3 + Cargo.toml | 1 + src/frontend/src/instance/otlp.rs | 2 +- src/servers/src/http.rs | 5 + src/servers/src/http/otlp.rs | 32 ++---- tests-integration/Cargo.toml | 1 + tests-integration/src/test_util.rs | 1 + tests-integration/tests/http.rs | 152 ++++++++++++++++++++++++++--- 8 files changed, 158 insertions(+), 39 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c5b8fc016a..0e9db00527 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7228,9 +7228,11 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3a8fddc9b68f5b80dae9d6f510b88e02396f006ad48cac349411fbecc80caae4" dependencies = [ + "hex", "opentelemetry 0.22.0", "opentelemetry_sdk 0.22.1", "prost 0.12.6", + "serde", "tonic 0.11.0", ] @@ -11775,6 +11777,7 @@ dependencies = [ "datanode", "datatypes", "dotenv", + "flate2", "flow", "frontend", "futures", diff --git a/Cargo.toml b/Cargo.toml index d412bf7e97..6720671402 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -136,6 +136,7 @@ opentelemetry-proto = { version = "0.5", features = [ "gen-tonic", "metrics", "trace", + "with-serde", ] } parquet = { version = "51.0.0", default-features = false, features = ["arrow", "async", "object_store"] } paste = "1.0" diff --git a/src/frontend/src/instance/otlp.rs b/src/frontend/src/instance/otlp.rs index 8d707d12bd..09335af080 100644 --- a/src/frontend/src/instance/otlp.rs +++ b/src/frontend/src/instance/otlp.rs @@ -87,7 +87,7 @@ impl OpenTelemetryProtocolHandler for Instance { OTLP_TRACES_ROWS.inc_by(rows as u64); - self.handle_row_inserts(requests, ctx) + self.handle_log_inserts(requests, ctx) .await .map_err(BoxedError::new) .context(error::ExecuteGrpcQuerySnafu) diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 5ac52157ea..2313d19bbe 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -866,6 +866,11 @@ impl HttpServer { Router::new() .route("/v1/metrics", routing::post(otlp::metrics)) .route("/v1/traces", routing::post(otlp::traces)) + .layer( + ServiceBuilder::new() + .layer(HandleErrorLayer::new(handle_error)) + .layer(RequestDecompressionLayer::new()), + ) .with_state(otlp_handler) } diff --git a/src/servers/src/http/otlp.rs b/src/servers/src/http/otlp.rs index a04d1d42a0..3efdaeec96 100644 --- a/src/servers/src/http/otlp.rs +++ b/src/servers/src/http/otlp.rs @@ -14,12 +14,12 @@ use std::sync::Arc; -use axum::extract::{RawBody, State}; +use axum::extract::State; use axum::http::header; use axum::response::IntoResponse; use axum::Extension; +use bytes::Bytes; use common_telemetry::tracing; -use hyper::Body; use opentelemetry_proto::tonic::collector::metrics::v1::{ ExportMetricsServiceRequest, ExportMetricsServiceResponse, }; @@ -39,7 +39,7 @@ use crate::query_handler::OpenTelemetryProtocolHandlerRef; pub async fn metrics( State(handler): State, Extension(mut query_ctx): Extension, - RawBody(body): RawBody, + bytes: Bytes, ) -> Result { let db = query_ctx.get_db_string(); query_ctx.set_channel(Channel::Otlp); @@ -47,7 +47,8 @@ pub async fn metrics( let _timer = crate::metrics::METRIC_HTTP_OPENTELEMETRY_METRICS_ELAPSED .with_label_values(&[db.as_str()]) .start_timer(); - let request = parse_metrics_body(body).await?; + let request = + ExportMetricsServiceRequest::decode(bytes).context(error::DecodeOtlpRequestSnafu)?; handler .metrics(request, query_ctx) @@ -60,15 +61,6 @@ pub async fn metrics( }) } -async fn parse_metrics_body(body: Body) -> Result { - hyper::body::to_bytes(body) - .await - .context(error::HyperSnafu) - .and_then(|buf| { - ExportMetricsServiceRequest::decode(&buf[..]).context(error::DecodeOtlpRequestSnafu) - }) -} - pub struct OtlpMetricsResponse { resp_body: ExportMetricsServiceResponse, write_cost: usize, @@ -88,7 +80,7 @@ impl IntoResponse for OtlpMetricsResponse { pub async fn traces( State(handler): State, Extension(mut query_ctx): Extension, - RawBody(body): RawBody, + bytes: Bytes, ) -> Result { let db = query_ctx.get_db_string(); query_ctx.set_channel(Channel::Otlp); @@ -96,7 +88,8 @@ pub async fn traces( let _timer = crate::metrics::METRIC_HTTP_OPENTELEMETRY_TRACES_ELAPSED .with_label_values(&[db.as_str()]) .start_timer(); - let request = parse_traces_body(body).await?; + let request = + ExportTraceServiceRequest::decode(bytes).context(error::DecodeOtlpRequestSnafu)?; handler .traces(request, query_ctx) .await @@ -108,15 +101,6 @@ pub async fn traces( }) } -async fn parse_traces_body(body: Body) -> Result { - hyper::body::to_bytes(body) - .await - .context(error::HyperSnafu) - .and_then(|buf| { - ExportTraceServiceRequest::decode(&buf[..]).context(error::DecodeOtlpRequestSnafu) - }) -} - pub struct OtlpTracesResponse { resp_body: ExportTraceServiceResponse, write_cost: usize, diff --git a/tests-integration/Cargo.toml b/tests-integration/Cargo.toml index aa5f74540d..997214bca0 100644 --- a/tests-integration/Cargo.toml +++ b/tests-integration/Cargo.toml @@ -40,6 +40,7 @@ common-wal.workspace = true datanode = { workspace = true } datatypes.workspace = true dotenv.workspace = true +flate2 = "1.0" flow.workspace = true frontend = { workspace = true, features = ["testing"] } futures.workspace = true diff --git a/tests-integration/src/test_util.rs b/tests-integration/src/test_util.rs index 07237c8bc1..cf125a5776 100644 --- a/tests-integration/src/test_util.rs +++ b/tests-integration/src/test_util.rs @@ -425,6 +425,7 @@ pub async fn setup_test_http_app_with_frontend_and_user_provider( Some(instance.instance.clone()), ) .with_log_ingest_handler(instance.instance.clone(), None) + .with_otlp_handler(instance.instance.clone()) .with_greptime_config_options(instance.opts.to_toml().unwrap()); if let Some(user_provider) = user_provider { diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index fe28387cd6..d467e42dd4 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -13,11 +13,17 @@ // limitations under the License. use std::collections::BTreeMap; +use std::io::Write; use api::prom_store::remote::WriteRequest; use auth::user_provider_from_option; use axum::http::{HeaderName, StatusCode}; use common_error::status_code::StatusCode as ErrorCode; +use flate2::write::GzEncoder; +use flate2::Compression; +use opentelemetry_proto::tonic::collector::metrics::v1::ExportMetricsServiceRequest; +use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; +use opentelemetry_proto::tonic::metrics::v1::ResourceMetrics; use prost::Message; use serde_json::{json, Value}; use servers::http::error_result::ErrorResponse; @@ -26,7 +32,7 @@ use servers::http::handler::HealthResponse; use servers::http::header::{GREPTIME_DB_HEADER_NAME, GREPTIME_TIMEZONE_HEADER_NAME}; use servers::http::influxdb_result_v1::{InfluxdbOutput, InfluxdbV1Response}; use servers::http::prometheus::{PrometheusJsonResponse, PrometheusResponse}; -use servers::http::test_helpers::TestClient; +use servers::http::test_helpers::{TestClient, TestResponse}; use servers::http::GreptimeQueryOutput; use servers::prom_store; use tests_integration::test_util::{ @@ -80,6 +86,9 @@ macro_rules! http_tests { test_pipeline_api, test_test_pipeline_api, test_plain_text_ingestion, + + test_otlp_metrics, + test_otlp_traces, ); )* }; @@ -1391,19 +1400,7 @@ transform: assert_eq!(res.status(), StatusCode::OK); let resp = res.text().await; - let resp: Value = serde_json::from_str(&resp).unwrap(); - let v = resp - .get("output") - .unwrap() - .as_array() - .unwrap() - .first() - .unwrap() - .get("records") - .unwrap() - .get("rows") - .unwrap() - .to_string(); + let v = get_rows_from_output(&resp); assert_eq!( v, @@ -1412,3 +1409,130 @@ transform: guard.remove_all().await; } + +pub async fn test_otlp_metrics(store_type: StorageType) { + // init + common_telemetry::init_default_ut_logging(); + let (app, mut guard) = setup_test_http_app_with_frontend(store_type, "test_otlp_metrics").await; + + let content = r#" +{"resource":{"attributes":[],"droppedAttributesCount":0},"scopeMetrics":[{"scope":{"name":"","version":"","attributes":[],"droppedAttributesCount":0},"metrics":[{"name":"gen","description":"","unit":"","data":{"gauge":{"dataPoints":[{"attributes":[],"startTimeUnixNano":0,"timeUnixNano":1726053452870391000,"exemplars":[],"flags":0,"value":{"asInt":9471}}]}}}],"schemaUrl":""}],"schemaUrl":"https://opentelemetry.io/schemas/1.13.0"} + "#; + + let metrics: ResourceMetrics = serde_json::from_str(content).unwrap(); + let req = ExportMetricsServiceRequest { + resource_metrics: vec![metrics], + }; + let body = req.encode_to_vec(); + + // handshake + let client = TestClient::new(app); + + // write metrics data + let res = send_req(&client, "/v1/otlp/v1/metrics", body.clone(), false).await; + assert_eq!(StatusCode::OK, res.status()); + + // select metrics data + let expected = r#"[[1726053452870391000,9471.0]]"#; + validate_data(&client, "select * from gen;", expected).await; + + // drop table + let res = client.get("/v1/sql?sql=drop table gen;").send().await; + assert_eq!(res.status(), StatusCode::OK); + + // write metrics data with gzip + let res = send_req(&client, "/v1/otlp/v1/metrics", body.clone(), true).await; + assert_eq!(StatusCode::OK, res.status()); + + // select metrics data again + validate_data(&client, "select * from gen;", expected).await; + + guard.remove_all().await; +} + +pub async fn test_otlp_traces(store_type: StorageType) { + // init + common_telemetry::init_default_ut_logging(); + let (app, mut guard) = setup_test_http_app_with_frontend(store_type, "test_otlp_traces").await; + + let content = r#" +{"resourceSpans":[{"resource":{"attributes":[{"key":"service.name","value":{"stringValue":"telemetrygen"}}],"droppedAttributesCount":0},"scopeSpans":[{"scope":{"name":"telemetrygen","version":"","attributes":[],"droppedAttributesCount":0},"spans":[{"traceId":"b5e5fb572cf0a3335dd194a14145fef5","spanId":"74c82efa6f628e80","traceState":"","parentSpanId":"3364d2da58c9fd2b","flags":0,"name":"okey-dokey-0","kind":2,"startTimeUnixNano":1726631197820927000,"endTimeUnixNano":1726631197821050000,"attributes":[{"key":"net.peer.ip","value":{"stringValue":"1.2.3.4"}},{"key":"peer.service","value":{"stringValue":"telemetrygen-client"}}],"droppedAttributesCount":0,"events":[],"droppedEventsCount":0,"links":[],"droppedLinksCount":0,"status":{"message":"","code":0}},{"traceId":"b5e5fb572cf0a3335dd194a14145fef5","spanId":"3364d2da58c9fd2b","traceState":"","parentSpanId":"","flags":0,"name":"lets-go","kind":3,"startTimeUnixNano":1726631197820927000,"endTimeUnixNano":1726631197821050000,"attributes":[{"key":"net.peer.ip","value":{"stringValue":"1.2.3.4"}},{"key":"peer.service","value":{"stringValue":"telemetrygen-server"}}],"droppedAttributesCount":0,"events":[],"droppedEventsCount":0,"links":[],"droppedLinksCount":0,"status":{"message":"","code":0}}],"schemaUrl":""}],"schemaUrl":"https://opentelemetry.io/schemas/1.4.0"}]} + "#; + + let req: ExportTraceServiceRequest = serde_json::from_str(content).unwrap(); + let body = req.encode_to_vec(); + + // handshake + let client = TestClient::new(app); + + // write traces data + let res = send_req(&client, "/v1/otlp/v1/traces", body.clone(), false).await; + assert_eq!(StatusCode::OK, res.status()); + + // select traces data + let expected = r#"[["b5e5fb572cf0a3335dd194a14145fef5","3364d2da58c9fd2b","","{\"service.name\":\"telemetrygen\"}","telemetrygen","","{}","","lets-go","SPAN_KIND_CLIENT","STATUS_CODE_UNSET","","{\"net.peer.ip\":\"1.2.3.4\",\"peer.service\":\"telemetrygen-server\"}","[]","[]",1726631197820927000,1726631197821050000,0.123,1726631197820927000],["b5e5fb572cf0a3335dd194a14145fef5","74c82efa6f628e80","3364d2da58c9fd2b","{\"service.name\":\"telemetrygen\"}","telemetrygen","","{}","","okey-dokey-0","SPAN_KIND_SERVER","STATUS_CODE_UNSET","","{\"net.peer.ip\":\"1.2.3.4\",\"peer.service\":\"telemetrygen-client\"}","[]","[]",1726631197820927000,1726631197821050000,0.123,1726631197820927000]]"#; + validate_data(&client, "select * from traces_preview_v01;", expected).await; + + // drop table + let res = client + .get("/v1/sql?sql=drop table traces_preview_v01;") + .send() + .await; + assert_eq!(res.status(), StatusCode::OK); + + // write metrics data with gzip + let res = send_req(&client, "/v1/otlp/v1/traces", body.clone(), true).await; + assert_eq!(StatusCode::OK, res.status()); + + // select metrics data again + validate_data(&client, "select * from traces_preview_v01;", expected).await; + + guard.remove_all().await; +} + +async fn validate_data(client: &TestClient, sql: &str, expected: &str) { + let res = client + .get(format!("/v1/sql?sql={sql}").as_str()) + .send() + .await; + assert_eq!(res.status(), StatusCode::OK); + let resp = res.text().await; + let v = get_rows_from_output(&resp); + + assert_eq!(v, expected); +} + +async fn send_req(client: &TestClient, path: &str, body: Vec, with_gzip: bool) -> TestResponse { + let mut req = client + .post(path) + .header("content-type", "application/x-protobuf"); + + let mut len = body.len(); + + if with_gzip { + let encoded = compress_vec_with_gzip(body); + len = encoded.len(); + req = req.header("content-encoding", "gzip").body(encoded); + } else { + req = req.body(body); + } + + req.header("content-length", len).send().await +} + +fn get_rows_from_output(output: &str) -> String { + let resp: Value = serde_json::from_str(output).unwrap(); + resp.get("output") + .and_then(Value::as_array) + .and_then(|v| v.first()) + .and_then(|v| v.get("records")) + .and_then(|v| v.get("rows")) + .unwrap() + .to_string() +} + +fn compress_vec_with_gzip(data: Vec) -> Vec { + let mut encoder = GzEncoder::new(Vec::new(), Compression::default()); + encoder.write_all(&data).unwrap(); + encoder.finish().unwrap() +} From 0847ff36ce49c3c560c7a828378c49ce010817b1 Mon Sep 17 00:00:00 2001 From: zyy17 Date: Wed, 18 Sep 2024 15:41:25 +0800 Subject: [PATCH 008/128] fix: config test failed and use `similar_asserts::assert_eq` to replace `assert_eq` for long string compare (#4731) * fix: config test failed and use 'similar_asserts::assert_eq' to replace 'assert_eq' for long string compare * Update Cargo.toml Co-authored-by: Yingwen * Update src/cmd/tests/load_config_test.rs Co-authored-by: Yingwen --------- Co-authored-by: Ruihang Xia Co-authored-by: Yingwen --- Cargo.lock | 39 ++++++++++++++++++++++++++++--- Cargo.toml | 1 + src/cmd/Cargo.toml | 1 + src/cmd/tests/load_config_test.rs | 37 +++++++---------------------- tests-integration/Cargo.toml | 1 + tests-integration/tests/http.rs | 2 +- 6 files changed, 49 insertions(+), 32 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0e9db00527..b121698979 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1175,6 +1175,17 @@ dependencies = [ "regex-automata 0.1.10", ] +[[package]] +name = "bstr" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40723b8fb387abc38f4f4a37c09073622e41dd12327033091ef8950659e6dc0c" +dependencies = [ + "memchr", + "regex-automata 0.4.7", + "serde", +] + [[package]] name = "btoi" version = "0.4.3" @@ -1761,6 +1772,7 @@ dependencies = [ "serde_json", "servers", "session", + "similar-asserts", "snafu 0.8.4", "store-api", "substrait 0.9.3", @@ -9657,7 +9669,7 @@ source = "git+https://github.com/discord9/RustPython?rev=9ed5137412#9ed51374125b dependencies = [ "ascii", "bitflags 1.3.2", - "bstr", + "bstr 0.2.17", "cfg-if", "hexf-parse", "itertools 0.10.5", @@ -9692,7 +9704,7 @@ version = "0.2.0" source = "git+https://github.com/discord9/RustPython?rev=9ed5137412#9ed51374125b5f1a9e5cee5dd7e27023b8591f1e" dependencies = [ "bitflags 1.3.2", - "bstr", + "bstr 0.2.17", "itertools 0.10.5", "lz4_flex 0.9.5", "num-bigint", @@ -9845,7 +9857,7 @@ dependencies = [ "ascii", "atty", "bitflags 1.3.2", - "bstr", + "bstr 0.2.17", "caseless", "cfg-if", "chrono", @@ -10613,6 +10625,26 @@ version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" +[[package]] +name = "similar" +version = "2.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1de1d4f81173b03af4c0cbed3c898f6bff5b870e4a7f5d6f4057d62a7a4b686e" +dependencies = [ + "bstr 1.10.0", + "unicode-segmentation", +] + +[[package]] +name = "similar-asserts" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfe85670573cd6f0fa97940f26e7e6601213c3b0555246c24234131f88c5709e" +dependencies = [ + "console", + "similar", +] + [[package]] name = "simple_asn1" version = "0.6.2" @@ -11801,6 +11833,7 @@ dependencies = [ "serde_json", "servers", "session", + "similar-asserts", "snafu 0.8.4", "sql", "sqlx", diff --git a/Cargo.toml b/Cargo.toml index 6720671402..3854ebc084 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -169,6 +169,7 @@ shadow-rs = "0.31" smallvec = { version = "1", features = ["serde"] } snafu = "0.8" sysinfo = "0.30" +similar-asserts = "1.6.0" # on branch v0.44.x sqlparser = { git = "https://github.com/GreptimeTeam/sqlparser-rs.git", rev = "54a267ac89c09b11c0c88934690530807185d3e7", features = [ "visitor", diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 71841aabec..b57d221187 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -70,6 +70,7 @@ serde.workspace = true serde_json.workspace = true servers.workspace = true session.workspace = true +similar-asserts.workspace = true snafu.workspace = true store-api.workspace = true substrait.workspace = true diff --git a/src/cmd/tests/load_config_test.rs b/src/cmd/tests/load_config_test.rs index 68d67a53a4..7a50234431 100644 --- a/src/cmd/tests/load_config_test.rs +++ b/src/cmd/tests/load_config_test.rs @@ -16,12 +16,10 @@ use std::time::Duration; use cmd::options::GreptimeOptions; use cmd::standalone::StandaloneOptions; -use common_base::readable_size::ReadableSize; use common_config::Configurable; use common_grpc::channel_manager::{ DEFAULT_MAX_GRPC_RECV_MESSAGE_SIZE, DEFAULT_MAX_GRPC_SEND_MESSAGE_SIZE, }; -use common_runtime::global::RuntimeOptions; use common_telemetry::logging::{LoggingOptions, DEFAULT_OTLP_ENDPOINT}; use common_wal::config::raft_engine::RaftEngineConfig; use common_wal::config::DatanodeWalConfig; @@ -45,10 +43,6 @@ fn test_load_datanode_example_config() { .unwrap(); let expected = GreptimeOptions:: { - runtime: RuntimeOptions { - global_rt_size: 8, - compact_rt_size: 4, - }, component: DatanodeOptions { node_id: Some(42), meta_client: Some(MetaClientOptions { @@ -76,8 +70,6 @@ fn test_load_datanode_example_config() { RegionEngineConfig::Mito(MitoConfig { auto_flush_interval: Duration::from_secs(3600), scan_parallelism: 0, - global_write_buffer_reject_size: ReadableSize::gb(2), - max_background_jobs: 4, experimental_write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), ..Default::default() }), @@ -102,9 +94,10 @@ fn test_load_datanode_example_config() { rpc_max_send_message_size: Some(DEFAULT_MAX_GRPC_SEND_MESSAGE_SIZE), ..Default::default() }, + ..Default::default() }; - assert_eq!(options, expected); + similar_asserts::assert_eq!(options, expected); } #[test] @@ -114,10 +107,6 @@ fn test_load_frontend_example_config() { GreptimeOptions::::load_layered_options(example_config.to_str(), "") .unwrap(); let expected = GreptimeOptions:: { - runtime: RuntimeOptions { - global_rt_size: 8, - compact_rt_size: 4, - }, component: FrontendOptions { default_timezone: Some("UTC".to_string()), meta_client: Some(MetaClientOptions { @@ -150,8 +139,9 @@ fn test_load_frontend_example_config() { }, ..Default::default() }, + ..Default::default() }; - assert_eq!(options, expected); + similar_asserts::assert_eq!(options, expected); } #[test] @@ -161,10 +151,6 @@ fn test_load_metasrv_example_config() { GreptimeOptions::::load_layered_options(example_config.to_str(), "") .unwrap(); let expected = GreptimeOptions:: { - runtime: RuntimeOptions { - global_rt_size: 8, - compact_rt_size: 4, - }, component: MetasrvOptions { selector: SelectorType::default(), data_home: "/tmp/metasrv/".to_string(), @@ -182,8 +168,9 @@ fn test_load_metasrv_example_config() { }, ..Default::default() }, + ..Default::default() }; - assert_eq!(options, expected); + similar_asserts::assert_eq!(options, expected); } #[test] @@ -193,10 +180,6 @@ fn test_load_standalone_example_config() { GreptimeOptions::::load_layered_options(example_config.to_str(), "") .unwrap(); let expected = GreptimeOptions:: { - runtime: RuntimeOptions { - global_rt_size: 8, - compact_rt_size: 4, - }, component: StandaloneOptions { default_timezone: Some("UTC".to_string()), wal: DatanodeWalConfig::RaftEngine(RaftEngineConfig { @@ -208,11 +191,8 @@ fn test_load_standalone_example_config() { region_engine: vec![ RegionEngineConfig::Mito(MitoConfig { auto_flush_interval: Duration::from_secs(3600), - scan_parallelism: 0, - global_write_buffer_reject_size: ReadableSize::gb(2), - sst_meta_cache_size: ReadableSize::mb(128), - max_background_jobs: 4, experimental_write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), + scan_parallelism: 0, ..Default::default() }), RegionEngineConfig::File(EngineConfig {}), @@ -234,6 +214,7 @@ fn test_load_standalone_example_config() { }, ..Default::default() }, + ..Default::default() }; - assert_eq!(options, expected); + similar_asserts::assert_eq!(options, expected); } diff --git a/tests-integration/Cargo.toml b/tests-integration/Cargo.toml index 997214bca0..80fb76c55a 100644 --- a/tests-integration/Cargo.toml +++ b/tests-integration/Cargo.toml @@ -60,6 +60,7 @@ rstest_reuse.workspace = true serde_json.workspace = true servers = { workspace = true, features = ["testing"] } session.workspace = true +similar-asserts.workspace = true snafu.workspace = true sql.workspace = true sqlx = { version = "0.6", features = [ diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index d467e42dd4..e11060fbbd 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -888,7 +888,7 @@ write_interval = "30s" .trim() .to_string(); let body_text = drop_lines_with_inconsistent_results(res_get.text().await); - assert_eq!(body_text, expected_toml_str); + similar_asserts::assert_eq!(body_text, expected_toml_str); } fn drop_lines_with_inconsistent_results(input: String) -> String { From 50b3bb4c0ddf935a1bf6cc3e9525679c903397a4 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Wed, 18 Sep 2024 17:19:05 +0800 Subject: [PATCH 009/128] fix: sort cargo toml (#4735) --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 3854ebc084..d199caac09 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -166,10 +166,10 @@ serde = { version = "1.0", features = ["derive"] } serde_json = { version = "1.0", features = ["float_roundtrip"] } serde_with = "3" shadow-rs = "0.31" +similar-asserts = "1.6.0" smallvec = { version = "1", features = ["serde"] } snafu = "0.8" sysinfo = "0.30" -similar-asserts = "1.6.0" # on branch v0.44.x sqlparser = { git = "https://github.com/GreptimeTeam/sqlparser-rs.git", rev = "54a267ac89c09b11c0c88934690530807185d3e7", features = [ "visitor", From f73fb8213309c71de271fb0060dacedddcbb2d2a Mon Sep 17 00:00:00 2001 From: Yohan Wal <59358312+CookiePieWw@users.noreply.github.com> Date: Wed, 18 Sep 2024 19:07:30 +0800 Subject: [PATCH 010/128] feat: add respective `json_is` UDFs for JSON type (#4726) * feat: add respective json_is UDFs * refactor: rename to_json to parse_json * chore: happy clippy * chore: some rename * fix: small fixes --- src/common/function/src/scalars/json.rs | 18 +- .../function/src/scalars/json/json_get.rs | 2 +- .../function/src/scalars/json/json_is.rs | 215 ++++++++++++++++++ .../src/scalars/json/json_to_string.rs | 2 +- .../json/{to_json.rs => parse_json.rs} | 21 +- .../cases/standalone/common/function/json.sql | 56 ----- .../{json.result => json/json_get.result} | 151 ++++++++---- .../common/function/json/json_get.sql | 79 +++++++ .../common/function/json/json_is.result | 175 ++++++++++++++ .../common/function/json/json_is.sql | 53 +++++ .../standalone/common/types/json/json.result | 30 +-- .../standalone/common/types/json/json.sql | 30 +-- 12 files changed, 688 insertions(+), 144 deletions(-) create mode 100644 src/common/function/src/scalars/json/json_is.rs rename src/common/function/src/scalars/json/{to_json.rs => parse_json.rs} (92%) delete mode 100644 tests/cases/standalone/common/function/json.sql rename tests/cases/standalone/common/function/{json.result => json/json_get.result} (61%) create mode 100644 tests/cases/standalone/common/function/json/json_get.sql create mode 100644 tests/cases/standalone/common/function/json/json_is.result create mode 100644 tests/cases/standalone/common/function/json/json_is.sql diff --git a/src/common/function/src/scalars/json.rs b/src/common/function/src/scalars/json.rs index 26d63d3b45..279577b495 100644 --- a/src/common/function/src/scalars/json.rs +++ b/src/common/function/src/scalars/json.rs @@ -14,12 +14,16 @@ use std::sync::Arc; mod json_get; +mod json_is; mod json_to_string; -mod to_json; +mod parse_json; use json_get::{JsonGetBool, JsonGetFloat, JsonGetInt, JsonGetString}; +use json_is::{ + JsonIsArray, JsonIsBool, JsonIsFloat, JsonIsInt, JsonIsNull, JsonIsObject, JsonIsString, +}; use json_to_string::JsonToStringFunction; -use to_json::ToJsonFunction; +use parse_json::ParseJsonFunction; use crate::function_registry::FunctionRegistry; @@ -28,11 +32,19 @@ pub(crate) struct JsonFunction; impl JsonFunction { pub fn register(registry: &FunctionRegistry) { registry.register(Arc::new(JsonToStringFunction)); - registry.register(Arc::new(ToJsonFunction)); + registry.register(Arc::new(ParseJsonFunction)); registry.register(Arc::new(JsonGetInt)); registry.register(Arc::new(JsonGetFloat)); registry.register(Arc::new(JsonGetString)); registry.register(Arc::new(JsonGetBool)); + + registry.register(Arc::new(JsonIsNull)); + registry.register(Arc::new(JsonIsInt)); + registry.register(Arc::new(JsonIsFloat)); + registry.register(Arc::new(JsonIsString)); + registry.register(Arc::new(JsonIsBool)); + registry.register(Arc::new(JsonIsArray)); + registry.register(Arc::new(JsonIsObject)); } } diff --git a/src/common/function/src/scalars/json/json_get.rs b/src/common/function/src/scalars/json/json_get.rs index 78ddc1d264..d31f7a0c6e 100644 --- a/src/common/function/src/scalars/json/json_get.rs +++ b/src/common/function/src/scalars/json/json_get.rs @@ -47,7 +47,7 @@ fn get_json_by_path(json: &[u8], path: &str) -> Option> { /// If the path does not exist or the value is not the type specified, return `NULL`. macro_rules! json_get { // e.g. name = JsonGetInt, type = Int64, rust_type = i64, doc = "Get the value from the JSONB by the given path and return it as an integer." - ($name: ident, $type: ident, $rust_type: ident, $doc:expr) => { + ($name:ident, $type:ident, $rust_type:ident, $doc:expr) => { paste::paste! { #[doc = $doc] #[derive(Clone, Debug, Default)] diff --git a/src/common/function/src/scalars/json/json_is.rs b/src/common/function/src/scalars/json/json_is.rs new file mode 100644 index 0000000000..e0580ad9d4 --- /dev/null +++ b/src/common/function/src/scalars/json/json_is.rs @@ -0,0 +1,215 @@ +// 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::fmt::{self, Display}; + +use common_query::error::{InvalidFuncArgsSnafu, Result, UnsupportedInputDataTypeSnafu}; +use common_query::prelude::Signature; +use datafusion::logical_expr::Volatility; +use datatypes::data_type::ConcreteDataType; +use datatypes::prelude::VectorRef; +use datatypes::scalars::ScalarVectorBuilder; +use datatypes::vectors::{BooleanVectorBuilder, MutableVector}; +use snafu::ensure; + +use crate::function::{Function, FunctionContext}; + +/// Checks if the input is a JSON object of the given type. +macro_rules! json_is { + ($name:ident, $json_type:ident, $doc:expr) => { + paste::paste! { + #[derive(Clone, Debug, Default)] + pub struct $name; + + impl Function for $name { + fn name(&self) -> &str { + stringify!([<$name:snake>]) + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::boolean_datatype()) + } + + fn signature(&self) -> Signature { + Signature::exact(vec![ConcreteDataType::json_datatype()], Volatility::Immutable) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect exactly one, have: {}", + columns.len() + ), + } + ); + + let jsons = &columns[0]; + let size = jsons.len(); + let datatype = jsons.data_type(); + let mut results = BooleanVectorBuilder::with_capacity(size); + + match datatype { + // JSON data type uses binary vector + ConcreteDataType::Binary(_) => { + for i in 0..size { + let json = jsons.get_ref(i); + let json = json.as_binary(); + let result = match json { + Ok(Some(json)) => { + Some(jsonb::[](json)) + } + _ => None, + }; + results.push(result); + } + } + _ => { + return UnsupportedInputDataTypeSnafu { + function: stringify!([<$name:snake>]), + datatypes: columns.iter().map(|c| c.data_type()).collect::>(), + } + .fail(); + } + } + + Ok(results.to_vector()) + } + } + + impl Display for $name { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", stringify!([<$name:snake>]).to_ascii_uppercase()) + } + } + } + } +} + +json_is!(JsonIsNull, null, "Checks if the input JSONB is null"); +json_is!( + JsonIsBool, + boolean, + "Checks if the input JSONB is a boolean type JSON value" +); +json_is!( + JsonIsInt, + i64, + "Checks if the input JSONB is a integer type JSON value" +); +json_is!( + JsonIsFloat, + number, + "Checks if the input JSONB is a JSON float" +); +json_is!( + JsonIsString, + string, + "Checks if the input JSONB is a JSON string" +); +json_is!( + JsonIsArray, + array, + "Checks if the input JSONB is a JSON array" +); +json_is!( + JsonIsObject, + object, + "Checks if the input JSONB is a JSON object" +); + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use datatypes::scalars::ScalarVector; + use datatypes::vectors::BinaryVector; + + use super::*; + + #[test] + fn test_json_is_functions() { + let json_is_functions: [&dyn Function; 6] = [ + &JsonIsBool, + &JsonIsInt, + &JsonIsFloat, + &JsonIsString, + &JsonIsArray, + &JsonIsObject, + ]; + let expected_names = [ + "json_is_bool", + "json_is_int", + "json_is_float", + "json_is_string", + "json_is_array", + "json_is_object", + ]; + for (func, expected_name) in json_is_functions.iter().zip(expected_names.iter()) { + assert_eq!(func.name(), *expected_name); + assert_eq!( + func.return_type(&[ConcreteDataType::json_datatype()]) + .unwrap(), + ConcreteDataType::boolean_datatype() + ); + assert_eq!( + func.signature(), + Signature::exact( + vec![ConcreteDataType::json_datatype()], + Volatility::Immutable + ) + ); + } + + let json_strings = [ + r#"true"#, + r#"1"#, + r#"1.0"#, + r#""The pig fly through a castle, and has been attracted by the princess.""#, + r#"[1, 2]"#, + r#"{"a": 1}"#, + ]; + let expected_results = [ + [true, false, false, false, false, false], + [false, true, false, false, false, false], + // Integers are also floats + [false, true, true, false, false, false], + [false, false, false, true, false, false], + [false, false, false, false, true, false], + [false, false, false, false, false, true], + ]; + + let jsonbs = json_strings + .iter() + .map(|s| { + let value = jsonb::parse_value(s.as_bytes()).unwrap(); + value.to_vec() + }) + .collect::>(); + let json_vector = BinaryVector::from_vec(jsonbs); + let args: Vec = vec![Arc::new(json_vector)]; + + for (func, expected_result) in json_is_functions.iter().zip(expected_results.iter()) { + let vector = func.eval(FunctionContext::default(), &args).unwrap(); + assert_eq!(vector.len(), json_strings.len()); + + for (i, expected) in expected_result.iter().enumerate() { + let result = vector.get_ref(i); + let result = result.as_boolean().unwrap().unwrap(); + assert_eq!(result, *expected); + } + } + } +} diff --git a/src/common/function/src/scalars/json/json_to_string.rs b/src/common/function/src/scalars/json/json_to_string.rs index 8a5e569a14..9873000d6e 100644 --- a/src/common/function/src/scalars/json/json_to_string.rs +++ b/src/common/function/src/scalars/json/json_to_string.rs @@ -119,7 +119,7 @@ mod tests { use super::*; #[test] - fn test_get_by_path_function() { + fn test_json_to_string_function() { let json_to_string = JsonToStringFunction; assert_eq!("json_to_string", json_to_string.name()); diff --git a/src/common/function/src/scalars/json/to_json.rs b/src/common/function/src/scalars/json/parse_json.rs similarity index 92% rename from src/common/function/src/scalars/json/to_json.rs rename to src/common/function/src/scalars/json/parse_json.rs index 9c3cc90b66..64300838d8 100644 --- a/src/common/function/src/scalars/json/to_json.rs +++ b/src/common/function/src/scalars/json/parse_json.rs @@ -27,11 +27,11 @@ use crate::function::{Function, FunctionContext}; /// Parses the `String` into `JSONB`. #[derive(Clone, Debug, Default)] -pub struct ToJsonFunction; +pub struct ParseJsonFunction; -const NAME: &str = "to_json"; +const NAME: &str = "parse_json"; -impl Function for ToJsonFunction { +impl Function for ParseJsonFunction { fn name(&self) -> &str { NAME } @@ -101,9 +101,9 @@ impl Function for ToJsonFunction { } } -impl Display for ToJsonFunction { +impl Display for ParseJsonFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "TO_JSON") + write!(f, "PARSE_JSON") } } @@ -119,17 +119,17 @@ mod tests { #[test] fn test_get_by_path_function() { - let to_json = ToJsonFunction; + let parse_json = ParseJsonFunction; - assert_eq!("to_json", to_json.name()); + assert_eq!("parse_json", parse_json.name()); assert_eq!( ConcreteDataType::json_datatype(), - to_json + parse_json .return_type(&[ConcreteDataType::json_datatype()]) .unwrap() ); - assert!(matches!(to_json.signature(), + assert!(matches!(parse_json.signature(), Signature { type_signature: TypeSignature::Exact(valid_types), volatility: Volatility::Immutable @@ -152,13 +152,12 @@ mod tests { let json_string_vector = StringVector::from_vec(json_strings.to_vec()); let args: Vec = vec![Arc::new(json_string_vector)]; - let vector = to_json.eval(FunctionContext::default(), &args).unwrap(); + let vector = parse_json.eval(FunctionContext::default(), &args).unwrap(); assert_eq!(3, vector.len()); for (i, gt) in jsonbs.iter().enumerate() { let result = vector.get_ref(i); let result = result.as_binary().unwrap().unwrap(); - // remove whitespaces assert_eq!(gt, result); } } diff --git a/tests/cases/standalone/common/function/json.sql b/tests/cases/standalone/common/function/json.sql deleted file mode 100644 index c6214ae0f8..0000000000 --- a/tests/cases/standalone/common/function/json.sql +++ /dev/null @@ -1,56 +0,0 @@ --- json_get functions -- -SELECT json_get_int(to_json('{"a": {"b": {"c": 1}}}'), 'a.b.c'); - -SELECT json_get_float(to_json('{"a": {"b": {"c": 1.234}}}'), 'a:b.c'); - -SELECT json_get_string(to_json('{"a": {"b": {"c": "foo"}}}'), 'a.b:c'); - -SELECT json_get_bool(to_json('{"a": {"b": {"c": true}}}'), 'a.b["c"]'); - -SELECT json_get_int(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); - -SELECT json_get_string(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); - --- test functions with table rows -- -CREATE TABLE jsons(j JSON, ts timestamp time index); - -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1}}}'), 1); - -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1.234}}}'), 2); - -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": "foo"}}}'), 3); - -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": true}}}'), 4); - -SELECT json_get_int(j, 'a.b.c') FROM jsons; - -SELECT json_get_float(j, 'a["b"].c') FROM jsons; - -SELECT json_get_string(j, 'a.b.c?(@ == 1)') FROM jsons; - -SELECT json_get_bool(j, 'a.b.c') FROM jsons; - -SELECT json_get_int(j, 'a.b["c"]') FROM jsons; - -DROP TABLE jsons; - --- test functions with arrays -- -CREATE TABLE jsons(j JSON, ts timestamp time index); - -INSERT INTO jsons VALUES(to_json('["a", "bcde", "", "Long time ago, there is a little pig flying in the sky"]'), 1); - -INSERT INTO jsons VALUES(to_json('[true, false, false, false]'), 2); - -INSERT INTO jsons VALUES(to_json('[1, 0, -2147483649, 2147483648]'), 3); - -INSERT INTO jsons VALUES(to_json('[1.2, 3.1415926535897932384626, -3e123, 1e100]'), 4); - -SELECT json_get_int(j, '[0]') FROM jsons; - -SELECT json_get_float(j, '[1]') FROM jsons; - -SELECT json_get_bool(j, '[2]') FROM jsons; - -SELECT json_get_string(j, '[3]') FROM jsons; - -DROP TABLE jsons; diff --git a/tests/cases/standalone/common/function/json.result b/tests/cases/standalone/common/function/json/json_get.result similarity index 61% rename from tests/cases/standalone/common/function/json.result rename to tests/cases/standalone/common/function/json/json_get.result index f2a59b9d70..01767387a9 100644 --- a/tests/cases/standalone/common/function/json.result +++ b/tests/cases/standalone/common/function/json/json_get.result @@ -1,70 +1,70 @@ -- json_get functions -- -SELECT json_get_int(to_json('{"a": {"b": {"c": 1}}}'), 'a.b.c'); +SELECT json_get_int(parse_json('{"a": {"b": {"c": 1}}}'), 'a.b.c'); -+---------------------------------------------------------------------+ -| json_get_int(to_json(Utf8("{"a": {"b": {"c": 1}}}")),Utf8("a.b.c")) | -+---------------------------------------------------------------------+ -| 1 | -+---------------------------------------------------------------------+ ++------------------------------------------------------------------------+ +| json_get_int(parse_json(Utf8("{"a": {"b": {"c": 1}}}")),Utf8("a.b.c")) | ++------------------------------------------------------------------------+ +| 1 | ++------------------------------------------------------------------------+ -SELECT json_get_float(to_json('{"a": {"b": {"c": 1.234}}}'), 'a:b.c'); +SELECT json_get_float(parse_json('{"a": {"b": {"c": 1.234}}}'), 'a:b.c'); -+---------------------------------------------------------------------------+ -| json_get_float(to_json(Utf8("{"a": {"b": {"c": 1.234}}}")),Utf8("a:b.c")) | -+---------------------------------------------------------------------------+ -| 1.234 | -+---------------------------------------------------------------------------+ ++------------------------------------------------------------------------------+ +| json_get_float(parse_json(Utf8("{"a": {"b": {"c": 1.234}}}")),Utf8("a:b.c")) | ++------------------------------------------------------------------------------+ +| 1.234 | ++------------------------------------------------------------------------------+ -SELECT json_get_string(to_json('{"a": {"b": {"c": "foo"}}}'), 'a.b:c'); +SELECT json_get_string(parse_json('{"a": {"b": {"c": "foo"}}}'), 'a.b:c'); -+----------------------------------------------------------------------------+ -| json_get_string(to_json(Utf8("{"a": {"b": {"c": "foo"}}}")),Utf8("a.b:c")) | -+----------------------------------------------------------------------------+ -| foo | -+----------------------------------------------------------------------------+ ++-------------------------------------------------------------------------------+ +| json_get_string(parse_json(Utf8("{"a": {"b": {"c": "foo"}}}")),Utf8("a.b:c")) | ++-------------------------------------------------------------------------------+ +| foo | ++-------------------------------------------------------------------------------+ -SELECT json_get_bool(to_json('{"a": {"b": {"c": true}}}'), 'a.b["c"]'); +SELECT json_get_bool(parse_json('{"a": {"b": {"c": true}}}'), 'a.b["c"]'); -+----------------------------------------------------------------------------+ -| json_get_bool(to_json(Utf8("{"a": {"b": {"c": true}}}")),Utf8("a.b["c"]")) | -+----------------------------------------------------------------------------+ -| true | -+----------------------------------------------------------------------------+ ++-------------------------------------------------------------------------------+ +| json_get_bool(parse_json(Utf8("{"a": {"b": {"c": true}}}")),Utf8("a.b["c"]")) | ++-------------------------------------------------------------------------------+ +| true | ++-------------------------------------------------------------------------------+ -SELECT json_get_int(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); - -+--------------------------------------------------------------------------+ -| json_get_int(to_json(Utf8("{"a": {"b": {"c": {"d": 1}}}}")),Utf8("a.b")) | -+--------------------------------------------------------------------------+ -| | -+--------------------------------------------------------------------------+ - -SELECT json_get_string(to_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); +SELECT json_get_int(parse_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); +-----------------------------------------------------------------------------+ -| json_get_string(to_json(Utf8("{"a": {"b": {"c": {"d": 1}}}}")),Utf8("a.b")) | +| json_get_int(parse_json(Utf8("{"a": {"b": {"c": {"d": 1}}}}")),Utf8("a.b")) | +-----------------------------------------------------------------------------+ | | +-----------------------------------------------------------------------------+ +SELECT json_get_string(parse_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); + ++--------------------------------------------------------------------------------+ +| json_get_string(parse_json(Utf8("{"a": {"b": {"c": {"d": 1}}}}")),Utf8("a.b")) | ++--------------------------------------------------------------------------------+ +| | ++--------------------------------------------------------------------------------+ + -- test functions with table rows -- CREATE TABLE jsons(j JSON, ts timestamp time index); Affected Rows: 0 -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1}}}'), 1); +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1}}}'), 1); Affected Rows: 1 -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": 1.234}}}'), 2); +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1.234}}}'), 2); Affected Rows: 1 -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": "foo"}}}'), 3); +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": "foo"}}}'), 3); Affected Rows: 1 -INSERT INTO jsons VALUES(to_json('{"a": {"b": {"c": true}}}'), 4); +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": true}}}'), 4); Affected Rows: 1 @@ -132,19 +132,19 @@ CREATE TABLE jsons(j JSON, ts timestamp time index); Affected Rows: 0 -INSERT INTO jsons VALUES(to_json('["a", "bcde", "", "Long time ago, there is a little pig flying in the sky"]'), 1); +INSERT INTO jsons VALUES(parse_json('["a", "bcde", "", "Long time ago, there is a little pig flying in the sky"]'), 1); Affected Rows: 1 -INSERT INTO jsons VALUES(to_json('[true, false, false, false]'), 2); +INSERT INTO jsons VALUES(parse_json('[true, false, false, false]'), 2); Affected Rows: 1 -INSERT INTO jsons VALUES(to_json('[1, 0, -2147483649, 2147483648]'), 3); +INSERT INTO jsons VALUES(parse_json('[1, 0, -2147483649, 2147483648]'), 3); Affected Rows: 1 -INSERT INTO jsons VALUES(to_json('[1.2, 3.1415926535897932384626, -3e123, 1e100]'), 4); +INSERT INTO jsons VALUES(parse_json('[1.2, 3.1415926535897932384626, -3e123, 1e100]'), 4); Affected Rows: 1 @@ -196,3 +196,70 @@ DROP TABLE jsons; Affected Rows: 0 +-- test functions in WHERE clause -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +Affected Rows: 0 + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1}}}'), 1); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1.234}}}'), 2); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": "foo"}}}'), 3); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": true}}}'), 4); + +Affected Rows: 1 + +SELECT json_to_string(j) FROM jsons WHERE json_get_int(j, 'a.b.c') = 1; + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| {"a":{"b":{"c":1}}} | +| {"a":{"b":{"c":true}}} | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_get_float(j, 'a.b.c') = 1.234; + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| {"a":{"b":{"c":1.234}}} | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_get_string(j, 'a.b.c') = 'foo'; + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| {"a":{"b":{"c":"foo"}}} | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_get_bool(j, 'a.b.c') = true; + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| {"a":{"b":{"c":true}}} | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE CAST(json_get_int(j, 'a.b.c') AS BOOLEAN); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| {"a":{"b":{"c":1}}} | +| {"a":{"b":{"c":true}}} | ++-------------------------+ + +DROP TABLE jsons; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/function/json/json_get.sql b/tests/cases/standalone/common/function/json/json_get.sql new file mode 100644 index 0000000000..3247536b07 --- /dev/null +++ b/tests/cases/standalone/common/function/json/json_get.sql @@ -0,0 +1,79 @@ +-- json_get functions -- +SELECT json_get_int(parse_json('{"a": {"b": {"c": 1}}}'), 'a.b.c'); + +SELECT json_get_float(parse_json('{"a": {"b": {"c": 1.234}}}'), 'a:b.c'); + +SELECT json_get_string(parse_json('{"a": {"b": {"c": "foo"}}}'), 'a.b:c'); + +SELECT json_get_bool(parse_json('{"a": {"b": {"c": true}}}'), 'a.b["c"]'); + +SELECT json_get_int(parse_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); + +SELECT json_get_string(parse_json('{"a": {"b": {"c": {"d": 1}}}}'), 'a.b'); + +-- test functions with table rows -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1}}}'), 1); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1.234}}}'), 2); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": "foo"}}}'), 3); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": true}}}'), 4); + +SELECT json_get_int(j, 'a.b.c') FROM jsons; + +SELECT json_get_float(j, 'a["b"].c') FROM jsons; + +SELECT json_get_string(j, 'a.b.c?(@ == 1)') FROM jsons; + +SELECT json_get_bool(j, 'a.b.c') FROM jsons; + +SELECT json_get_int(j, 'a.b["c"]') FROM jsons; + +DROP TABLE jsons; + +-- test functions with arrays -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +INSERT INTO jsons VALUES(parse_json('["a", "bcde", "", "Long time ago, there is a little pig flying in the sky"]'), 1); + +INSERT INTO jsons VALUES(parse_json('[true, false, false, false]'), 2); + +INSERT INTO jsons VALUES(parse_json('[1, 0, -2147483649, 2147483648]'), 3); + +INSERT INTO jsons VALUES(parse_json('[1.2, 3.1415926535897932384626, -3e123, 1e100]'), 4); + +SELECT json_get_int(j, '[0]') FROM jsons; + +SELECT json_get_float(j, '[1]') FROM jsons; + +SELECT json_get_bool(j, '[2]') FROM jsons; + +SELECT json_get_string(j, '[3]') FROM jsons; + +DROP TABLE jsons; + +-- test functions in WHERE clause -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1}}}'), 1); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": 1.234}}}'), 2); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": "foo"}}}'), 3); + +INSERT INTO jsons VALUES(parse_json('{"a": {"b": {"c": true}}}'), 4); + +SELECT json_to_string(j) FROM jsons WHERE json_get_int(j, 'a.b.c') = 1; + +SELECT json_to_string(j) FROM jsons WHERE json_get_float(j, 'a.b.c') = 1.234; + +SELECT json_to_string(j) FROM jsons WHERE json_get_string(j, 'a.b.c') = 'foo'; + +SELECT json_to_string(j) FROM jsons WHERE json_get_bool(j, 'a.b.c') = true; + +SELECT json_to_string(j) FROM jsons WHERE CAST(json_get_int(j, 'a.b.c') AS BOOLEAN); + +DROP TABLE jsons; diff --git a/tests/cases/standalone/common/function/json/json_is.result b/tests/cases/standalone/common/function/json/json_is.result new file mode 100644 index 0000000000..d162fa1611 --- /dev/null +++ b/tests/cases/standalone/common/function/json/json_is.result @@ -0,0 +1,175 @@ +-- json_is functions -- +SELECT json_is_object(parse_json('{"a": 1}')); + ++----------------------------------------------+ +| json_is_object(parse_json(Utf8("{"a": 1}"))) | ++----------------------------------------------+ +| true | ++----------------------------------------------+ + +SELECT json_is_array(parse_json('[1, 2, 3]')); + ++----------------------------------------------+ +| json_is_array(parse_json(Utf8("[1, 2, 3]"))) | ++----------------------------------------------+ +| true | ++----------------------------------------------+ + +SELECT json_is_int(parse_json('1')); + ++------------------------------------+ +| json_is_int(parse_json(Utf8("1"))) | ++------------------------------------+ +| true | ++------------------------------------+ + +SELECT json_is_bool(parse_json('true')); + ++----------------------------------------+ +| json_is_bool(parse_json(Utf8("true"))) | ++----------------------------------------+ +| true | ++----------------------------------------+ + +SELECT json_is_null(parse_json('null')); + ++----------------------------------------+ +| json_is_null(parse_json(Utf8("null"))) | ++----------------------------------------+ +| true | ++----------------------------------------+ + +SELECT json_is_float(parse_json('1.2')); + ++----------------------------------------+ +| json_is_float(parse_json(Utf8("1.2"))) | ++----------------------------------------+ +| true | ++----------------------------------------+ + +SELECT json_is_string(parse_json('"foo"')); + ++-------------------------------------------+ +| json_is_string(parse_json(Utf8(""foo""))) | ++-------------------------------------------+ +| true | ++-------------------------------------------+ + +SELECT json_is_null(parse_json('{"a": 1}')); + ++--------------------------------------------+ +| json_is_null(parse_json(Utf8("{"a": 1}"))) | ++--------------------------------------------+ +| false | ++--------------------------------------------+ + +SELECT json_is_string(parse_json('[1, 2, 3]')); + ++-----------------------------------------------+ +| json_is_string(parse_json(Utf8("[1, 2, 3]"))) | ++-----------------------------------------------+ +| false | ++-----------------------------------------------+ + +SELECT json_is_float(parse_json('1')); + ++--------------------------------------+ +| json_is_float(parse_json(Utf8("1"))) | ++--------------------------------------+ +| true | ++--------------------------------------+ + +-- test json_is functions in table rows and WHERE clause -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +Affected Rows: 0 + +INSERT INTO jsons VALUES(parse_json('{"a": 1}'), 1); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('[1, 2, 3]'), 2); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('1'), 3); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('true'), 4); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('null'), 5); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('1.2'), 6); + +Affected Rows: 1 + +INSERT INTO jsons VALUES(parse_json('"foo"'), 7); + +Affected Rows: 1 + +SELECT json_to_string(j) FROM jsons WHERE json_is_object(j); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| {"a":1} | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_is_array(j); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| [1,2,3] | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_is_int(j); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| 1 | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_is_bool(j); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| true | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_is_null(j); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| null | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_is_float(j); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| 1 | +| 1.2 | ++-------------------------+ + +SELECT json_to_string(j) FROM jsons WHERE json_is_string(j); + ++-------------------------+ +| json_to_string(jsons.j) | ++-------------------------+ +| "foo" | ++-------------------------+ + +DROP TABLE jsons; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/function/json/json_is.sql b/tests/cases/standalone/common/function/json/json_is.sql new file mode 100644 index 0000000000..72e88d1da2 --- /dev/null +++ b/tests/cases/standalone/common/function/json/json_is.sql @@ -0,0 +1,53 @@ +-- json_is functions -- +SELECT json_is_object(parse_json('{"a": 1}')); + +SELECT json_is_array(parse_json('[1, 2, 3]')); + +SELECT json_is_int(parse_json('1')); + +SELECT json_is_bool(parse_json('true')); + +SELECT json_is_null(parse_json('null')); + +SELECT json_is_float(parse_json('1.2')); + +SELECT json_is_string(parse_json('"foo"')); + +SELECT json_is_null(parse_json('{"a": 1}')); + +SELECT json_is_string(parse_json('[1, 2, 3]')); + +SELECT json_is_float(parse_json('1')); + +-- test json_is functions in table rows and WHERE clause -- +CREATE TABLE jsons(j JSON, ts timestamp time index); + +INSERT INTO jsons VALUES(parse_json('{"a": 1}'), 1); + +INSERT INTO jsons VALUES(parse_json('[1, 2, 3]'), 2); + +INSERT INTO jsons VALUES(parse_json('1'), 3); + +INSERT INTO jsons VALUES(parse_json('true'), 4); + +INSERT INTO jsons VALUES(parse_json('null'), 5); + +INSERT INTO jsons VALUES(parse_json('1.2'), 6); + +INSERT INTO jsons VALUES(parse_json('"foo"'), 7); + +SELECT json_to_string(j) FROM jsons WHERE json_is_object(j); + +SELECT json_to_string(j) FROM jsons WHERE json_is_array(j); + +SELECT json_to_string(j) FROM jsons WHERE json_is_int(j); + +SELECT json_to_string(j) FROM jsons WHERE json_is_bool(j); + +SELECT json_to_string(j) FROM jsons WHERE json_is_null(j); + +SELECT json_to_string(j) FROM jsons WHERE json_is_float(j); + +SELECT json_to_string(j) FROM jsons WHERE json_is_string(j); + +DROP TABLE jsons; diff --git a/tests/cases/standalone/common/types/json/json.result b/tests/cases/standalone/common/types/json/json.result index 710d0b230a..8392b4cb4a 100644 --- a/tests/cases/standalone/common/types/json/json.result +++ b/tests/cases/standalone/common/types/json/json.result @@ -41,19 +41,19 @@ INSERT INTO jsons VALUES('[null]', 0), Affected Rows: 12 -INSERT INTO jsons VALUES(to_json('[null]'), 12), -(to_json('[true]'), 13), -(to_json('[false]'), 14), -(to_json('[0]'), 15), -(to_json('["foo"]'), 16), -(to_json('[]'), 17), -(to_json('{}'), 18), -(to_json('[0,1]'), 19), -(to_json('{"foo":"bar"}'), 20), -(to_json('{"a":null,"foo":"bar"}'), 21), -(to_json('[-1]'), 22), -(to_json('[-2147483648]'), 23), -(to_json('{"entities": { +INSERT INTO jsons VALUES(parse_json('[null]'), 12), +(parse_json('[true]'), 13), +(parse_json('[false]'), 14), +(parse_json('[0]'), 15), +(parse_json('["foo"]'), 16), +(parse_json('[]'), 17), +(parse_json('{}'), 18), +(parse_json('[0,1]'), 19), +(parse_json('{"foo":"bar"}'), 20), +(parse_json('{"a":null,"foo":"bar"}'), 21), +(parse_json('[-1]'), 22), +(parse_json('[-2147483648]'), 23), +(parse_json('{"entities": { "description": { "urls": [ { @@ -117,11 +117,11 @@ DELETE FROM jsons; Affected Rows: 25 -INSERT INTO jsons VALUES(to_json('{"a":1, "b":2, "c":3'), 4); +INSERT INTO jsons VALUES(parse_json('{"a":1, "b":2, "c":3'), 4); Error: 3001(EngineExecuteQuery), DataFusion error: Invalid function args: Cannot convert the string to json, have: {"a":1, "b":2, "c":3 -INSERT INTO jsons VALUES(to_json('Morning my friends, have a nice day :)'), 5); +INSERT INTO jsons VALUES(parse_json('Morning my friends, have a nice day :)'), 5); Error: 3001(EngineExecuteQuery), DataFusion error: Invalid function args: Cannot convert the string to json, have: Morning my friends, have a nice day :) diff --git a/tests/cases/standalone/common/types/json/json.sql b/tests/cases/standalone/common/types/json/json.sql index 57fce9a8ea..868edc59e8 100644 --- a/tests/cases/standalone/common/types/json/json.sql +++ b/tests/cases/standalone/common/types/json/json.sql @@ -37,19 +37,19 @@ INSERT INTO jsons VALUES('[null]', 0), } }}', 11); -INSERT INTO jsons VALUES(to_json('[null]'), 12), -(to_json('[true]'), 13), -(to_json('[false]'), 14), -(to_json('[0]'), 15), -(to_json('["foo"]'), 16), -(to_json('[]'), 17), -(to_json('{}'), 18), -(to_json('[0,1]'), 19), -(to_json('{"foo":"bar"}'), 20), -(to_json('{"a":null,"foo":"bar"}'), 21), -(to_json('[-1]'), 22), -(to_json('[-2147483648]'), 23), -(to_json('{"entities": { +INSERT INTO jsons VALUES(parse_json('[null]'), 12), +(parse_json('[true]'), 13), +(parse_json('[false]'), 14), +(parse_json('[0]'), 15), +(parse_json('["foo"]'), 16), +(parse_json('[]'), 17), +(parse_json('{}'), 18), +(parse_json('[0,1]'), 19), +(parse_json('{"foo":"bar"}'), 20), +(parse_json('{"a":null,"foo":"bar"}'), 21), +(parse_json('[-1]'), 22), +(parse_json('[-2147483648]'), 23), +(parse_json('{"entities": { "description": { "urls": [ { @@ -79,9 +79,9 @@ SELECT json_to_string(j), t FROM jsons; --Insert invalid json strings-- DELETE FROM jsons; -INSERT INTO jsons VALUES(to_json('{"a":1, "b":2, "c":3'), 4); +INSERT INTO jsons VALUES(parse_json('{"a":1, "b":2, "c":3'), 4); -INSERT INTO jsons VALUES(to_json('Morning my friends, have a nice day :)'), 5); +INSERT INTO jsons VALUES(parse_json('Morning my friends, have a nice day :)'), 5); SELECT json_to_string(j), t FROM jsons; From befb6d85f09b6eec51147257b7976344da55e1ca Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Thu, 19 Sep 2024 07:17:39 +0900 Subject: [PATCH 011/128] fix: determine region role by using is_readonly (#4725) fix: correct `is_writable` behavior --- src/mito2/src/engine.rs | 6 +++--- src/mito2/src/region.rs | 5 +++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index 62912b8ffb..c0655eb690 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -462,10 +462,10 @@ impl EngineInner { fn role(&self, region_id: RegionId) -> Option { self.workers.get_region(region_id).map(|region| { - if region.is_writable() { - RegionRole::Leader - } else { + if region.is_readonly() { RegionRole::Follower + } else { + RegionRole::Leader } }) } diff --git a/src/mito2/src/region.rs b/src/mito2/src/region.rs index 086fbef7d0..e2a4801643 100644 --- a/src/mito2/src/region.rs +++ b/src/mito2/src/region.rs @@ -174,6 +174,11 @@ impl MitoRegion { self.manifest_ctx.state.load() == RegionState::Writable } + /// Returns whether the region is readonly. + pub(crate) fn is_readonly(&self) -> bool { + self.manifest_ctx.state.load() == RegionState::ReadOnly + } + /// Returns the state of the region. pub(crate) fn state(&self) -> RegionState { self.manifest_ctx.state.load() From d1e0602c7687d0bcaae8670dce7b3b08b6e62951 Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:12:13 +0800 Subject: [PATCH 012/128] fix: opensrv Use After Free update (#4732) * chore: version skew * fix: even more version skew * feat: use `ring` instead of `aws-lc` for remove nasm assembler on windows * feat: use `ring` for pgwire * feat: change to use `aws-lc-sys` on windows instead * feat: change back to use `ring` * chore: provide CryptoProvider * feat: use upstream repo * feat: install ring crypto lib in main * chore: use same fn to install in tests * feat: make pgwire use `ring` --- Cargo.lock | 101 ++++++++++++++----- Cargo.toml | 2 +- src/cmd/src/bin/greptime.rs | 4 +- src/cmd/src/error.rs | 13 ++- src/servers/Cargo.toml | 17 +++- src/servers/src/lib.rs | 16 +++ src/servers/src/tls.rs | 2 + src/servers/tests/mysql/mysql_server_test.rs | 2 + src/servers/tests/postgres/mod.rs | 3 + 9 files changed, 122 insertions(+), 38 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b121698979..fdad3a09be 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -811,6 +811,33 @@ dependencies = [ "cc", ] +[[package]] +name = "aws-lc-rs" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2f95446d919226d587817a7d21379e6eb099b97b45110a7f272a444ca5c54070" +dependencies = [ + "aws-lc-sys", + "mirai-annotations", + "paste", + "zeroize", +] + +[[package]] +name = "aws-lc-sys" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5055edc4a9a1b2a917a818258cdfb86a535947feebd9981adc99667a062c6f85" +dependencies = [ + "bindgen", + "cc", + "cmake", + "dunce", + "fs_extra", + "libc", + "paste", +] + [[package]] name = "axum" version = "0.6.20" @@ -986,15 +1013,18 @@ dependencies = [ "bitflags 2.5.0", "cexpr", "clang-sys", - "itertools 0.12.1", + "itertools 0.10.5", "lazy_static", "lazycell", + "log", + "prettyplease", "proc-macro2", "quote", "regex", "rustc-hash 1.1.0", "shlex", "syn 2.0.66", + "which", ] [[package]] @@ -1247,9 +1277,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.6.0" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" +checksum = "8318a53db07bb3f8dca91a600466bdb3f2eaadeedfdbcf02e1accbad9271ba50" dependencies = [ "serde", ] @@ -1390,13 +1420,13 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.99" +version = "1.1.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96c51067fd44124faa7f870b4b1c969379ad32b2ba805aa959430ceaa384f695" +checksum = "45bcde016d64c21da4be18b655631e5ab6d3107607e71a73a9f53eb48aae23fb" dependencies = [ "jobserver", "libc", - "once_cell", + "shlex", ] [[package]] @@ -3527,6 +3557,12 @@ version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "75b325c5dbd37f80359721ad39aca5a29fb04c89279657cffdda8736d0c0b9d2" +[[package]] +name = "dunce" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" + [[package]] name = "duration-str" version = "0.11.2" @@ -4085,6 +4121,12 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "fs_extra" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42703706b716c37f96a77aea830392ad231f44c9e9a67872fa5548707e11b11c" + [[package]] name = "fsevent-sys" version = "4.1.0" @@ -4790,7 +4832,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.5.7", + "socket2 0.4.10", "tokio", "tower-service", "tracing", @@ -6403,6 +6445,12 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "mirai-annotations" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c9be0862c1b3f26a88803c4a49de6889c10e608b3ee9344e6ef5b45fb37ad3d1" + [[package]] name = "mito2" version = "0.9.3" @@ -7153,17 +7201,17 @@ dependencies = [ [[package]] name = "opensrv-mysql" version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4148ab944991b0a33be74d2636a815268974578812a9e4cf7dc785325e858154" +source = "git+https://github.com/datafuselabs/opensrv?rev=6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb#6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb" dependencies = [ "async-trait", "byteorder", + "bytes", "chrono", "mysql_common 0.32.4", "nom", "pin-project-lite", "tokio", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", ] [[package]] @@ -7779,29 +7827,24 @@ dependencies = [ [[package]] name = "pgwire" -version = "0.20.0" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c00492c52bb65e0421211b7f4c5d9de7586e53786a3b244efb00f74851206bf6" +checksum = "3770f56e1e8a608c6de40011b9a00c6b669c14d121024411701b4bc3b2a5be99" dependencies = [ "async-trait", - "base64 0.22.1", "bytes", "chrono", "derive-new 0.6.0", "futures", "hex", - "log", "md5", "postgres-types", "rand", "ring 0.17.8", - "stringprep", "thiserror", - "time", "tokio", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", "tokio-util", - "x509-certificate", ] [[package]] @@ -8392,7 +8435,7 @@ checksum = "22505a5c94da8e3b7c2996394d1c933236c4d743e81a410bcca4e6989fc066a4" dependencies = [ "bytes", "heck 0.5.0", - "itertools 0.12.1", + "itertools 0.10.5", "log", "multimap", "once_cell", @@ -8444,7 +8487,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" dependencies = [ "anyhow", - "itertools 0.12.1", + "itertools 0.10.5", "proc-macro2", "quote", "syn 2.0.66", @@ -8604,7 +8647,7 @@ dependencies = [ "indoc", "libc", "memoffset 0.9.1", - "parking_lot 0.12.3", + "parking_lot 0.11.2", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -9568,6 +9611,7 @@ version = "0.23.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "05cff451f60db80f490f3c182b77c35260baace73209e9cdbbe526bfe3a4d402" dependencies = [ + "aws-lc-rs", "log", "once_cell", "ring 0.17.8", @@ -9631,6 +9675,7 @@ version = "0.102.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff448f7e92e913c4b7d4c6d8e4540a1724b319b4152b8aef6d4cf8339712b33e" dependencies = [ + "aws-lc-rs", "ring 0.17.8", "rustls-pki-types", "untrusted 0.9.0", @@ -10403,6 +10448,7 @@ dependencies = [ "arrow-schema", "async-trait", "auth", + "aws-lc-sys", "axum", "axum-macros", "base64 0.21.7", @@ -10467,7 +10513,7 @@ dependencies = [ "regex", "reqwest", "rust-embed", - "rustls 0.22.4", + "rustls 0.23.10", "rustls-pemfile 2.1.2", "rustls-pki-types", "schemars", @@ -10485,7 +10531,7 @@ dependencies = [ "tokio", "tokio-postgres", "tokio-postgres-rustls", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", "tokio-stream", "tokio-test", "tokio-util", @@ -12143,16 +12189,15 @@ dependencies = [ [[package]] name = "tokio-postgres-rustls" -version = "0.11.1" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ea13f22eda7127c827983bdaf0d7fff9df21c8817bab02815ac277a21143677" +checksum = "04fb792ccd6bbcd4bba408eb8a292f70fc4a3589e5d793626f45190e6454b6ab" dependencies = [ - "futures", "ring 0.17.8", - "rustls 0.22.4", + "rustls 0.23.10", "tokio", "tokio-postgres", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", "x509-certificate", ] diff --git a/Cargo.toml b/Cargo.toml index d199caac09..046bf82478 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -99,7 +99,7 @@ base64 = "0.21" bigdecimal = "0.4.2" bitflags = "2.4.1" bytemuck = "1.12" -bytes = { version = "1.5", features = ["serde"] } +bytes = { version = "1.7", features = ["serde"] } chrono = { version = "0.4", features = ["serde"] } clap = { version = "4.4", features = ["derive"] } config = "0.13.0" diff --git a/src/cmd/src/bin/greptime.rs b/src/cmd/src/bin/greptime.rs index f36d0f1331..54659833fa 100644 --- a/src/cmd/src/bin/greptime.rs +++ b/src/cmd/src/bin/greptime.rs @@ -15,10 +15,11 @@ #![doc = include_str!("../../../../README.md")] use clap::{Parser, Subcommand}; -use cmd::error::Result; +use cmd::error::{InitTlsProviderSnafu, Result}; use cmd::options::GlobalOptions; use cmd::{cli, datanode, flownode, frontend, metasrv, standalone, App}; use common_version::version; +use servers::install_ring_crypto_provider; #[derive(Parser)] #[command(name = "greptime", author, version, long_version = version(), about)] @@ -94,6 +95,7 @@ async fn main() -> Result<()> { async fn main_body() -> Result<()> { setup_human_panic(); + install_ring_crypto_provider().map_err(|msg| InitTlsProviderSnafu { msg }.build())?; start(Command::parse()).await } diff --git a/src/cmd/src/error.rs b/src/cmd/src/error.rs index 08c81c414c..f042b48478 100644 --- a/src/cmd/src/error.rs +++ b/src/cmd/src/error.rs @@ -24,6 +24,12 @@ use snafu::{Location, Snafu}; #[snafu(visibility(pub))] #[stack_trace_debug] pub enum Error { + #[snafu(display("Failed to install ring crypto provider: {}", msg))] + InitTlsProvider { + #[snafu(implicit)] + location: Location, + msg: String, + }, #[snafu(display("Failed to create default catalog and schema"))] InitMetadata { #[snafu(implicit)] @@ -369,9 +375,10 @@ impl ErrorExt for Error { } Error::SubstraitEncodeLogicalPlan { source, .. } => source.status_code(), - Error::SerdeJson { .. } | Error::FileIo { .. } | Error::SpawnThread { .. } => { - StatusCode::Unexpected - } + Error::SerdeJson { .. } + | Error::FileIo { .. } + | Error::SpawnThread { .. } + | Error::InitTlsProvider { .. } => StatusCode::Unexpected, Error::Other { source, .. } => source.status_code(), diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 626fdaa404..d088961e73 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -65,10 +65,12 @@ notify.workspace = true object-pool = "0.5" once_cell.workspace = true openmetrics-parser = "0.4" -opensrv-mysql = "0.7.0" +# use crates.io version after current revision is merged in next release +# opensrv-mysql = "0.7.0" +opensrv-mysql = { git = "https://github.com/datafuselabs/opensrv", rev = "6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb" } opentelemetry-proto.workspace = true parking_lot = "0.12" -pgwire = "0.20" +pgwire = { version = "0.22", default-features = false, features = ["server-api-ring"] } pin-project = "1.0" pipeline.workspace = true postgres-types = { version = "0.2", features = ["with-chrono-0_4", "with-serde_json-1"] } @@ -85,7 +87,7 @@ rand.workspace = true regex.workspace = true reqwest.workspace = true rust-embed = { version = "6.6", features = ["debug-embed"] } -rustls = "0.22" +rustls = { version = "0.23", default-features = false, features = ["ring", "logging", "std", "tls12"] } rustls-pemfile = "2.0" rustls-pki-types = "1.0" schemars.workspace = true @@ -98,7 +100,7 @@ sql.workspace = true strum.workspace = true table.workspace = true tokio.workspace = true -tokio-rustls = "0.25" +tokio-rustls = "0.26" tokio-stream = { workspace = true, features = ["net"] } tokio-util.workspace = true tonic.workspace = true @@ -130,12 +132,17 @@ session = { workspace = true, features = ["testing"] } table.workspace = true tempfile = "3.0.0" tokio-postgres = "0.7" -tokio-postgres-rustls = "0.11" +tokio-postgres-rustls = "0.12" tokio-test = "0.4" [target.'cfg(not(windows))'.dev-dependencies] pprof = { version = "0.13", features = ["criterion", "flamegraph"] } +[target.'cfg(windows)'.dependencies] +aws-lc-sys = { version = "0.21.0", features = [ + "prebuilt-nasm", +] } # use prebuilt nasm on windows per https://github.com/aws/aws-lc-rs/blob/main/aws-lc-sys/README.md#use-of-prebuilt-nasm-objects + [build-dependencies] common-version.workspace = true diff --git a/src/servers/src/lib.rs b/src/servers/src/lib.rs index a8f97877bd..ff1af967fa 100644 --- a/src/servers/src/lib.rs +++ b/src/servers/src/lib.rs @@ -55,3 +55,19 @@ pub struct SqlPlan { plan: Option, schema: Option, } + +/// Install the ring crypto provider for rustls process-wide. see: +/// +/// https://docs.rs/rustls/latest/rustls/crypto/struct.CryptoProvider.html#using-the-per-process-default-cryptoprovider +/// +/// for more information. +pub fn install_ring_crypto_provider() -> Result<(), String> { + rustls::crypto::CryptoProvider::install_default(rustls::crypto::ring::default_provider()) + .map_err(|ret| { + format!( + "CryptoProvider already installed as: {:?}, but providing {:?}", + rustls::crypto::CryptoProvider::get_default(), + ret + ) + }) +} diff --git a/src/servers/src/tls.rs b/src/servers/src/tls.rs index b2b3550596..70c2be5d9d 100644 --- a/src/servers/src/tls.rs +++ b/src/servers/src/tls.rs @@ -239,6 +239,7 @@ pub fn maybe_watch_tls_config(tls_server_config: Arc) #[cfg(test)] mod tests { use super::*; + use crate::install_ring_crypto_provider; use crate::tls::TlsMode::Disable; #[test] @@ -392,6 +393,7 @@ mod tests { #[test] fn test_tls_file_change_watch() { common_telemetry::init_default_ut_logging(); + let _ = install_ring_crypto_provider(); let dir = tempfile::tempdir().unwrap(); let cert_path = dir.path().join("serevr.crt"); diff --git a/src/servers/tests/mysql/mysql_server_test.rs b/src/servers/tests/mysql/mysql_server_test.rs index e077409a62..ba2cdbdab2 100644 --- a/src/servers/tests/mysql/mysql_server_test.rs +++ b/src/servers/tests/mysql/mysql_server_test.rs @@ -28,6 +28,7 @@ use mysql_async::{Conn, Row, SslOpts}; use rand::rngs::StdRng; use rand::Rng; use servers::error::Result; +use servers::install_ring_crypto_provider; use servers::mysql::server::{MysqlServer, MysqlSpawnConfig, MysqlSpawnRef}; use servers::server::Server; use servers::tls::{ReloadableTlsServerConfig, TlsOption}; @@ -45,6 +46,7 @@ struct MysqlOpts<'a> { } fn create_mysql_server(table: TableRef, opts: MysqlOpts<'_>) -> Result> { + let _ = install_ring_crypto_provider(); let query_handler = create_testing_sql_query_handler(table); let io_runtime = RuntimeBuilder::default() .worker_threads(4) diff --git a/src/servers/tests/postgres/mod.rs b/src/servers/tests/postgres/mod.rs index ad135dd9d0..f3ff827db4 100644 --- a/src/servers/tests/postgres/mod.rs +++ b/src/servers/tests/postgres/mod.rs @@ -27,6 +27,7 @@ use rustls::client::danger::{ServerCertVerified, ServerCertVerifier}; use rustls::{Error, SignatureScheme}; use rustls_pki_types::{CertificateDer, ServerName}; use servers::error::Result; +use servers::install_ring_crypto_provider; use servers::postgres::PostgresServer; use servers::server::Server; use servers::tls::{ReloadableTlsServerConfig, TlsOption}; @@ -357,6 +358,8 @@ async fn test_extended_query() -> Result<()> { async fn start_test_server(server_tls: TlsOption) -> Result { common_telemetry::init_default_ut_logging(); + let _ = install_ring_crypto_provider(); + let table = MemTable::default_numbers_table(); let pg_server = create_postgres_server(table, false, server_tls, None)?; let listening = "127.0.0.1:0".parse::().unwrap(); From 08bd40333cb228c51f74529f6edf66b56c508bc9 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Thu, 19 Sep 2024 12:38:41 +0800 Subject: [PATCH 013/128] feat: add an option to turn on compression for arrow output (#4730) * feat: add an option to turn on compression for arrow output * fix: typo --- Cargo.lock | 1 + Cargo.toml | 2 +- src/servers/src/http.rs | 2 +- src/servers/src/http/arrow_result.rs | 95 ++++++++++++++++++++++++++-- src/servers/src/http/handler.rs | 9 ++- 5 files changed, 100 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index fdad3a09be..3176657706 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -443,6 +443,7 @@ dependencies = [ "arrow-schema", "flatbuffers", "lz4_flex 0.11.3", + "zstd 0.13.1", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index 046bf82478..c1eea12a53 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -90,7 +90,7 @@ aquamarine = "0.3" arrow = { version = "51.0.0", features = ["prettyprint"] } arrow-array = { version = "51.0.0", default-features = false, features = ["chrono-tz"] } arrow-flight = "51.0" -arrow-ipc = { version = "51.0.0", default-features = false, features = ["lz4"] } +arrow-ipc = { version = "51.0.0", default-features = false, features = ["lz4", "zstd"] } arrow-schema = { version = "51.0", features = ["serde"] } async-stream = "0.3" async-trait = "0.1" diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 2313d19bbe..956a650fcc 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -1136,7 +1136,7 @@ mod test { RecordBatches::try_new(schema.clone(), vec![recordbatch.clone()]).unwrap(); let outputs = vec![Ok(Output::new_with_record_batches(recordbatches))]; let json_resp = match format { - ResponseFormat::Arrow => ArrowResponse::from_output(outputs).await, + ResponseFormat::Arrow => ArrowResponse::from_output(outputs, None).await, ResponseFormat::Csv => CsvResponse::from_output(outputs).await, ResponseFormat::Table => TableResponse::from_output(outputs).await, ResponseFormat::GreptimedbV1 => GreptimedbV1Response::from_output(outputs).await, diff --git a/src/servers/src/http/arrow_result.rs b/src/servers/src/http/arrow_result.rs index e6d2441ee2..6a739fee04 100644 --- a/src/servers/src/http/arrow_result.rs +++ b/src/servers/src/http/arrow_result.rs @@ -16,7 +16,8 @@ use std::pin::Pin; use std::sync::Arc; use arrow::datatypes::Schema; -use arrow_ipc::writer::FileWriter; +use arrow_ipc::writer::{FileWriter, IpcWriteOptions}; +use arrow_ipc::CompressionType; use axum::http::{header, HeaderValue}; use axum::response::{IntoResponse, Response}; use common_error::status_code::StatusCode; @@ -41,10 +42,15 @@ pub struct ArrowResponse { async fn write_arrow_bytes( mut recordbatches: Pin>, schema: &Arc, + compression: Option, ) -> Result, Error> { let mut bytes = Vec::new(); { - let mut writer = FileWriter::try_new(&mut bytes, schema).context(error::ArrowSnafu)?; + let options = IpcWriteOptions::default() + .try_with_compression(compression) + .context(error::ArrowSnafu)?; + let mut writer = FileWriter::try_new_with_options(&mut bytes, schema, options) + .context(error::ArrowSnafu)?; while let Some(rb) = recordbatches.next().await { let rb = rb.context(error::CollectRecordbatchSnafu)?; @@ -59,8 +65,22 @@ async fn write_arrow_bytes( Ok(bytes) } +fn compression_type(compression: Option) -> Option { + match compression + .map(|compression| compression.to_lowercase()) + .as_deref() + { + Some("zstd") => Some(CompressionType::ZSTD), + Some("lz4") => Some(CompressionType::LZ4_FRAME), + _ => None, + } +} + impl ArrowResponse { - pub async fn from_output(mut outputs: Vec>) -> HttpResponse { + pub async fn from_output( + mut outputs: Vec>, + compression: Option, + ) -> HttpResponse { if outputs.len() > 1 { return HttpResponse::Error(ErrorResponse::from_error_message( StatusCode::InvalidArguments, @@ -68,6 +88,8 @@ impl ArrowResponse { )); } + let compression = compression_type(compression); + match outputs.pop() { None => HttpResponse::Arrow(ArrowResponse { data: vec![], @@ -80,7 +102,9 @@ impl ArrowResponse { }), OutputData::RecordBatches(batches) => { let schema = batches.schema(); - match write_arrow_bytes(batches.as_stream(), schema.arrow_schema()).await { + match write_arrow_bytes(batches.as_stream(), schema.arrow_schema(), compression) + .await + { Ok(payload) => HttpResponse::Arrow(ArrowResponse { data: payload, execution_time_ms: 0, @@ -90,7 +114,7 @@ impl ArrowResponse { } OutputData::Stream(batches) => { let schema = batches.schema(); - match write_arrow_bytes(batches, schema.arrow_schema()).await { + match write_arrow_bytes(batches, schema.arrow_schema(), compression).await { Ok(payload) => HttpResponse::Arrow(ArrowResponse { data: payload, execution_time_ms: 0, @@ -136,3 +160,64 @@ impl IntoResponse for ArrowResponse { .into_response() } } + +#[cfg(test)] +mod test { + use std::io::Cursor; + + use arrow_ipc::reader::FileReader; + use arrow_schema::DataType; + use common_recordbatch::{RecordBatch, RecordBatches}; + use datatypes::prelude::*; + use datatypes::schema::{ColumnSchema, Schema}; + use datatypes::vectors::{StringVector, UInt32Vector}; + + use super::*; + + #[tokio::test] + async fn test_arrow_output() { + let column_schemas = vec![ + ColumnSchema::new("numbers", ConcreteDataType::uint32_datatype(), false), + ColumnSchema::new("strings", ConcreteDataType::string_datatype(), true), + ]; + let schema = Arc::new(Schema::new(column_schemas)); + let columns: Vec = vec![ + Arc::new(UInt32Vector::from_slice(vec![1, 2, 3, 4])), + Arc::new(StringVector::from(vec![ + None, + Some("hello"), + Some("greptime"), + None, + ])), + ]; + + for compression in [None, Some("zstd".to_string()), Some("lz4".to_string())].into_iter() { + let recordbatch = RecordBatch::new(schema.clone(), columns.clone()).unwrap(); + let recordbatches = + RecordBatches::try_new(schema.clone(), vec![recordbatch.clone()]).unwrap(); + let outputs = vec![Ok(Output::new_with_record_batches(recordbatches))]; + + let http_resp = ArrowResponse::from_output(outputs, compression).await; + match http_resp { + HttpResponse::Arrow(resp) => { + let output = resp.data; + let mut reader = + FileReader::try_new(Cursor::new(output), None).expect("Arrow reader error"); + let schema = reader.schema(); + assert_eq!(schema.fields[0].name(), "numbers"); + assert_eq!(schema.fields[0].data_type(), &DataType::UInt32); + assert_eq!(schema.fields[1].name(), "strings"); + assert_eq!(schema.fields[1].data_type(), &DataType::Utf8); + + let rb = reader.next().unwrap().expect("read record batch failed"); + assert_eq!(rb.num_columns(), 2); + assert_eq!(rb.num_rows(), 4); + } + HttpResponse::Error(e) => { + panic!("unexpected {:?}", e); + } + _ => unreachable!(), + } + } + } +} diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 4d5ca58461..1befc22240 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -51,7 +51,8 @@ use crate::query_handler::sql::ServerSqlQueryHandlerRef; pub struct SqlQuery { pub db: Option, pub sql: Option, - // (Optional) result format: [`greptimedb_v1`, `influxdb_v1`, `csv`], + // (Optional) result format: [`greptimedb_v1`, `influxdb_v1`, `csv`, + // `arrow`], // the default value is `greptimedb_v1` pub format: Option, // Returns epoch timestamps with the specified precision. @@ -64,6 +65,8 @@ pub struct SqlQuery { // param too. pub epoch: Option, pub limit: Option, + // For arrow output + pub compression: Option, } /// Handler to execute sql @@ -128,7 +131,9 @@ pub async fn sql( }; let mut resp = match format { - ResponseFormat::Arrow => ArrowResponse::from_output(outputs).await, + ResponseFormat::Arrow => { + ArrowResponse::from_output(outputs, query_params.compression).await + } ResponseFormat::Csv => CsvResponse::from_output(outputs).await, ResponseFormat::Table => TableResponse::from_output(outputs).await, ResponseFormat::GreptimedbV1 => GreptimedbV1Response::from_output(outputs).await, From b5f7138d337ee2229c3b45633e9ccc38f8600104 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Thu, 19 Sep 2024 12:44:02 +0800 Subject: [PATCH 014/128] refactor(tables): improve `tables` performance (#4737) * chore: cherrypick 52e8eebb2dbbbe81179583c05094004a5eedd7fd * refactor/tables: Change variable from immutable to mutable in KvBackendCatalogManager's method * refactor/tables: Replace unbounded channel with bounded and use semaphore for concurrency control in KvBackendCatalogManager * refactor/tables: Add common-runtime dependency and update KvBackendCatalogManager to use common_runtime::spawn_global * refactor/tables: Await on sending error through channel in KvBackendCatalogManager --- Cargo.lock | 3 + src/catalog/Cargo.toml | 2 + src/catalog/src/kvbackend/manager.rs | 83 ++++++++---- src/common/meta/Cargo.toml | 1 + src/common/meta/src/key/catalog_name.rs | 3 +- src/common/meta/src/key/datanode_table.rs | 3 +- src/common/meta/src/key/flow/flow_name.rs | 3 +- src/common/meta/src/key/flow/flow_route.rs | 3 +- src/common/meta/src/key/flow/flownode_flow.rs | 3 +- src/common/meta/src/key/flow/table_flow.rs | 3 +- src/common/meta/src/key/schema_name.rs | 3 +- src/common/meta/src/key/table_name.rs | 3 +- src/common/meta/src/range_stream.rs | 126 ++++-------------- src/common/meta/src/state_store.rs | 3 +- src/log-store/src/raft_engine/backend.rs | 3 +- src/meta-srv/src/service/store/cached_kv.rs | 5 +- 16 files changed, 108 insertions(+), 142 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3176657706..786c1c3a8b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1381,6 +1381,7 @@ dependencies = [ "common-meta", "common-query", "common-recordbatch", + "common-runtime", "common-telemetry", "common-test-util", "common-time", @@ -1408,6 +1409,7 @@ dependencies = [ "store-api", "table", "tokio", + "tokio-stream", ] [[package]] @@ -2109,6 +2111,7 @@ dependencies = [ "anymap2", "api", "async-recursion", + "async-stream", "async-trait", "base64 0.21.7", "bytes", diff --git a/src/catalog/Cargo.toml b/src/catalog/Cargo.toml index a484171733..cfea5e02c6 100644 --- a/src/catalog/Cargo.toml +++ b/src/catalog/Cargo.toml @@ -24,6 +24,7 @@ common-macro.workspace = true common-meta.workspace = true common-query.workspace = true common-recordbatch.workspace = true +common-runtime.workspace = true common-telemetry.workspace = true common-time.workspace = true common-version.workspace = true @@ -48,6 +49,7 @@ sql.workspace = true store-api.workspace = true table.workspace = true tokio.workspace = true +tokio-stream = "0.1" [dev-dependencies] cache.workspace = true diff --git a/src/catalog/src/kvbackend/manager.rs b/src/catalog/src/kvbackend/manager.rs index feb5e31d09..1559022514 100644 --- a/src/catalog/src/kvbackend/manager.rs +++ b/src/catalog/src/kvbackend/manager.rs @@ -42,6 +42,8 @@ use table::dist_table::DistTable; use table::table::numbers::{NumbersTable, NUMBERS_TABLE_NAME}; use table::table_name::TableName; use table::TableRef; +use tokio::sync::Semaphore; +use tokio_stream::wrappers::ReceiverStream; use crate::error::{ CacheNotFoundSnafu, GetTableCacheSnafu, InvalidTableInfoInCatalogSnafu, ListCatalogsSnafu, @@ -179,21 +181,18 @@ impl CatalogManager for KvBackendCatalogManager { schema: &str, query_ctx: Option<&QueryContext>, ) -> Result> { - let stream = self + let mut tables = self .table_metadata_manager .table_name_manager() - .tables(catalog, schema); - let mut tables = stream + .tables(catalog, schema) + .map_ok(|(table_name, _)| table_name) .try_collect::>() .await .map_err(BoxedError::new) - .context(ListTablesSnafu { catalog, schema })? - .into_iter() - .map(|(k, _)| k) - .collect::>(); - tables.extend_from_slice(&self.system_catalog.table_names(schema, query_ctx)); + .context(ListTablesSnafu { catalog, schema })?; - Ok(tables.into_iter().collect()) + tables.extend(self.system_catalog.table_names(schema, query_ctx)); + Ok(tables) } async fn catalog_exists(&self, catalog: &str) -> Result { @@ -303,36 +302,68 @@ impl CatalogManager for KvBackendCatalogManager { } }); - let table_id_stream = self - .table_metadata_manager - .table_name_manager() - .tables(catalog, schema) - .map_ok(|(_, v)| v.table_id()); const BATCH_SIZE: usize = 128; - let user_tables = try_stream!({ + const CONCURRENCY: usize = 8; + + let (tx, rx) = tokio::sync::mpsc::channel(64); + let metadata_manager = self.table_metadata_manager.clone(); + let catalog = catalog.to_string(); + let schema = schema.to_string(); + let semaphore = Arc::new(Semaphore::new(CONCURRENCY)); + + common_runtime::spawn_global(async move { + let table_id_stream = metadata_manager + .table_name_manager() + .tables(&catalog, &schema) + .map_ok(|(_, v)| v.table_id()); // Split table ids into chunks let mut table_id_chunks = table_id_stream.ready_chunks(BATCH_SIZE); while let Some(table_ids) = table_id_chunks.next().await { - let table_ids = table_ids + let table_ids = match table_ids .into_iter() .collect::, _>>() .map_err(BoxedError::new) - .context(ListTablesSnafu { catalog, schema })?; + .context(ListTablesSnafu { + catalog: &catalog, + schema: &schema, + }) { + Ok(table_ids) => table_ids, + Err(e) => { + let _ = tx.send(Err(e)).await; + return; + } + }; - let table_info_values = self - .table_metadata_manager - .table_info_manager() - .batch_get(&table_ids) - .await - .context(TableMetadataManagerSnafu)?; + let metadata_manager = metadata_manager.clone(); + let tx = tx.clone(); + let semaphore = semaphore.clone(); + common_runtime::spawn_global(async move { + // we don't explicitly close the semaphore so just ignore the potential error. + let _ = semaphore.acquire().await; + let table_info_values = match metadata_manager + .table_info_manager() + .batch_get(&table_ids) + .await + .context(TableMetadataManagerSnafu) + { + Ok(table_info_values) => table_info_values, + Err(e) => { + let _ = tx.send(Err(e)).await; + return; + } + }; - for table_info_value in table_info_values.into_values() { - yield build_table(table_info_value)?; - } + for table in table_info_values.into_values().map(build_table) { + if tx.send(table).await.is_err() { + return; + } + } + }); } }); + let user_tables = ReceiverStream::new(rx); Box::pin(sys_tables.chain(user_tables)) } } diff --git a/src/common/meta/Cargo.toml b/src/common/meta/Cargo.toml index 28f518190a..591055d472 100644 --- a/src/common/meta/Cargo.toml +++ b/src/common/meta/Cargo.toml @@ -15,6 +15,7 @@ workspace = true anymap2 = "0.13.0" api.workspace = true async-recursion = "1.0" +async-stream = "0.3" async-trait.workspace = true base64.workspace = true bytes.workspace = true diff --git a/src/common/meta/src/key/catalog_name.rs b/src/common/meta/src/key/catalog_name.rs index a3a9841618..9238f95c82 100644 --- a/src/common/meta/src/key/catalog_name.rs +++ b/src/common/meta/src/key/catalog_name.rs @@ -147,7 +147,8 @@ impl CatalogManager { req, DEFAULT_PAGE_SIZE, Arc::new(catalog_decoder), - ); + ) + .into_stream(); Box::pin(stream) } diff --git a/src/common/meta/src/key/datanode_table.rs b/src/common/meta/src/key/datanode_table.rs index b53060cd16..b10d536672 100644 --- a/src/common/meta/src/key/datanode_table.rs +++ b/src/common/meta/src/key/datanode_table.rs @@ -167,7 +167,8 @@ impl DatanodeTableManager { req, DEFAULT_PAGE_SIZE, Arc::new(datanode_table_value_decoder), - ); + ) + .into_stream(); Box::pin(stream) } diff --git a/src/common/meta/src/key/flow/flow_name.rs b/src/common/meta/src/key/flow/flow_name.rs index 201c354f99..3a331be800 100644 --- a/src/common/meta/src/key/flow/flow_name.rs +++ b/src/common/meta/src/key/flow/flow_name.rs @@ -200,7 +200,8 @@ impl FlowNameManager { req, DEFAULT_PAGE_SIZE, Arc::new(flow_name_decoder), - ); + ) + .into_stream(); Box::pin(stream) } diff --git a/src/common/meta/src/key/flow/flow_route.rs b/src/common/meta/src/key/flow/flow_route.rs index 96db01aef0..47ee94ce95 100644 --- a/src/common/meta/src/key/flow/flow_route.rs +++ b/src/common/meta/src/key/flow/flow_route.rs @@ -180,7 +180,8 @@ impl FlowRouteManager { req, DEFAULT_PAGE_SIZE, Arc::new(flow_route_decoder), - ); + ) + .into_stream(); Box::pin(stream) } diff --git a/src/common/meta/src/key/flow/flownode_flow.rs b/src/common/meta/src/key/flow/flownode_flow.rs index 4b4c31a7b0..552abfcdbe 100644 --- a/src/common/meta/src/key/flow/flownode_flow.rs +++ b/src/common/meta/src/key/flow/flownode_flow.rs @@ -180,7 +180,8 @@ impl FlownodeFlowManager { req, DEFAULT_PAGE_SIZE, Arc::new(flownode_flow_key_decoder), - ); + ) + .into_stream(); Box::pin(stream.map_ok(|key| (key.flow_id(), key.partition_id()))) } diff --git a/src/common/meta/src/key/flow/table_flow.rs b/src/common/meta/src/key/flow/table_flow.rs index 5297734a23..c4f47cde51 100644 --- a/src/common/meta/src/key/flow/table_flow.rs +++ b/src/common/meta/src/key/flow/table_flow.rs @@ -207,7 +207,8 @@ impl TableFlowManager { req, DEFAULT_PAGE_SIZE, Arc::new(table_flow_decoder), - ); + ) + .into_stream(); Box::pin(stream) } diff --git a/src/common/meta/src/key/schema_name.rs b/src/common/meta/src/key/schema_name.rs index 7c9822e18d..f9cbc4a9f2 100644 --- a/src/common/meta/src/key/schema_name.rs +++ b/src/common/meta/src/key/schema_name.rs @@ -230,7 +230,8 @@ impl SchemaManager { req, DEFAULT_PAGE_SIZE, Arc::new(schema_decoder), - ); + ) + .into_stream(); Box::pin(stream) } diff --git a/src/common/meta/src/key/table_name.rs b/src/common/meta/src/key/table_name.rs index 7054ffc4e9..a632e3a233 100644 --- a/src/common/meta/src/key/table_name.rs +++ b/src/common/meta/src/key/table_name.rs @@ -259,7 +259,8 @@ impl TableNameManager { req, DEFAULT_PAGE_SIZE, Arc::new(table_decoder), - ); + ) + .into_stream(); Box::pin(stream) } diff --git a/src/common/meta/src/range_stream.rs b/src/common/meta/src/range_stream.rs index b14a2bf8f1..be54865281 100644 --- a/src/common/meta/src/range_stream.rs +++ b/src/common/meta/src/range_stream.rs @@ -12,14 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::VecDeque; -use std::pin::Pin; use std::sync::Arc; -use std::task::{Context, Poll}; +use async_stream::try_stream; use common_telemetry::debug; -use futures::future::BoxFuture; -use futures::{ready, FutureExt, Stream}; +use futures::Stream; use snafu::ensure; use crate::error::{self, Result}; @@ -30,17 +27,6 @@ use crate::util::get_next_prefix_key; pub type KeyValueDecoderFn = dyn Fn(KeyValue) -> Result + Send + Sync; -enum PaginationStreamState { - /// At the start of reading. - Init, - /// Decoding key value pairs. - Decoding(SimpleKeyValueDecoder), - /// Retrieving data from backend. - Reading(BoxFuture<'static, Result<(PaginationStreamFactory, Option)>>), - /// Error - Error, -} - /// The Range Request's default page size. /// /// It dependents on upstream KvStore server side grpc message size limitation. @@ -65,8 +51,6 @@ struct PaginationStreamFactory { /// keys. pub range_end: Vec, - /// page_size is the pagination page size. - page_size: usize, /// keys_only when set returns only the keys and not the values. pub keys_only: bool, @@ -89,7 +73,6 @@ impl PaginationStreamFactory { kv: kv.clone(), key, range_end, - page_size, keys_only, more, adaptive_page_size: if page_size == 0 { @@ -137,7 +120,7 @@ impl PaginationStreamFactory { } } - async fn read_next(mut self) -> Result<(Self, Option)> { + async fn read_next(&mut self) -> Result> { if self.more { let resp = self .adaptive_range(RangeRequest { @@ -151,33 +134,22 @@ impl PaginationStreamFactory { let key = resp .kvs .last() - .map(|kv| kv.key.clone()) - .unwrap_or_else(Vec::new); + .map(|kv| kv.key.as_slice()) + .unwrap_or_default(); - let next_key = get_next_prefix_key(&key); - - Ok(( - Self { - kv: self.kv, - key: next_key, - range_end: self.range_end, - page_size: self.page_size, - keys_only: self.keys_only, - more: resp.more, - adaptive_page_size: self.adaptive_page_size, - }, - Some(resp), - )) + let next_key = get_next_prefix_key(key); + self.key = next_key; + self.more = resp.more; + Ok(Some(resp)) } else { - Ok((self, None)) + Ok(None) } } } pub struct PaginationStream { - state: PaginationStreamState, decoder_fn: Arc>, - factory: Option, + factory: PaginationStreamFactory, } impl PaginationStream { @@ -189,82 +161,28 @@ impl PaginationStream { decoder_fn: Arc>, ) -> Self { Self { - state: PaginationStreamState::Init, decoder_fn, - factory: Some(PaginationStreamFactory::new( + factory: PaginationStreamFactory::new( &kv, req.key, req.range_end, page_size, req.keys_only, true, - )), + ), } } } -struct SimpleKeyValueDecoder { - kv: VecDeque, - decoder: Arc>, -} - -impl Iterator for SimpleKeyValueDecoder { - type Item = Result; - - fn next(&mut self) -> Option { - if let Some(kv) = self.kv.pop_front() { - Some((self.decoder)(kv)) - } else { - None - } - } -} - -impl Stream for PaginationStream { - type Item = Result; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - loop { - match &mut self.state { - PaginationStreamState::Decoding(decoder) => match decoder.next() { - Some(Ok(result)) => return Poll::Ready(Some(Ok(result))), - Some(Err(e)) => { - self.state = PaginationStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - None => self.state = PaginationStreamState::Init, - }, - PaginationStreamState::Init => { - let factory = self.factory.take().expect("lost factory"); - if !factory.more { - // Ensures the factory always exists. - self.factory = Some(factory); - return Poll::Ready(None); - } - let fut = factory.read_next().boxed(); - self.state = PaginationStreamState::Reading(fut); +impl PaginationStream { + pub fn into_stream(mut self) -> impl Stream> { + try_stream!({ + while let Some(resp) = self.factory.read_next().await? { + for kv in resp.kvs { + yield (self.decoder_fn)(kv)? } - PaginationStreamState::Reading(f) => match ready!(f.poll_unpin(cx)) { - Ok((factory, Some(resp))) => { - self.factory = Some(factory); - let decoder = SimpleKeyValueDecoder { - kv: resp.kvs.into(), - decoder: self.decoder_fn.clone(), - }; - self.state = PaginationStreamState::Decoding(decoder); - } - Ok((factory, None)) => { - self.factory = Some(factory); - self.state = PaginationStreamState::Init; - } - Err(e) => { - self.state = PaginationStreamState::Error; - return Poll::Ready(Some(Err(e))); - } - }, - PaginationStreamState::Error => return Poll::Ready(None), // Ends the stream as error happens. } - } + }) } } @@ -333,7 +251,8 @@ mod tests { }, DEFAULT_PAGE_SIZE, Arc::new(decoder), - ); + ) + .into_stream(); let kv = stream.try_collect::>().await.unwrap(); assert!(kv.is_empty()); @@ -374,6 +293,7 @@ mod tests { Arc::new(decoder), ); let kv = stream + .into_stream() .try_collect::>() .await .unwrap() diff --git a/src/common/meta/src/state_store.rs b/src/common/meta/src/state_store.rs index 1cf1ea8649..89d5dfd0ff 100644 --- a/src/common/meta/src/state_store.rs +++ b/src/common/meta/src/state_store.rs @@ -172,7 +172,8 @@ impl StateStore for KvStateStore { req, self.max_num_per_range_request.unwrap_or_default(), Arc::new(decode_kv), - ); + ) + .into_stream(); let stream = stream.map(move |r| { let path = path.clone(); diff --git a/src/log-store/src/raft_engine/backend.rs b/src/log-store/src/raft_engine/backend.rs index e2cd65c8fa..33cb64a2e8 100644 --- a/src/log-store/src/raft_engine/backend.rs +++ b/src/log-store/src/raft_engine/backend.rs @@ -264,9 +264,8 @@ impl KvBackend for RaftEngineBackend { let mut response = BatchGetResponse { kvs: Vec::with_capacity(req.keys.len()), }; - let engine = self.engine.read().unwrap(); for key in req.keys { - let Some(value) = engine.get(SYSTEM_NAMESPACE, &key) else { + let Some(value) = self.engine.read().unwrap().get(SYSTEM_NAMESPACE, &key) else { continue; }; response.kvs.push(KeyValue { key, value }); diff --git a/src/meta-srv/src/service/store/cached_kv.rs b/src/meta-srv/src/service/store/cached_kv.rs index ddf7b3e516..d4b6f84f58 100644 --- a/src/meta-srv/src/service/store/cached_kv.rs +++ b/src/meta-srv/src/service/store/cached_kv.rs @@ -103,9 +103,10 @@ impl LeaderCachedKvBackend { RangeRequest::new().with_prefix(prefix.as_bytes()), DEFAULT_PAGE_SIZE, Arc::new(Ok), - ); + ) + .into_stream(); - let kvs = stream.try_collect::>().await?.into_iter().collect(); + let kvs = stream.try_collect::>().await?; self.cache .batch_put(BatchPutRequest { From 52d627e37db26b9830024de9b005444207299436 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Thu, 19 Sep 2024 13:14:47 +0800 Subject: [PATCH 015/128] chore: add log ingest interceptor (#4734) * chore: add log ingest interceptor * chore: rename * chore: update interceptor signature --- src/frontend/src/instance/log_handler.rs | 11 ++++- src/frontend/src/server.rs | 8 ++- src/pipeline/src/etl/processor/dissect.rs | 8 +-- src/servers/src/http.rs | 18 +++---- src/servers/src/http/event.rs | 10 +++- src/servers/src/interceptor.rs | 59 +++++++++++++++++++++++ tests-integration/src/test_util.rs | 2 +- 7 files changed, 96 insertions(+), 20 deletions(-) diff --git a/src/frontend/src/instance/log_handler.rs b/src/frontend/src/instance/log_handler.rs index 7edda5ccf1..441501b242 100644 --- a/src/frontend/src/instance/log_handler.rs +++ b/src/frontend/src/instance/log_handler.rs @@ -20,7 +20,10 @@ use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; use common_error::ext::BoxedError; use pipeline::{GreptimeTransformer, Pipeline, PipelineInfo, PipelineVersion}; -use servers::error::{AuthSnafu, ExecuteGrpcRequestSnafu, PipelineSnafu, Result as ServerResult}; +use servers::error::{ + AuthSnafu, Error as ServerError, ExecuteGrpcRequestSnafu, PipelineSnafu, Result as ServerResult, +}; +use servers::interceptor::{LogIngestInterceptor, LogIngestInterceptorRef}; use servers::query_handler::LogHandler; use session::context::QueryContextRef; use snafu::ResultExt; @@ -40,6 +43,12 @@ impl LogHandler for Instance { .check_permission(ctx.current_user(), PermissionReq::LogWrite) .context(AuthSnafu)?; + let log = self + .plugins + .get::>() + .as_ref() + .pre_ingest(log, ctx.clone())?; + self.handle_log_inserts(log, ctx).await } diff --git a/src/frontend/src/server.rs b/src/frontend/src/server.rs index 9660b27620..115002c3ab 100644 --- a/src/frontend/src/server.rs +++ b/src/frontend/src/server.rs @@ -18,11 +18,13 @@ use std::sync::Arc; use auth::UserProviderRef; use common_base::Plugins; use common_config::{Configurable, Mode}; +use servers::error::Error as ServerError; use servers::grpc::builder::GrpcServerBuilder; use servers::grpc::greptime_handler::GreptimeRequestHandler; use servers::grpc::{GrpcOptions, GrpcServer, GrpcServerConfig}; use servers::http::event::LogValidatorRef; use servers::http::{HttpServer, HttpServerBuilder}; +use servers::interceptor::LogIngestInterceptorRef; use servers::metrics_handler::MetricsHandler; use servers::mysql::server::{MysqlServer, MysqlSpawnConfig, MysqlSpawnRef}; use servers::postgres::PostgresServer; @@ -81,8 +83,10 @@ where Some(self.instance.clone()), ); - builder = builder - .with_log_ingest_handler(self.instance.clone(), self.plugins.get::()); + let validator = self.plugins.get::(); + let ingest_interceptor = self.plugins.get::>(); + builder = + builder.with_log_ingest_handler(self.instance.clone(), validator, ingest_interceptor); if let Some(user_provider) = self.plugins.get::() { builder = builder.with_user_provider(user_provider); diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs index dca88d3843..f9925916fc 100644 --- a/src/pipeline/src/etl/processor/dissect.rs +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -15,7 +15,6 @@ use std::ops::Deref; use ahash::{HashMap, HashMapExt, HashSet, HashSetExt}; -use common_telemetry::warn; use itertools::Itertools; use crate::etl::field::{Fields, InputFieldInfo, OneInputMultiOutputField}; @@ -742,11 +741,8 @@ impl DissectProcessor { let chs = val.chars().collect::>(); for pattern in &self.patterns { - match self.process_pattern(&chs, pattern) { - Ok(map) => return Ok(map), - Err(e) => { - warn!("dissect processor: {}", e); - } + if let Ok(map) = self.process_pattern(&chs, pattern) { + return Ok(map); } } diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 956a650fcc..18388998e7 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -63,6 +63,7 @@ use crate::http::prometheus::{ build_info_query, format_query, instant_query, label_values_query, labels_query, range_query, series_query, }; +use crate::interceptor::LogIngestInterceptorRef; use crate::metrics::http_metrics_layer; use crate::metrics_handler::MetricsHandler; use crate::prometheus_handler::PrometheusHandlerRef; @@ -596,11 +597,16 @@ impl HttpServerBuilder { self, handler: LogHandlerRef, validator: Option, + ingest_interceptor: Option>, ) -> Self { Self { router: self.router.nest( &format!("/{HTTP_API_VERSION}/events"), - HttpServer::route_log(handler, validator), + HttpServer::route_log(LogState { + log_handler: handler, + log_validator: validator, + ingest_interceptor, + }), ), ..self } @@ -739,10 +745,7 @@ impl HttpServer { .with_state(metrics_handler) } - fn route_log( - log_handler: LogHandlerRef, - log_validator: Option, - ) -> Router { + fn route_log(log_state: LogState) -> Router { Router::new() .route("/logs", routing::post(event::log_ingester)) .route( @@ -759,10 +762,7 @@ impl HttpServer { .layer(HandleErrorLayer::new(handle_error)) .layer(RequestDecompressionLayer::new()), ) - .with_state(LogState { - log_handler, - log_validator, - }) + .with_state(log_state) } fn route_sql(api_state: ApiState) -> ApiRouter { diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index dbd7f1232a..f0a0902837 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -37,11 +37,13 @@ use session::context::{Channel, QueryContext, QueryContextRef}; use snafu::{ensure, OptionExt, ResultExt}; use crate::error::{ - InvalidParameterSnafu, ParseJsonSnafu, PipelineSnafu, Result, UnsupportedContentTypeSnafu, + Error, InvalidParameterSnafu, ParseJsonSnafu, PipelineSnafu, Result, + UnsupportedContentTypeSnafu, }; use crate::http::greptime_manage_resp::GreptimedbManageResponse; use crate::http::greptime_result_v1::GreptimedbV1Response; use crate::http::HttpResponse; +use crate::interceptor::{LogIngestInterceptor, LogIngestInterceptorRef}; use crate::metrics::{ METRIC_FAILURE_VALUE, METRIC_HTTP_LOGS_INGESTION_COUNTER, METRIC_HTTP_LOGS_INGESTION_ELAPSED, METRIC_HTTP_LOGS_TRANSFORM_ELAPSED, METRIC_SUCCESS_VALUE, @@ -378,6 +380,11 @@ pub async fn log_ingester( query_ctx.set_channel(Channel::Http); let query_ctx = Arc::new(query_ctx); + let value = log_state + .ingest_interceptor + .as_ref() + .pre_pipeline(value, query_ctx.clone())?; + ingest_logs_inner( handler, pipeline_name, @@ -506,6 +513,7 @@ pub type LogValidatorRef = Arc; pub struct LogState { pub log_handler: LogHandlerRef, pub log_validator: Option, + pub ingest_interceptor: Option>, } #[cfg(test)] diff --git a/src/servers/src/interceptor.rs b/src/servers/src/interceptor.rs index e4aceeb442..d3478a56ea 100644 --- a/src/servers/src/interceptor.rs +++ b/src/servers/src/interceptor.rs @@ -23,6 +23,7 @@ use common_error::ext::ErrorExt; use common_query::Output; use query::parser::PromQuery; use query::plan::LogicalPlan; +use serde_json::Value; use session::context::QueryContextRef; use sql::statements::statement::Statement; @@ -397,3 +398,61 @@ impl PromStoreProtocolInterceptor for Option, + _query_ctx: QueryContextRef, + ) -> Result, Self::Error> { + Ok(values) + } + + /// Called before insertion. + fn pre_ingest( + &self, + request: RowInsertRequests, + _query_ctx: QueryContextRef, + ) -> Result { + Ok(request) + } +} + +pub type LogIngestInterceptorRef = + Arc + Send + Sync + 'static>; + +impl LogIngestInterceptor for Option<&LogIngestInterceptorRef> +where + E: ErrorExt, +{ + type Error = E; + + fn pre_pipeline( + &self, + values: Vec, + query_ctx: QueryContextRef, + ) -> Result, Self::Error> { + if let Some(this) = self { + this.pre_pipeline(values, query_ctx) + } else { + Ok(values) + } + } + + fn pre_ingest( + &self, + request: RowInsertRequests, + query_ctx: QueryContextRef, + ) -> Result { + if let Some(this) = self { + this.pre_ingest(request, query_ctx) + } else { + Ok(request) + } + } +} diff --git a/tests-integration/src/test_util.rs b/tests-integration/src/test_util.rs index cf125a5776..a055527e2b 100644 --- a/tests-integration/src/test_util.rs +++ b/tests-integration/src/test_util.rs @@ -424,7 +424,7 @@ pub async fn setup_test_http_app_with_frontend_and_user_provider( ServerSqlQueryHandlerAdapter::arc(instance.instance.clone()), Some(instance.instance.clone()), ) - .with_log_ingest_handler(instance.instance.clone(), None) + .with_log_ingest_handler(instance.instance.clone(), None, None) .with_otlp_handler(instance.instance.clone()) .with_greptime_config_options(instance.opts.to_toml().unwrap()); From 8786624515b85cdcf238ae88f92b7cfbb80abab0 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Thu, 19 Sep 2024 13:30:56 +0800 Subject: [PATCH 016/128] feat: improve support for postgres extended protocol (#4721) * feat: improve support for postgres extended protocol * fix: lint fix * fix: test code * fix: adopt upstream * refactor: remove dup code * refactor: avoid copy on error message --- src/query/src/datafusion.rs | 18 +- src/servers/src/postgres/fixtures.rs | 22 +- src/servers/src/postgres/handler.rs | 49 ++- src/servers/src/postgres/types.rs | 471 ++++++++++++++------------- 4 files changed, 313 insertions(+), 247 deletions(-) diff --git a/src/query/src/datafusion.rs b/src/query/src/datafusion.rs index 03eadfde97..6ed5844de0 100644 --- a/src/query/src/datafusion.rs +++ b/src/query/src/datafusion.rs @@ -398,11 +398,19 @@ impl QueryEngine for DatafusionQueryEngine { query_ctx: QueryContextRef, ) -> Result { let ctx = self.engine_context(query_ctx); - let optimised_plan = self.optimize(&ctx, &plan)?; - Ok(DescribeResult { - schema: optimised_plan.schema()?, - logical_plan: optimised_plan, - }) + if let Ok(optimised_plan) = self.optimize(&ctx, &plan) { + Ok(DescribeResult { + schema: optimised_plan.schema()?, + logical_plan: optimised_plan, + }) + } else { + // Table's like those in information_schema cannot be optimized when + // it contains parameters. So we fallback to original plans. + Ok(DescribeResult { + schema: plan.schema()?, + logical_plan: plan, + }) + } } async fn execute(&self, plan: LogicalPlan, query_ctx: QueryContextRef) -> Result { diff --git a/src/servers/src/postgres/fixtures.rs b/src/servers/src/postgres/fixtures.rs index 5b02480da9..18c3661b93 100644 --- a/src/servers/src/postgres/fixtures.rs +++ b/src/servers/src/postgres/fixtures.rs @@ -54,17 +54,19 @@ static SET_TRANSACTION_PATTERN: Lazy = static TRANSACTION_PATTERN: Lazy = Lazy::new(|| Regex::new("(?i)^(BEGIN|ROLLBACK|COMMIT);?").unwrap()); +/// Test if given query statement matches the patterns +pub(crate) fn matches(query: &str) -> bool { + TRANSACTION_PATTERN.captures(query).is_some() + || SHOW_PATTERN.captures(query).is_some() + || SET_TRANSACTION_PATTERN.is_match(query) +} + /// Process unsupported SQL and return fixed result as a compatibility solution -pub(crate) fn process<'a>( - query: &str, - _query_ctx: QueryContextRef, -) -> Option>>> { +pub(crate) fn process<'a>(query: &str, _query_ctx: QueryContextRef) -> Option>> { // Transaction directives: if let Some(tx) = TRANSACTION_PATTERN.captures(query) { let tx_tag = &tx[1]; - Some(Ok(vec![Response::Execution(Tag::new( - &tx_tag.to_uppercase(), - ))])) + Some(vec![Response::Execution(Tag::new(&tx_tag.to_uppercase()))]) } else if let Some(show_var) = SHOW_PATTERN.captures(query) { let show_var = show_var[1].to_lowercase(); if let Some(value) = VAR_VALUES.get(&show_var.as_ref()) { @@ -81,12 +83,12 @@ pub(crate) fn process<'a>( vec![vec![value.to_string()]], )); - Some(Ok(vec![Response::Query(QueryResponse::new(schema, data))])) + Some(vec![Response::Query(QueryResponse::new(schema, data))]) } else { None } } else if SET_TRANSACTION_PATTERN.is_match(query) { - Some(Ok(vec![Response::Execution(Tag::new("SET"))])) + Some(vec![Response::Execution(Tag::new("SET"))]) } else { None } @@ -101,7 +103,6 @@ mod test { fn assert_tag(q: &str, t: &str, query_context: QueryContextRef) { if let Response::Execution(tag) = process(q, query_context.clone()) .unwrap_or_else(|| panic!("fail to match {}", q)) - .expect("unexpected error") .remove(0) { assert_eq!(Tag::new(t), tag); @@ -113,7 +114,6 @@ mod test { fn get_data<'a>(q: &str, query_context: QueryContextRef) -> QueryResponse<'a> { if let Response::Query(resp) = process(q, query_context.clone()) .unwrap_or_else(|| panic!("fail to match {}", q)) - .expect("unexpected error") .remove(0) { resp diff --git a/src/servers/src/postgres/handler.rs b/src/servers/src/postgres/handler.rs index 190684ed34..53d907d814 100644 --- a/src/servers/src/postgres/handler.rs +++ b/src/servers/src/postgres/handler.rs @@ -59,8 +59,13 @@ impl SimpleQueryHandler for PostgresServerHandler { .with_label_values(&[crate::metrics::METRIC_POSTGRES_SIMPLE_QUERY, db.as_str()]) .start_timer(); + if query.is_empty() { + // early return if query is empty + return Ok(vec![Response::EmptyQuery]); + } + if let Some(resps) = fixtures::process(query, query_ctx.clone()) { - resps + Ok(resps) } else { let outputs = self.query_handler.do_query(query, query_ctx.clone()).await; @@ -184,6 +189,16 @@ impl QueryParser for DefaultQueryParser { async fn parse_sql(&self, sql: &str, _types: &[Type]) -> PgWireResult { crate::metrics::METRIC_POSTGRES_PREPARED_COUNT.inc(); let query_ctx = self.session.new_query_context(); + + // do not parse if query is empty or matches rules + if sql.is_empty() || fixtures::matches(sql) { + return Ok(SqlPlan { + query: sql.to_owned(), + plan: None, + schema: None, + }); + } + let mut stmts = ParserContext::create_with_dialect(sql, &PostgreSqlDialect {}, ParseOptions::default()) .map_err(|e| PgWireError::ApiError(Box::new(e)))?; @@ -193,6 +208,7 @@ impl QueryParser for DefaultQueryParser { )))) } else { let stmt = stmts.remove(0); + let describe_result = self .query_handler .do_describe(stmt, query_ctx) @@ -244,6 +260,16 @@ impl ExtendedQueryHandler for PostgresServerHandler { let sql_plan = &portal.statement.statement; + if sql_plan.query.is_empty() { + // early return if query is empty + return Ok(Response::EmptyQuery); + } + + if let Some(mut resps) = fixtures::process(&sql_plan.query, query_ctx.clone()) { + // if the statement matches our predefined rules, return it early + return Ok(resps.remove(0)); + } + let output = if let Some(plan) = &sql_plan.plan { let plan = plan .replace_params_with_values(parameters_to_scalar_values(plan, portal)?.as_ref()) @@ -297,6 +323,17 @@ impl ExtendedQueryHandler for PostgresServerHandler { .map(|fields| DescribeStatementResponse::new(param_types, fields)) .map_err(|e| PgWireError::ApiError(Box::new(e))) } else { + if let Some(mut resp) = + fixtures::process(&sql_plan.query, self.session.new_query_context()) + { + if let Response::Query(query_response) = resp.remove(0) { + return Ok(DescribeStatementResponse::new( + param_types, + (*query_response.row_schema()).clone(), + )); + } + } + Ok(DescribeStatementResponse::new(param_types, vec![])) } } @@ -317,6 +354,16 @@ impl ExtendedQueryHandler for PostgresServerHandler { .map(DescribePortalResponse::new) .map_err(|e| PgWireError::ApiError(Box::new(e))) } else { + if let Some(mut resp) = + fixtures::process(&sql_plan.query, self.session.new_query_context()) + { + if let Response::Query(query_response) = resp.remove(0) { + return Ok(DescribePortalResponse::new( + (*query_response.row_schema()).clone(), + )); + } + } + Ok(DescribePortalResponse::new(vec![])) } } diff --git a/src/servers/src/postgres/types.rs b/src/servers/src/postgres/types.rs index 2bec6c2999..9f9d94905e 100644 --- a/src/servers/src/postgres/types.rs +++ b/src/servers/src/postgres/types.rs @@ -239,14 +239,14 @@ pub(super) fn parameter_to_string(portal: &Portal, idx: usize) -> PgWir .unwrap_or_else(|| "".to_owned())), _ => Err(invalid_parameter_error( "unsupported_parameter_type", - Some(¶m_type.to_string()), + Some(param_type.to_string()), )), } } -pub(super) fn invalid_parameter_error(msg: &str, detail: Option<&str>) -> PgWireError { +pub(super) fn invalid_parameter_error(msg: &str, detail: Option) -> PgWireError { let mut error_info = PgErrorCode::Ec22023.to_err_info(msg.to_string()); - error_info.detail = detail.map(|s| s.to_owned()); + error_info.detail = detail; PgWireError::UserError(Box::new(error_info)) } @@ -279,303 +279,314 @@ pub(super) fn parameters_to_scalar_values( .get_param_types() .map_err(|e| PgWireError::ApiError(Box::new(e)))?; - // ensure parameter count consistent for: client parameter types, server - // parameter types and parameter count - if param_types.len() != param_count { - return Err(invalid_parameter_error( - "invalid_parameter_count", - Some(&format!( - "Expected: {}, found: {}", - param_types.len(), - param_count - )), - )); - } - for idx in 0..param_count { - let server_type = - if let Some(Some(server_infer_type)) = param_types.get(&format!("${}", idx + 1)) { - server_infer_type - } else { - // at the moment we require type information inferenced by - // server so here we return error if the type is unknown from - // server-side. - // - // It might be possible to parse the parameter just using client - // specified type, we will implement that if there is a case. - return Err(invalid_parameter_error("unknown_parameter_type", None)); - }; + let server_type = param_types + .get(&format!("${}", idx + 1)) + .and_then(|t| t.as_ref()); let client_type = if let Some(client_given_type) = client_param_types.get(idx) { client_given_type.clone() + } else if let Some(server_provided_type) = &server_type { + type_gt_to_pg(server_provided_type).map_err(|e| PgWireError::ApiError(Box::new(e)))? } else { - type_gt_to_pg(server_type).map_err(|e| PgWireError::ApiError(Box::new(e)))? + return Err(invalid_parameter_error( + "unknown_parameter_type", + Some(format!( + "Cannot get parameter type information for parameter {}", + idx + )), + )); }; let value = match &client_type { &Type::VARCHAR | &Type::TEXT => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::String(_) => ScalarValue::Utf8(data), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::String(_) => ScalarValue::Utf8(data), + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::Utf8(data) } } &Type::BOOL => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Boolean(_) => ScalarValue::Boolean(data), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Boolean(_) => ScalarValue::Boolean(data), + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::Boolean(data) } } &Type::INT2 => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), - ConcreteDataType::Int16(_) => ScalarValue::Int16(data), - ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), - ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), - ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), - ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), - ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), - ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), - ConcreteDataType::Timestamp(unit) => { - to_timestamp_scalar_value(data, unit, server_type)? - } - ConcreteDataType::DateTime(_) => ScalarValue::Date64(data.map(|d| d as i64)), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), + ConcreteDataType::Int16(_) => ScalarValue::Int16(data), + ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), + ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), + ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), + ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), + ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), + ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), + ConcreteDataType::Timestamp(unit) => { + to_timestamp_scalar_value(data, unit, server_type)? + } + ConcreteDataType::DateTime(_) => { + ScalarValue::Date64(data.map(|d| d as i64)) + } + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::Int16(data) } } &Type::INT4 => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), - ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), - ConcreteDataType::Int32(_) => ScalarValue::Int32(data), - ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), - ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), - ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), - ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), - ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), - ConcreteDataType::Timestamp(unit) => { - to_timestamp_scalar_value(data, unit, server_type)? - } - ConcreteDataType::DateTime(_) => ScalarValue::Date64(data.map(|d| d as i64)), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), + ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), + ConcreteDataType::Int32(_) => ScalarValue::Int32(data), + ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), + ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), + ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), + ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), + ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), + ConcreteDataType::Timestamp(unit) => { + to_timestamp_scalar_value(data, unit, server_type)? + } + ConcreteDataType::DateTime(_) => { + ScalarValue::Date64(data.map(|d| d as i64)) + } + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::Int32(data) } } &Type::INT8 => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), - ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), - ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), - ConcreteDataType::Int64(_) => ScalarValue::Int64(data), - ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), - ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), - ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), - ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), - ConcreteDataType::Timestamp(unit) => { - to_timestamp_scalar_value(data, unit, server_type)? - } - ConcreteDataType::DateTime(_) => ScalarValue::Date64(data), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), + ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), + ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), + ConcreteDataType::Int64(_) => ScalarValue::Int64(data), + ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), + ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), + ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), + ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), + ConcreteDataType::Timestamp(unit) => { + to_timestamp_scalar_value(data, unit, server_type)? + } + ConcreteDataType::DateTime(_) => ScalarValue::Date64(data), + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::Int64(data) } } &Type::FLOAT4 => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), - ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), - ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), - ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), - ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), - ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), - ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), - ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), - ConcreteDataType::Float32(_) => ScalarValue::Float32(data), - ConcreteDataType::Float64(_) => ScalarValue::Float64(data.map(|n| n as f64)), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), + ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), + ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), + ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), + ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), + ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), + ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), + ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), + ConcreteDataType::Float32(_) => ScalarValue::Float32(data), + ConcreteDataType::Float64(_) => { + ScalarValue::Float64(data.map(|n| n as f64)) + } + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::Float32(data) } } &Type::FLOAT8 => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), - ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), - ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), - ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), - ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), - ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), - ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), - ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), - ConcreteDataType::Float32(_) => ScalarValue::Float32(data.map(|n| n as f32)), - ConcreteDataType::Float64(_) => ScalarValue::Float64(data), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Int8(_) => ScalarValue::Int8(data.map(|n| n as i8)), + ConcreteDataType::Int16(_) => ScalarValue::Int16(data.map(|n| n as i16)), + ConcreteDataType::Int32(_) => ScalarValue::Int32(data.map(|n| n as i32)), + ConcreteDataType::Int64(_) => ScalarValue::Int64(data.map(|n| n as i64)), + ConcreteDataType::UInt8(_) => ScalarValue::UInt8(data.map(|n| n as u8)), + ConcreteDataType::UInt16(_) => ScalarValue::UInt16(data.map(|n| n as u16)), + ConcreteDataType::UInt32(_) => ScalarValue::UInt32(data.map(|n| n as u32)), + ConcreteDataType::UInt64(_) => ScalarValue::UInt64(data.map(|n| n as u64)), + ConcreteDataType::Float32(_) => { + ScalarValue::Float32(data.map(|n| n as f32)) + } + ConcreteDataType::Float64(_) => ScalarValue::Float64(data), + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::Float64(data) } } &Type::TIMESTAMP => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Timestamp(unit) => match *unit { - TimestampType::Second(_) => ScalarValue::TimestampSecond( - data.map(|ts| ts.and_utc().timestamp()), - None, - ), - TimestampType::Millisecond(_) => ScalarValue::TimestampMillisecond( - data.map(|ts| ts.and_utc().timestamp_millis()), - None, - ), - TimestampType::Microsecond(_) => ScalarValue::TimestampMicrosecond( - data.map(|ts| ts.and_utc().timestamp_micros()), - None, - ), - TimestampType::Nanosecond(_) => ScalarValue::TimestampNanosecond( - data.map(|ts| ts.and_utc().timestamp_micros()), - None, - ), - }, - ConcreteDataType::DateTime(_) => { - ScalarValue::Date64(data.map(|d| d.and_utc().timestamp_millis())) - } - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )) + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Timestamp(unit) => match *unit { + TimestampType::Second(_) => ScalarValue::TimestampSecond( + data.map(|ts| ts.and_utc().timestamp()), + None, + ), + TimestampType::Millisecond(_) => ScalarValue::TimestampMillisecond( + data.map(|ts| ts.and_utc().timestamp_millis()), + None, + ), + TimestampType::Microsecond(_) => ScalarValue::TimestampMicrosecond( + data.map(|ts| ts.and_utc().timestamp_micros()), + None, + ), + TimestampType::Nanosecond(_) => ScalarValue::TimestampNanosecond( + data.map(|ts| ts.and_utc().timestamp_micros()), + None, + ), + }, + ConcreteDataType::DateTime(_) => { + ScalarValue::Date64(data.map(|d| d.and_utc().timestamp_millis())) + } + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )) + } } + } else { + ScalarValue::TimestampMillisecond( + data.map(|ts| ts.and_utc().timestamp_millis()), + None, + ) } } &Type::DATE => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Date(_) => ScalarValue::Date32(data.map(|d| { - (d - NaiveDate::from_ymd_opt(1970, 1, 1).unwrap()).num_days() as i32 - })), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )); + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Date(_) => ScalarValue::Date32(data.map(|d| { + (d - NaiveDate::from(NaiveDateTime::UNIX_EPOCH)).num_days() as i32 + })), + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )); + } } + } else { + ScalarValue::Date32(data.map(|d| { + (d - NaiveDate::from(NaiveDateTime::UNIX_EPOCH)).num_days() as i32 + })) } } &Type::INTERVAL => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Interval(_) => { - ScalarValue::IntervalMonthDayNano(data.map(|i| Interval::from(i).to_i128())) - } - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )); + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Interval(_) => ScalarValue::IntervalMonthDayNano( + data.map(|i| Interval::from(i).to_i128()), + ), + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )); + } } + } else { + ScalarValue::IntervalMonthDayNano(data.map(|i| Interval::from(i).to_i128())) } } &Type::BYTEA => { let data = portal.parameter::>(idx, &client_type)?; - match server_type { - ConcreteDataType::String(_) => { - ScalarValue::Utf8(data.map(|d| String::from_utf8_lossy(&d).to_string())) - } - ConcreteDataType::Binary(_) => ScalarValue::Binary(data), - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )); + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::String(_) => { + ScalarValue::Utf8(data.map(|d| String::from_utf8_lossy(&d).to_string())) + } + ConcreteDataType::Binary(_) => ScalarValue::Binary(data), + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )); + } } + } else { + ScalarValue::Binary(data) } } &Type::JSONB => { let data = portal.parameter::(idx, &client_type)?; - match server_type { - ConcreteDataType::Binary(_) => { - ScalarValue::Binary(data.map(|d| jsonb::Value::from(d).to_vec())) - } - _ => { - return Err(invalid_parameter_error( - "invalid_parameter_type", - Some(&format!( - "Expected: {}, found: {}", - server_type, client_type - )), - )); + if let Some(server_type) = &server_type { + match server_type { + ConcreteDataType::Binary(_) => { + ScalarValue::Binary(data.map(|d| jsonb::Value::from(d).to_vec())) + } + _ => { + return Err(invalid_parameter_error( + "invalid_parameter_type", + Some(format!("Expected: {}, found: {}", server_type, client_type)), + )); + } } + } else { + ScalarValue::Binary(data.map(|d| jsonb::Value::from(d).to_vec())) } } _ => Err(invalid_parameter_error( "unsupported_parameter_value", - Some(&format!("Found type: {}", client_type)), + Some(format!("Found type: {}", client_type)), ))?, }; From 0295f8dbea58b2285762ace1741b71ee5b2bc2b0 Mon Sep 17 00:00:00 2001 From: Yohan Wal <59358312+CookiePieWw@users.noreply.github.com> Date: Thu, 19 Sep 2024 13:49:10 +0800 Subject: [PATCH 017/128] docs: json datatype rfc (#4515) * docs: json datatype rfc * docs: turn to a jsonb proposal * chore: fix typo * feat: add store and query process * fix: typo * fix: use query nodes instead of query plans * feat: a detailed overview of query * fix: grammar * fix: use independent cast function * fix: unify cast function * fix: refine, make statements clear * docs: update rfc according to impl * docs: refine * docs: fix wrong arrows * docs: refine * docs: fix some errors qaq --- docs/rfcs/2024-08-06-json-datatype.md | 197 ++++++++++++++++++++++++++ 1 file changed, 197 insertions(+) create mode 100644 docs/rfcs/2024-08-06-json-datatype.md diff --git a/docs/rfcs/2024-08-06-json-datatype.md b/docs/rfcs/2024-08-06-json-datatype.md new file mode 100644 index 0000000000..8a617a1709 --- /dev/null +++ b/docs/rfcs/2024-08-06-json-datatype.md @@ -0,0 +1,197 @@ +--- +Feature Name: Json Datatype +Tracking Issue: https://github.com/GreptimeTeam/greptimedb/issues/4230 +Date: 2024-8-6 +Author: "Yuhan Wang " +--- + +# Summary +This RFC proposes a method for storing and querying JSON data in the database. + +# Motivation +JSON is widely used across various scenarios. Direct support for writing and querying JSON can significantly enhance the database's flexibility. + +# Details + +## Storage and Query + +GreptimeDB's type system is built on Arrow/DataFusion, where each data type in GreptimeDB corresponds to a data type in Arrow/DataFusion. The proposed JSON type will be implemented on top of the existing `Binary` type, leveraging the current `datatype::value::Value` and `datatype::vectors::BinaryVector` implementations, utilizing the JSONB format as the encoding of JSON data. JSON data is stored and processed similarly to binary data within the storage layer and query engine. + +This approach brings problems when dealing with insertions and queries of JSON columns. + +## Insertion + +Users commonly write JSON data as strings. Thus we need to make conversions between string and JSONB. There are 2 ways to do this: + +1. MySQL and PostgreSQL servers provide auto-conversions between strings and JSONB. When a string is inserted into a JSON column, the server will try to parse the string as JSON and convert it to JSONB. The non-JSON strings will be rejected. + +2. A function `parse_json` is provided to convert string to JSONB. If the string is not a valid JSON string, the function will return an error. + +For example, in MySQL client: +```SQL +CREATE TABLE IF NOT EXISTS test ( + ts TIMESTAMP TIME INDEX, + a INT, + b JSON +); + +INSERT INTO test VALUES( + 0, + 0, + '{ + "name": "jHl2oDDnPc1i2OzlP5Y", + "timestamp": "2024-07-25T04:33:11.369386Z", + "attributes": { "event_attributes": 48.28667 } + }' +); + +INSERT INTO test VALUES( + 0, + 0, + parse_json('{ + "name": "jHl2oDDnPc1i2OzlP5Y", + "timestamp": "2024-07-25T04:33:11.369386Z", + "attributes": { "event_attributes": 48.28667 } + }') +); +``` +Are both valid. + +The dataflow of the insertion process is as follows: +``` +Insert JSON strings directly through client: + Parse Insert + String(Serialized JSON)┌──────────┐Arrow Binary(JSONB)┌──────┐Arrow Binary(JSONB) + Client ---------------------->│ Server │------------------>│ Mito │------------------> Storage + └──────────┘ └──────┘ + (Server identifies JSON type and performs auto-conversion) + +Insert JSON strings through parse_json function: + Parse Insert + String(Serialized JSON)┌──────────┐String(Serialized JSON)┌─────┐Arrow Binary(JSONB)┌──────┐Arrow Binary(JSONB) + Client ---------------------->│ Server │---------------------->│ UDF │------------------>│ Mito │------------------> Storage + └──────────┘ └─────┘ └──────┘ + (Conversion is performed by UDF inside Query Engine) +``` + +Servers identify JSON column through column schema and perform auto-conversions. But when using prepared statements and binding parameters, the corresponding cached plans in datafusion generated by prepared statements cannot identify JSON columns. Under this circumstance, the servers identify JSON columns through the given parameters and perform auto-conversions. + +The following is an example of inserting JSON data through prepared statements: +```Rust +sqlx::query( + "create table test(ts timestamp time index, j json)", +) +.execute(&pool) +.await +.unwrap(); + +let json = serde_json::json!({ + "code": 200, + "success": true, + "payload": { + "features": [ + "serde", + "json" + ], + "homepage": null + } +}); + +// Valid, can identify serde_json::Value as JSON type +sqlx::query("insert into test values($1, $2)") + .bind(i) + .bind(json) + .execute(&pool) + .await + .unwrap(); + +// Invalid, cannot identify String as JSON type +sqlx::query("insert into test values($1, $2)") + .bind(i) + .bind(json.to_string()) + .execute(&pool) + .await + .unwrap(); +``` + +## Query + +Correspondingly, users prefer to display JSON data as strings. Thus we need to make conversions between JSON data and strings before presenting JSON data. There are also 2 ways to do this: auto-conversions on MySQL and PostgreSQL servers, and function `json_to_string`. + +For example, in MySQL client: +```SQL +SELECT b FROM test; + +SELECT json_to_string(b) FROM test; +``` +Will both return the JSON as human-readable strings. + +Specifically, to perform auto-conversions, we attach a message to JSON data in the `metadata` of `Field` in Arrow/Datafusion schema when scanning a JSON column. Frontend servers could identify JSON data and convert it to strings. + +The dataflow of the query process is as follows: +``` +Query directly through client: + Decode Scan + String(Serialized JSON)┌──────────┐Arrow Binary(JSONB)┌──────────────┐Arrow Binary(JSONB) + Client <----------------------│ Server │<------------------│ Query Engine │<----------------- Storage + └──────────┘ └──────────────┘ +(Server identifies JSON type and performs auto-conversion based on column metadata) + +Query through json_to_string function: + Scan & Decode + String(Serialized JSON)┌──────────┐String(Serialized JSON)┌──────────────┐Arrow Binary(JSONB) + Client <----------------------│ Server │<----------------------│ Query Engine │<----------------- Storage + └──────────┘ └──────────────┘ + (Conversion is performed by UDF inside Query Engine) + +``` + +However, if a function uses JSON type as its return type, the metadata method mentioned above is not applicable. Thus the functions of JSON type should specify the return type explicitly instead of returning a JSON type, such as `json_get_int` and `json_get_float` which return corresponding data of `INT` and `FLOAT` type respectively. + +## Functions +Similar to the common JSON type, JSON data can be queried with functions. + +For example: +```SQL +CREATE TABLE IF NOT EXISTS test ( + ts TIMESTAMP TIME INDEX, + a INT, + b JSON +); + +INSERT INTO test VALUES( + 0, + 0, + '{ + "name": "jHl2oDDnPc1i2OzlP5Y", + "timestamp": "2024-07-25T04:33:11.369386Z", + "attributes": { "event_attributes": 48.28667 } + }' +); + +SELECT json_get_string(b, 'name') FROM test; ++---------------------+ +| b.name | ++---------------------+ +| jHl2oDDnPc1i2OzlP5Y | ++---------------------+ + +SELECT json_get_float(b, 'attributes.event_attributes') FROM test; ++--------------------------------+ +| b.attributes.event_attributes | ++--------------------------------+ +| 48.28667 | ++--------------------------------+ + +``` +And more functions can be added in the future. + +# Drawbacks + +As a general purpose JSON data type, JSONB may not be as efficient as specialized data types for specific scenarios. + +The auto-conversion mechanism is not supported in all scenarios. We need to find workarounds for these scenarios. + +# Alternatives + +Extract and flatten JSON schema to store in a structured format through pipeline. For nested data, we can provide nested types like `STRUCT` or `ARRAY`. From d0f5b2ad7d0cb8feea8bc303fe3ff63ba8a550bf Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Thu, 19 Sep 2024 17:34:06 +0800 Subject: [PATCH 018/128] fix: use gcc-10 in release dev build (#4741) --- docker/dev-builder/ubuntu/Dockerfile | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/dev-builder/ubuntu/Dockerfile b/docker/dev-builder/ubuntu/Dockerfile index 3f76d80eff..d70824e911 100644 --- a/docker/dev-builder/ubuntu/Dockerfile +++ b/docker/dev-builder/ubuntu/Dockerfile @@ -24,6 +24,10 @@ RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y \ python3.10 \ python3.10-dev +# https://github.com/GreptimeTeam/greptimedb/actions/runs/10935485852/job/30357457188#step:3:7106 +# `aws-lc-sys` require gcc >= 10.3.0 to work, hence alias to use gcc-10 +RUN alias gcc="gcc-10" && alias cc="gcc-10" + # Remove Python 3.8 and install pip. RUN apt-get -y purge python3.8 && \ apt-get -y autoremove && \ From 49004391d348af785aa910691e48d2037b621ffa Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Thu, 19 Sep 2024 18:40:10 +0900 Subject: [PATCH 019/128] chore(fuzz): print table name for debugging (#4738) * chore(fuzz): print table name for debugging * chore: apply suggestions --- .../targets/failover/fuzz_failover_metric_regions.rs | 7 ++++++- tests-fuzz/targets/failover/fuzz_failover_mito_regions.rs | 6 +++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/tests-fuzz/targets/failover/fuzz_failover_metric_regions.rs b/tests-fuzz/targets/failover/fuzz_failover_metric_regions.rs index 5eaf43ab0a..147c3ead1e 100644 --- a/tests-fuzz/targets/failover/fuzz_failover_metric_regions.rs +++ b/tests-fuzz/targets/failover/fuzz_failover_metric_regions.rs @@ -257,7 +257,12 @@ async fn execute_failover(ctx: FuzzContext, input: FuzzInput) -> Result<()> { for (table_ctx, insert_expr) in tables.values() { let sql = format!("select count(1) as count from {}", table_ctx.name); let values = count_values(&ctx.greptime, &sql).await?; - assert_eq!(values.count as usize, insert_expr.values_list.len()); + let expected_rows = insert_expr.values_list.len() as u64; + assert_eq!( + values.count as u64, expected_rows, + "Expected rows: {}, got: {}, table: {}", + expected_rows, values.count, table_ctx.name + ); } // Clean up diff --git a/tests-fuzz/targets/failover/fuzz_failover_mito_regions.rs b/tests-fuzz/targets/failover/fuzz_failover_mito_regions.rs index 5fab5f2603..f456550b3f 100644 --- a/tests-fuzz/targets/failover/fuzz_failover_mito_regions.rs +++ b/tests-fuzz/targets/failover/fuzz_failover_mito_regions.rs @@ -323,7 +323,11 @@ async fn execute_failover(ctx: FuzzContext, input: FuzzInput) -> Result<()> { for (table_ctx, expected_rows) in table_ctxs.iter().zip(affected_rows) { let sql = format!("select count(1) as count from {}", table_ctx.name); let values = count_values(&ctx.greptime, &sql).await?; - assert_eq!(values.count as u64, expected_rows); + assert_eq!( + values.count as u64, expected_rows, + "Expected rows: {}, got: {}, table: {}", + expected_rows, values.count, table_ctx.name + ); } for table_ctx in table_ctxs { From 95787825f1f6c09839271a5e6cc6d11b062bbaad Mon Sep 17 00:00:00 2001 From: Yohan Wal <59358312+CookiePieWw@users.noreply.github.com> Date: Thu, 19 Sep 2024 17:44:44 +0800 Subject: [PATCH 020/128] build(deps): use original jsonb repo (#4742) --- Cargo.lock | 2 +- Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 786c1c3a8b..88c7f7f080 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5481,7 +5481,7 @@ dependencies = [ [[package]] name = "jsonb" version = "0.4.1" -source = "git+https://github.com/CookiePieWw/jsonb.git?rev=d0166c130fce903bf6c58643417a3173a6172d31#d0166c130fce903bf6c58643417a3173a6172d31" +source = "git+https://github.com/datafuselabs/jsonb.git?rev=46ad50fc71cf75afbf98eec455f7892a6387c1fc#46ad50fc71cf75afbf98eec455f7892a6387c1fc" dependencies = [ "byteorder", "fast-float", diff --git a/Cargo.toml b/Cargo.toml index c1eea12a53..25cd5b5dd2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -124,7 +124,7 @@ greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", r humantime = "2.1" humantime-serde = "1.1" itertools = "0.10" -jsonb = { git = "https://github.com/CookiePieWw/jsonb.git", rev = "d0166c130fce903bf6c58643417a3173a6172d31", default-features = false } +jsonb = { git = "https://github.com/datafuselabs/jsonb.git", rev = "46ad50fc71cf75afbf98eec455f7892a6387c1fc", default-features = false } lazy_static = "1.4" meter-core = { git = "https://github.com/GreptimeTeam/greptime-meter.git", rev = "80eb97c24c88af4dd9a86f8bbaf50e741d4eb8cd" } mockall = "0.11.4" From 1acda74c260dc989ba6c2aa010782aa6cf67cafb Mon Sep 17 00:00:00 2001 From: liyang Date: Thu, 19 Sep 2024 19:14:34 +0800 Subject: [PATCH 021/128] fix: cannot input tag for the dev-builder image (#4743) --- .github/actions/build-dev-builder-images/action.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/actions/build-dev-builder-images/action.yml b/.github/actions/build-dev-builder-images/action.yml index 6bb14e14f4..9c30caad35 100644 --- a/.github/actions/build-dev-builder-images/action.yml +++ b/.github/actions/build-dev-builder-images/action.yml @@ -50,7 +50,7 @@ runs: BUILDX_MULTI_PLATFORM_BUILD=all \ IMAGE_REGISTRY=${{ inputs.dockerhub-image-registry }} \ IMAGE_NAMESPACE=${{ inputs.dockerhub-image-namespace }} \ - IMAGE_TAG=${{ inputs.version }} + DEV_BUILDER_IMAGE_TAG=${{ inputs.version }} - name: Build and push dev-builder-centos image shell: bash @@ -61,7 +61,7 @@ runs: BUILDX_MULTI_PLATFORM_BUILD=amd64 \ IMAGE_REGISTRY=${{ inputs.dockerhub-image-registry }} \ IMAGE_NAMESPACE=${{ inputs.dockerhub-image-namespace }} \ - IMAGE_TAG=${{ inputs.version }} + DEV_BUILDER_IMAGE_TAG=${{ inputs.version }} - name: Build and push dev-builder-android image # Only build image for amd64 platform. shell: bash @@ -71,6 +71,6 @@ runs: BASE_IMAGE=android \ IMAGE_REGISTRY=${{ inputs.dockerhub-image-registry }} \ IMAGE_NAMESPACE=${{ inputs.dockerhub-image-namespace }} \ - IMAGE_TAG=${{ inputs.version }} && \ + DEV_BUILDER_IMAGE_TAG=${{ inputs.version }} && \ docker push ${{ inputs.dockerhub-image-registry }}/${{ inputs.dockerhub-image-namespace }}/dev-builder-android:${{ inputs.version }} From f5cf25b0dbe588e78f2c03a7cb854aeed176067e Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Thu, 19 Sep 2024 20:29:33 +0800 Subject: [PATCH 022/128] refactor: remove DfPlan wrapper (#4733) * refactor: remove DfPlan wrapper Signed-off-by: Ruihang Xia * clean up Signed-off-by: Ruihang Xia * remove unused errors Signed-off-by: Ruihang Xia * fix test assertion Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- Cargo.lock | 12 +- src/cmd/src/cli/repl.rs | 3 +- src/datanode/src/region_server.rs | 2 +- src/datanode/src/tests.rs | 2 +- src/flow/src/df_optimizer.rs | 2 - src/flow/src/transform.rs | 3 - src/frontend/Cargo.toml | 1 + src/frontend/src/instance.rs | 2 +- src/operator/src/error.rs | 16 +- src/operator/src/statement.rs | 2 +- src/operator/src/statement/copy_table_to.rs | 3 +- src/operator/src/statement/ddl.rs | 24 +-- src/operator/src/statement/tql.rs | 2 +- src/pipeline/src/manager/table.rs | 7 +- src/query/src/datafusion.rs | 154 ++++++++++---------- src/query/src/error.rs | 9 +- src/query/src/plan.rs | 95 +----------- src/query/src/planner.rs | 8 +- src/query/src/query_engine.rs | 2 +- src/query/src/range_select/plan_rewrite.rs | 5 +- src/query/src/tests/query_engine_test.rs | 23 ++- src/script/src/table.rs | 5 +- src/servers/Cargo.toml | 1 + src/servers/src/error.rs | 18 +-- src/servers/src/http.rs | 2 +- src/servers/src/interceptor.rs | 2 +- src/servers/src/lib.rs | 2 +- src/servers/src/mysql/handler.rs | 36 +++-- src/servers/src/postgres/handler.rs | 14 +- src/servers/src/postgres/types.rs | 9 +- src/servers/src/prom_store.rs | 4 +- src/servers/src/query_handler/sql.rs | 2 +- src/servers/tests/http/influxdb_test.rs | 2 +- src/servers/tests/http/opentsdb_test.rs | 2 +- src/servers/tests/http/prom_store_test.rs | 2 +- src/servers/tests/mod.rs | 2 +- tests-integration/src/grpc.rs | 3 +- tests-integration/src/instance.rs | 6 +- 38 files changed, 195 insertions(+), 294 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 88c7f7f080..55e8ca8e4b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1014,7 +1014,7 @@ dependencies = [ "bitflags 2.5.0", "cexpr", "clang-sys", - "itertools 0.10.5", + "itertools 0.12.1", "lazy_static", "lazycell", "log", @@ -4019,6 +4019,7 @@ dependencies = [ "common-test-util", "common-time", "common-version", + "datafusion-expr", "datanode", "datatypes", "futures", @@ -4836,7 +4837,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.10", + "socket2 0.5.7", "tokio", "tower-service", "tracing", @@ -8439,7 +8440,7 @@ checksum = "22505a5c94da8e3b7c2996394d1c933236c4d743e81a410bcca4e6989fc066a4" dependencies = [ "bytes", "heck 0.5.0", - "itertools 0.10.5", + "itertools 0.12.1", "log", "multimap", "once_cell", @@ -8491,7 +8492,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" dependencies = [ "anyhow", - "itertools 0.10.5", + "itertools 0.12.1", "proc-macro2", "quote", "syn 2.0.66", @@ -8651,7 +8652,7 @@ dependencies = [ "indoc", "libc", "memoffset 0.9.1", - "parking_lot 0.11.2", + "parking_lot 0.12.3", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -10480,6 +10481,7 @@ dependencies = [ "dashmap", "datafusion", "datafusion-common", + "datafusion-expr", "datatypes", "derive_builder 0.12.0", "futures", diff --git a/src/cmd/src/cli/repl.rs b/src/cmd/src/cli/repl.rs index 6d7e211d7c..3fc2184d37 100644 --- a/src/cmd/src/cli/repl.rs +++ b/src/cmd/src/cli/repl.rs @@ -35,7 +35,6 @@ use either::Either; use meta_client::client::MetaClientBuilder; use query::datafusion::DatafusionQueryEngine; use query::parser::QueryLanguageParser; -use query::plan::LogicalPlan; use query::query_engine::{DefaultSerializer, QueryEngineState}; use query::QueryEngine; use rustyline::error::ReadlineError; @@ -179,7 +178,7 @@ impl Repl { .await .context(PlanStatementSnafu)?; - let LogicalPlan::DfPlan(plan) = query_engine + let plan = query_engine .optimize(&query_engine.engine_context(query_ctx), &plan) .context(PlanStatementSnafu)?; diff --git a/src/datanode/src/region_server.rs b/src/datanode/src/region_server.rs index 56068a38c3..54e1cdbafd 100644 --- a/src/datanode/src/region_server.rs +++ b/src/datanode/src/region_server.rs @@ -843,7 +843,7 @@ impl RegionServerInner { let result = self .query_engine - .execute(request.plan.into(), query_ctx) + .execute(request.plan, query_ctx) .await .context(ExecuteLogicalPlanSnafu)?; diff --git a/src/datanode/src/tests.rs b/src/datanode/src/tests.rs index 87285eabfa..89be76511d 100644 --- a/src/datanode/src/tests.rs +++ b/src/datanode/src/tests.rs @@ -24,8 +24,8 @@ use common_function::scalars::aggregate::AggregateFunctionMetaRef; use common_query::prelude::ScalarUdf; use common_query::Output; use common_runtime::Runtime; +use datafusion_expr::LogicalPlan; use query::dataframe::DataFrame; -use query::plan::LogicalPlan; use query::planner::LogicalPlanner; use query::query_engine::{DescribeResult, QueryEngineState}; use query::{QueryEngine, QueryEngineContext}; diff --git a/src/flow/src/df_optimizer.rs b/src/flow/src/df_optimizer.rs index d5368d5189..d73bf91bfb 100644 --- a/src/flow/src/df_optimizer.rs +++ b/src/flow/src/df_optimizer.rs @@ -41,7 +41,6 @@ use datafusion_expr::{ BinaryExpr, Expr, Operator, Projection, ScalarUDFImpl, Signature, TypeSignature, Volatility, }; use query::parser::QueryLanguageParser; -use query::plan::LogicalPlan; use query::query_engine::DefaultSerializer; use query::QueryEngine; use snafu::ResultExt; @@ -111,7 +110,6 @@ pub async fn sql_to_flow_plan( .await .map_err(BoxedError::new) .context(ExternalSnafu)?; - let LogicalPlan::DfPlan(plan) = plan; let opted_plan = apply_df_optimizer(plan).await?; diff --git a/src/flow/src/transform.rs b/src/flow/src/transform.rs index f6dff58856..6d4de97417 100644 --- a/src/flow/src/transform.rs +++ b/src/flow/src/transform.rs @@ -163,7 +163,6 @@ mod test { use itertools::Itertools; use prost::Message; use query::parser::QueryLanguageParser; - use query::plan::LogicalPlan; use query::query_engine::DefaultSerializer; use query::QueryEngine; use session::context::QueryContext; @@ -274,7 +273,6 @@ mod test { .plan(stmt, QueryContext::arc()) .await .unwrap(); - let LogicalPlan::DfPlan(plan) = plan; let plan = apply_df_optimizer(plan).await.unwrap(); // encode then decode so to rely on the impl of conversion from logical plan to substrait plan @@ -297,7 +295,6 @@ mod test { .plan(stmt, QueryContext::arc()) .await .unwrap(); - let LogicalPlan::DfPlan(plan) = plan; let plan = apply_df_optimizer(plan).await; assert!(plan.is_err()); diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 362d147d13..555a201280 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -37,6 +37,7 @@ common-runtime.workspace = true common-telemetry.workspace = true common-time.workspace = true common-version.workspace = true +datafusion-expr.workspace = true datanode.workspace = true humantime-serde.workspace = true lazy_static.workspace = true diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index 2c5544c51a..dcc56f7581 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -40,6 +40,7 @@ use common_procedure::options::ProcedureConfig; use common_procedure::ProcedureManagerRef; use common_query::Output; use common_telemetry::{debug, error, tracing}; +use datafusion_expr::LogicalPlan; use log_store::raft_engine::RaftEngineBackend; use operator::delete::DeleterRef; use operator::insert::InserterRef; @@ -48,7 +49,6 @@ use pipeline::pipeline_operator::PipelineOperator; use prometheus::HistogramTimer; use query::metrics::OnDone; use query::parser::{PromQuery, QueryLanguageParser, QueryStatement}; -use query::plan::LogicalPlan; use query::query_engine::options::{validate_catalog_and_schema, QueryOptions}; use query::query_engine::DescribeResult; use query::QueryEngineRef; diff --git a/src/operator/src/error.rs b/src/operator/src/error.rs index 02a880a903..1931814c8b 100644 --- a/src/operator/src/error.rs +++ b/src/operator/src/error.rs @@ -133,13 +133,6 @@ pub enum Error { source: query::error::Error, }, - #[snafu(display("Failed to get schema from logical plan"))] - GetSchema { - #[snafu(implicit)] - location: Location, - source: query::error::Error, - }, - #[snafu(display("Column datatype error"))] ColumnDataType { #[snafu(implicit)] @@ -184,6 +177,13 @@ pub enum Error { source: datatypes::error::Error, }, + #[snafu(display("Failed to convert datafusion schema"))] + ConvertSchema { + source: datatypes::error::Error, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Failed to convert expr to struct"))] InvalidExpr { #[snafu(implicit)] @@ -795,6 +795,7 @@ impl ErrorExt for Error { | Error::PrepareFileTable { .. } | Error::InferFileTableSchema { .. } | Error::SchemaIncompatible { .. } + | Error::ConvertSchema { .. } | Error::UnsupportedRegionRequest { .. } | Error::InvalidTableName { .. } | Error::InvalidViewName { .. } @@ -872,7 +873,6 @@ impl ErrorExt for Error { | Error::FindNewColumnsOnInsertion { source, .. } => source.status_code(), Error::ExecuteStatement { source, .. } - | Error::GetSchema { source, .. } | Error::ExtractTableNames { source, .. } | Error::PlanStatement { source, .. } | Error::ParseQuery { source, .. } diff --git a/src/operator/src/statement.rs b/src/operator/src/statement.rs index 35e6752d08..4dc43e0d92 100644 --- a/src/operator/src/statement.rs +++ b/src/operator/src/statement.rs @@ -42,9 +42,9 @@ use common_query::Output; use common_telemetry::tracing; use common_time::range::TimestampRange; use common_time::Timestamp; +use datafusion_expr::LogicalPlan; use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef}; use query::parser::QueryStatement; -use query::plan::LogicalPlan; use query::QueryEngineRef; use session::context::QueryContextRef; use session::table_name::table_idents_to_full_name; diff --git a/src/operator/src/statement/copy_table_to.rs b/src/operator/src/statement/copy_table_to.rs index 8a90d10955..0e53d79ac6 100644 --- a/src/operator/src/statement/copy_table_to.rs +++ b/src/operator/src/statement/copy_table_to.rs @@ -29,7 +29,6 @@ use datafusion::datasource::DefaultTableSource; use datafusion_common::TableReference as DfTableReference; use datafusion_expr::LogicalPlanBuilder; use object_store::ObjectStore; -use query::plan::LogicalPlan; use session::context::QueryContextRef; use snafu::{OptionExt, ResultExt}; use table::requests::CopyTableRequest; @@ -133,7 +132,7 @@ impl StatementExecutor { let output = self .query_engine - .execute(LogicalPlan::DfPlan(plan), query_ctx) + .execute(plan, query_ctx) .await .context(ExecLogicalPlanSnafu)?; let stream = match output.data { diff --git a/src/operator/src/statement/ddl.rs b/src/operator/src/statement/ddl.rs index aa1a070875..e9b186bd63 100644 --- a/src/operator/src/statement/ddl.rs +++ b/src/operator/src/statement/ddl.rs @@ -37,7 +37,7 @@ use common_query::Output; use common_telemetry::{debug, info, tracing}; use common_time::Timezone; use datatypes::prelude::ConcreteDataType; -use datatypes::schema::RawSchema; +use datatypes::schema::{RawSchema, Schema}; use datatypes::value::Value; use lazy_static::lazy_static; use partition::expr::{Operand, PartitionExpr, RestrictedOp}; @@ -69,11 +69,11 @@ use table::TableRef; use super::StatementExecutor; use crate::error::{ self, AlterExprToRequestSnafu, CatalogSnafu, ColumnDataTypeSnafu, ColumnNotFoundSnafu, - CreateLogicalTablesSnafu, CreateTableInfoSnafu, DeserializePartitionSnafu, EmptyDdlExprSnafu, - ExtractTableNamesSnafu, FlowNotFoundSnafu, InvalidPartitionRuleSnafu, InvalidPartitionSnafu, - InvalidTableNameSnafu, InvalidViewNameSnafu, InvalidViewStmtSnafu, ParseSqlValueSnafu, Result, - SchemaInUseSnafu, SchemaNotFoundSnafu, SchemaReadOnlySnafu, SubstraitCodecSnafu, - TableAlreadyExistsSnafu, TableMetadataManagerSnafu, TableNotFoundSnafu, + ConvertSchemaSnafu, CreateLogicalTablesSnafu, CreateTableInfoSnafu, DeserializePartitionSnafu, + EmptyDdlExprSnafu, ExtractTableNamesSnafu, FlowNotFoundSnafu, InvalidPartitionRuleSnafu, + InvalidPartitionSnafu, InvalidTableNameSnafu, InvalidViewNameSnafu, InvalidViewStmtSnafu, + ParseSqlValueSnafu, Result, SchemaInUseSnafu, SchemaNotFoundSnafu, SchemaReadOnlySnafu, + SubstraitCodecSnafu, TableAlreadyExistsSnafu, TableMetadataManagerSnafu, TableNotFoundSnafu, UnrecognizedTableOptionSnafu, ViewAlreadyExistsSnafu, }; use crate::expr_factory; @@ -406,9 +406,12 @@ impl StatementExecutor { // Save the columns in plan, it may changed when the schemas of tables in plan // are altered. - let plan_columns: Vec<_> = logical_plan + let schema: Schema = logical_plan .schema() - .context(error::GetSchemaSnafu)? + .clone() + .try_into() + .context(ConvertSchemaSnafu)?; + let plan_columns: Vec<_> = schema .column_schemas() .iter() .map(|c| c.name.clone()) @@ -434,9 +437,8 @@ impl StatementExecutor { // Extract the table names from the original plan // and rewrite them as fully qualified names. - let (table_names, plan) = - extract_and_rewrite_full_table_names(logical_plan.unwrap_df_plan(), ctx.clone()) - .context(ExtractTableNamesSnafu)?; + let (table_names, plan) = extract_and_rewrite_full_table_names(logical_plan, ctx.clone()) + .context(ExtractTableNamesSnafu)?; let table_names = table_names.into_iter().map(|t| t.into()).collect(); diff --git a/src/operator/src/statement/tql.rs b/src/operator/src/statement/tql.rs index 72b2db641b..008aba0d78 100644 --- a/src/operator/src/statement/tql.rs +++ b/src/operator/src/statement/tql.rs @@ -16,11 +16,11 @@ use std::collections::HashMap; use common_query::Output; use common_telemetry::tracing; +use datafusion_expr::LogicalPlan; use query::parser::{ PromQuery, QueryLanguageParser, ANALYZE_NODE_NAME, ANALYZE_VERBOSE_NODE_NAME, DEFAULT_LOOKBACK_STRING, EXPLAIN_NODE_NAME, EXPLAIN_VERBOSE_NODE_NAME, }; -use query::plan::LogicalPlan; use session::context::QueryContextRef; use snafu::ResultExt; use sql::statements::tql::Tql; diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 3c69f59f2a..8989412c3e 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -26,7 +26,7 @@ use common_telemetry::{debug, info}; use common_time::timestamp::{TimeUnit, Timestamp}; use datafusion::logical_expr::col; use datafusion_common::{TableReference, ToDFSchema}; -use datafusion_expr::{DmlStatement, LogicalPlan as DfLogicalPlan}; +use datafusion_expr::{DmlStatement, LogicalPlan}; use datatypes::prelude::ScalarVector; use datatypes::timestamp::TimestampNanosecond; use datatypes::vectors::{StringVector, TimestampNanosecondVector, Vector}; @@ -34,7 +34,6 @@ use moka::sync::Cache; use operator::insert::InserterRef; use operator::statement::StatementExecutorRef; use query::dataframe::DataFrame; -use query::plan::LogicalPlan; use query::QueryEngineRef; use session::context::{QueryContextBuilder, QueryContextRef}; use snafu::{ensure, OptionExt, ResultExt}; @@ -373,7 +372,7 @@ impl PipelineTable { Arc::new(dataframe.into_parts().1), ); - let plan = LogicalPlan::DfPlan(DfLogicalPlan::Dml(stmt)); + let plan = LogicalPlan::Dml(stmt); // 4. execute dml stmt let output = self @@ -427,7 +426,7 @@ impl PipelineTable { .limit(0, Some(1)) .context(BuildDfLogicalPlanSnafu)?; - let plan = LogicalPlan::DfPlan(dataframe.into_parts().1); + let plan = dataframe.into_parts().1; let table_info = self.table.table_info(); diff --git a/src/query/src/datafusion.rs b/src/query/src/datafusion.rs index 6ed5844de0..888ebbba83 100644 --- a/src/query/src/datafusion.rs +++ b/src/query/src/datafusion.rs @@ -36,7 +36,7 @@ use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::ExecutionPlan; use datafusion_common::ResolvedTableReference; -use datafusion_expr::{DmlStatement, LogicalPlan as DfLogicalPlan, WriteOp}; +use datafusion_expr::{DmlStatement, LogicalPlan as DfLogicalPlan, LogicalPlan, WriteOp}; use datatypes::prelude::VectorRef; use datatypes::schema::Schema; use futures_util::StreamExt; @@ -50,14 +50,13 @@ use crate::dataframe::DataFrame; pub use crate::datafusion::planner::DfContextProviderAdapter; use crate::dist_plan::MergeScanLogicalPlan; use crate::error::{ - CatalogSnafu, CreateRecordBatchSnafu, DataFusionSnafu, MissingTableMutationHandlerSnafu, - MissingTimestampColumnSnafu, QueryExecutionSnafu, Result, TableMutationSnafu, - TableNotFoundSnafu, TableReadOnlySnafu, UnsupportedExprSnafu, + CatalogSnafu, ConvertSchemaSnafu, CreateRecordBatchSnafu, DataFusionSnafu, + MissingTableMutationHandlerSnafu, MissingTimestampColumnSnafu, QueryExecutionSnafu, Result, + TableMutationSnafu, TableNotFoundSnafu, TableReadOnlySnafu, UnsupportedExprSnafu, }; use crate::executor::QueryExecutor; use crate::metrics::{OnDone, QUERY_STAGE_ELAPSED}; use crate::physical_wrapper::PhysicalPlanWrapperRef; -use crate::plan::LogicalPlan; use crate::planner::{DfLogicalPlanner, LogicalPlanner}; use crate::query_engine::{DescribeResult, QueryEngineContext, QueryEngineState}; use crate::{metrics, QueryEngine}; @@ -119,7 +118,7 @@ impl DatafusionQueryEngine { let table = self.find_table(&table_name, &query_ctx).await?; let output = self - .exec_query_plan(LogicalPlan::DfPlan((*dml.input).clone()), query_ctx.clone()) + .exec_query_plan((*dml.input).clone(), query_ctx.clone()) .await?; let mut stream = match output.data { OutputData::RecordBatches(batches) => batches.as_stream(), @@ -265,52 +264,48 @@ impl DatafusionQueryEngine { logical_plan: &LogicalPlan, ) -> Result> { let _timer = metrics::CREATE_PHYSICAL_ELAPSED.start_timer(); - match logical_plan { - LogicalPlan::DfPlan(df_plan) => { - let state = ctx.state(); + let state = ctx.state(); - // special handle EXPLAIN plan - if matches!(df_plan, DfLogicalPlan::Explain(_)) { - return state - .create_physical_plan(df_plan) - .await - .context(error::DatafusionSnafu) - .map_err(BoxedError::new) - .context(QueryExecutionSnafu); - } - - // analyze first - let analyzed_plan = state - .analyzer() - .execute_and_check(df_plan.clone(), state.config_options(), |_, _| {}) - .context(error::DatafusionSnafu) - .map_err(BoxedError::new) - .context(QueryExecutionSnafu)?; - // skip optimize for MergeScan - let optimized_plan = if let DfLogicalPlan::Extension(ext) = &analyzed_plan - && ext.node.name() == MergeScanLogicalPlan::name() - { - analyzed_plan.clone() - } else { - state - .optimizer() - .optimize(analyzed_plan, state, |_, _| {}) - .context(error::DatafusionSnafu) - .map_err(BoxedError::new) - .context(QueryExecutionSnafu)? - }; - - let physical_plan = state - .query_planner() - .create_physical_plan(&optimized_plan, state) - .await - .context(error::DatafusionSnafu) - .map_err(BoxedError::new) - .context(QueryExecutionSnafu)?; - - Ok(physical_plan) - } + // special handle EXPLAIN plan + if matches!(logical_plan, DfLogicalPlan::Explain(_)) { + return state + .create_physical_plan(logical_plan) + .await + .context(error::DatafusionSnafu) + .map_err(BoxedError::new) + .context(QueryExecutionSnafu); } + + // analyze first + let analyzed_plan = state + .analyzer() + .execute_and_check(logical_plan.clone(), state.config_options(), |_, _| {}) + .context(error::DatafusionSnafu) + .map_err(BoxedError::new) + .context(QueryExecutionSnafu)?; + // skip optimize for MergeScan + let optimized_plan = if let DfLogicalPlan::Extension(ext) = &analyzed_plan + && ext.node.name() == MergeScanLogicalPlan::name() + { + analyzed_plan.clone() + } else { + state + .optimizer() + .optimize(analyzed_plan, state, |_, _| {}) + .context(error::DatafusionSnafu) + .map_err(BoxedError::new) + .context(QueryExecutionSnafu)? + }; + + let physical_plan = state + .query_planner() + .create_physical_plan(&optimized_plan, state) + .await + .context(error::DatafusionSnafu) + .map_err(BoxedError::new) + .context(QueryExecutionSnafu)?; + + Ok(physical_plan) } #[tracing::instrument(skip_all)] @@ -320,28 +315,25 @@ impl DatafusionQueryEngine { plan: &LogicalPlan, ) -> Result { let _timer = metrics::OPTIMIZE_LOGICAL_ELAPSED.start_timer(); - match plan { - LogicalPlan::DfPlan(df_plan) => { - // Optimized by extension rules - let optimized_plan = self - .state - .optimize_by_extension_rules(df_plan.clone(), context) - .context(error::DatafusionSnafu) - .map_err(BoxedError::new) - .context(QueryExecutionSnafu)?; - // Optimized by datafusion optimizer - let optimized_plan = self - .state - .session_state() - .optimize(&optimized_plan) - .context(error::DatafusionSnafu) - .map_err(BoxedError::new) - .context(QueryExecutionSnafu)?; + // Optimized by extension rules + let optimized_plan = self + .state + .optimize_by_extension_rules(plan.clone(), context) + .context(error::DatafusionSnafu) + .map_err(BoxedError::new) + .context(QueryExecutionSnafu)?; - Ok(LogicalPlan::DfPlan(optimized_plan)) - } - } + // Optimized by datafusion optimizer + let optimized_plan = self + .state + .session_state() + .optimize(&optimized_plan) + .context(error::DatafusionSnafu) + .map_err(BoxedError::new) + .context(QueryExecutionSnafu)?; + + Ok(optimized_plan) } #[tracing::instrument(skip_all)] @@ -399,15 +391,25 @@ impl QueryEngine for DatafusionQueryEngine { ) -> Result { let ctx = self.engine_context(query_ctx); if let Ok(optimised_plan) = self.optimize(&ctx, &plan) { + let schema = optimised_plan + .schema() + .clone() + .try_into() + .context(ConvertSchemaSnafu)?; Ok(DescribeResult { - schema: optimised_plan.schema()?, + schema, logical_plan: optimised_plan, }) } else { // Table's like those in information_schema cannot be optimized when // it contains parameters. So we fallback to original plans. + let schema = plan + .schema() + .clone() + .try_into() + .context(ConvertSchemaSnafu)?; Ok(DescribeResult { - schema: plan.schema()?, + schema, logical_plan: plan, }) } @@ -415,9 +417,7 @@ impl QueryEngine for DatafusionQueryEngine { async fn execute(&self, plan: LogicalPlan, query_ctx: QueryContextRef) -> Result { match plan { - LogicalPlan::DfPlan(DfLogicalPlan::Dml(dml)) => { - self.exec_dml_statement(dml, query_ctx).await - } + LogicalPlan::Dml(dml) => self.exec_dml_statement(dml, query_ctx).await, _ => self.exec_query_plan(plan, query_ctx).await, } } @@ -577,10 +577,10 @@ mod tests { // TODO(sunng87): do not rely on to_string for compare assert_eq!( format!("{plan:?}"), - r#"DfPlan(Limit: skip=0, fetch=20 + r#"Limit: skip=0, fetch=20 Projection: SUM(numbers.number) Aggregate: groupBy=[[]], aggr=[[SUM(numbers.number)]] - TableScan: numbers)"# + TableScan: numbers"# ); } diff --git a/src/query/src/error.rs b/src/query/src/error.rs index 4eca47e175..7e246d11c3 100644 --- a/src/query/src/error.rs +++ b/src/query/src/error.rs @@ -108,13 +108,6 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to convert Datafusion schema"))] - ConvertDatafusionSchema { - source: datatypes::error::Error, - #[snafu(implicit)] - location: Location, - }, - #[snafu(display("Failed to parse timestamp `{}`", raw))] ParseTimestamp { raw: String, @@ -228,6 +221,7 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + #[snafu(display("Unknown table type, downcast failed"))] UnknownTable { #[snafu(implicit)] @@ -354,7 +348,6 @@ impl ErrorExt for Error { QueryAccessDenied { .. } => StatusCode::AccessDenied, Catalog { source, .. } => source.status_code(), - ConvertDatafusionSchema { source, .. } => source.status_code(), CreateRecordBatch { source, .. } => source.status_code(), QueryExecution { source, .. } | QueryPlan { source, .. } => source.status_code(), PlanSql { error, .. } => { diff --git a/src/query/src/plan.rs b/src/query/src/plan.rs index ea9dae3770..710e92129d 100644 --- a/src/query/src/plan.rs +++ b/src/query/src/plan.rs @@ -12,108 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet}; -use std::fmt::{Debug, Display}; +use std::collections::HashSet; -use common_query::prelude::ScalarValue; use datafusion::datasource::DefaultTableSource; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; -use datafusion_common::{ParamValues, TableReference}; +use datafusion_common::TableReference; use datafusion_expr::LogicalPlan as DfLogicalPlan; -use datatypes::data_type::ConcreteDataType; -use datatypes::schema::Schema; use session::context::QueryContextRef; use snafu::ResultExt; pub use table::metadata::TableType; use table::table::adapter::DfTableProviderAdapter; use table::table_name::TableName; -use crate::error::{ConvertDatafusionSchemaSnafu, DataFusionSnafu, Result}; - -/// A LogicalPlan represents the different types of relational -/// operators (such as Projection, Filter, etc) and can be created by -/// the SQL query planner. -/// -/// A LogicalPlan represents transforming an input relation (table) to -/// an output relation (table) with a (potentially) different -/// schema. A plan represents a dataflow tree where data flows -/// from leaves up to the root to produce the query result. -#[derive(Clone, Debug)] -pub enum LogicalPlan { - DfPlan(DfLogicalPlan), -} - -impl LogicalPlan { - /// Get the schema for this logical plan - pub fn schema(&self) -> Result { - match self { - Self::DfPlan(plan) => { - let df_schema = plan.schema(); - df_schema - .clone() - .try_into() - .context(ConvertDatafusionSchemaSnafu) - } - } - } - - /// Return a `format`able structure that produces a single line - /// per node. For example: - /// - /// ```text - /// Projection: employee.id - /// Filter: employee.state Eq Utf8(\"CO\")\ - /// CsvScan: employee projection=Some([0, 3]) - /// ``` - pub fn display_indent(&self) -> impl Display + '_ { - let LogicalPlan::DfPlan(plan) = self; - plan.display_indent() - } - - /// Walk the logical plan, find any `PlaceHolder` tokens, - /// and return a map of their IDs and ConcreteDataTypes - pub fn get_param_types(&self) -> Result>> { - let LogicalPlan::DfPlan(plan) = self; - let types = plan.get_parameter_types().context(DataFusionSnafu)?; - - Ok(types - .into_iter() - .map(|(k, v)| (k, v.map(|v| ConcreteDataType::from_arrow_type(&v)))) - .collect()) - } - - /// Return a logical plan with all placeholders/params (e.g $1 $2, - /// ...) replaced with corresponding values provided in the - /// params_values - pub fn replace_params_with_values(&self, values: &[ScalarValue]) -> Result { - let LogicalPlan::DfPlan(plan) = self; - - plan.clone() - .replace_params_with_values(&ParamValues::List(values.to_vec())) - .context(DataFusionSnafu) - .map(LogicalPlan::DfPlan) - } - - /// Unwrap the logical plan into a DataFusion logical plan - pub fn unwrap_df_plan(self) -> DfLogicalPlan { - match self { - LogicalPlan::DfPlan(plan) => plan, - } - } - - /// Returns the DataFusion logical plan reference - pub fn df_plan(&self) -> &DfLogicalPlan { - match self { - LogicalPlan::DfPlan(plan) => plan, - } - } -} - -impl From for LogicalPlan { - fn from(plan: DfLogicalPlan) -> Self { - Self::DfPlan(plan) - } -} +use crate::error::{DataFusionSnafu, Result}; struct TableNamesExtractAndRewriter { pub(crate) table_names: HashSet, diff --git a/src/query/src/planner.rs b/src/query/src/planner.rs index 4c09860335..0f4f74133a 100644 --- a/src/query/src/planner.rs +++ b/src/query/src/planner.rs @@ -22,7 +22,7 @@ use common_telemetry::tracing; use datafusion::common::DFSchema; use datafusion::execution::context::SessionState; use datafusion::sql::planner::PlannerContext; -use datafusion_expr::Expr as DfExpr; +use datafusion_expr::{Expr as DfExpr, LogicalPlan}; use datafusion_sql::planner::{ParserOptions, SqlToRel}; use promql_parser::parser::EvalStmt; use session::context::QueryContextRef; @@ -32,7 +32,6 @@ use sql::statements::statement::Statement; use crate::error::{DataFusionSnafu, PlanSqlSnafu, QueryPlanSnafu, Result, SqlSnafu}; use crate::parser::QueryStatement; -use crate::plan::LogicalPlan; use crate::promql::planner::PromPlanner; use crate::query_engine::{DefaultPlanDecoder, QueryEngineState}; use crate::range_select::plan_rewrite::RangePlanRewriter; @@ -109,7 +108,7 @@ impl DfLogicalPlanner { .optimize_by_extension_rules(plan, &context) .context(DataFusionSnafu)?; - Ok(LogicalPlan::DfPlan(plan)) + Ok(plan) } /// Generate a relational expression from a SQL expression @@ -160,7 +159,6 @@ impl DfLogicalPlanner { ); PromPlanner::stmt_to_plan(table_provider, stmt, &self.session_state) .await - .map(LogicalPlan::DfPlan) .map_err(BoxedError::new) .context(QueryPlanSnafu) } @@ -168,7 +166,7 @@ impl DfLogicalPlanner { #[tracing::instrument(skip_all)] fn optimize_logical_plan(&self, plan: LogicalPlan) -> Result { self.engine_state - .optimize_logical_plan(plan.unwrap_df_plan()) + .optimize_logical_plan(plan) .context(DataFusionSnafu) .map(Into::into) } diff --git a/src/query/src/query_engine.rs b/src/query/src/query_engine.rs index e2d3e01c91..61f6e1a8f0 100644 --- a/src/query/src/query_engine.rs +++ b/src/query/src/query_engine.rs @@ -30,6 +30,7 @@ use common_function::handlers::{ use common_function::scalars::aggregate::AggregateFunctionMetaRef; use common_query::prelude::ScalarUdf; use common_query::Output; +use datafusion_expr::LogicalPlan; use datatypes::schema::Schema; pub use default_serializer::{DefaultPlanDecoder, DefaultSerializer}; use session::context::QueryContextRef; @@ -38,7 +39,6 @@ use table::TableRef; use crate::dataframe::DataFrame; use crate::datafusion::DatafusionQueryEngine; use crate::error::Result; -use crate::plan::LogicalPlan; use crate::planner::LogicalPlanner; pub use crate::query_engine::context::QueryEngineContext; pub use crate::query_engine::state::QueryEngineState; diff --git a/src/query/src/range_select/plan_rewrite.rs b/src/query/src/range_select/plan_rewrite.rs index 9eb16b359e..43b0928539 100644 --- a/src/query/src/range_select/plan_rewrite.rs +++ b/src/query/src/range_select/plan_rewrite.rs @@ -556,7 +556,6 @@ mod test { use super::*; use crate::parser::QueryLanguageParser; - use crate::plan::LogicalPlan as GreptimeLogicalPlan; use crate::{QueryEngineFactory, QueryEngineRef}; async fn create_test_engine() -> QueryEngineRef { @@ -611,14 +610,14 @@ mod test { QueryEngineFactory::new(catalog_list, None, None, None, None, false).query_engine() } - async fn do_query(sql: &str) -> Result { + async fn do_query(sql: &str) -> Result { let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap(); let engine = create_test_engine().await; engine.planner().plan(stmt, QueryContext::arc()).await } async fn query_plan_compare(sql: &str, expected: String) { - let GreptimeLogicalPlan::DfPlan(plan) = do_query(sql).await.unwrap(); + let plan = do_query(sql).await.unwrap(); assert_eq!(plan.display_indent_schema().to_string(), expected); } diff --git a/src/query/src/tests/query_engine_test.rs b/src/query/src/tests/query_engine_test.rs index 6359940d4f..687346dbcb 100644 --- a/src/query/src/tests/query_engine_test.rs +++ b/src/query/src/tests/query_engine_test.rs @@ -35,7 +35,6 @@ use table::test_util::MemTable; use crate::error::{QueryExecutionSnafu, Result}; use crate::parser::QueryLanguageParser; -use crate::plan::LogicalPlan; use crate::query_engine::options::QueryOptions; use crate::query_engine::QueryEngineFactory; use crate::tests::exec_selection; @@ -64,18 +63,16 @@ async fn test_datafusion_query_engine() -> Result<()> { let limit = 10; let table_provider = Arc::new(DfTableProviderAdapter::new(table.clone())); - let plan = LogicalPlan::DfPlan( - LogicalPlanBuilder::scan( - "numbers", - Arc::new(DefaultTableSource { table_provider }), - None, - ) - .unwrap() - .limit(0, Some(limit)) - .unwrap() - .build() - .unwrap(), - ); + let plan = LogicalPlanBuilder::scan( + "numbers", + Arc::new(DefaultTableSource { table_provider }), + None, + ) + .unwrap() + .limit(0, Some(limit)) + .unwrap() + .build() + .unwrap(); let output = engine.execute(plan, QueryContext::arc()).await?; diff --git a/src/script/src/table.rs b/src/script/src/table.rs index bbc04e6f64..cc20cd4f84 100644 --- a/src/script/src/table.rs +++ b/src/script/src/table.rs @@ -33,7 +33,6 @@ use datafusion_common::TableReference; use datafusion_expr::LogicalPlanBuilder; use datatypes::prelude::ScalarVector; use datatypes::vectors::{StringVector, Vector}; -use query::plan::LogicalPlan; use query::QueryEngineRef; use servers::query_handler::grpc::GrpcQueryHandlerRef; use session::context::{QueryContextBuilder, QueryContextRef}; @@ -224,7 +223,7 @@ impl ScriptsTable { let output = self .query_engine - .execute(LogicalPlan::DfPlan(plan), query_ctx(&table_info)) + .execute(plan, query_ctx(&table_info)) .await .context(ExecuteInternalStatementSnafu)?; let stream = match output.data { @@ -279,7 +278,7 @@ impl ScriptsTable { .context(BuildDfLogicalPlanSnafu)?; let output = query_engine - .execute(LogicalPlan::DfPlan(plan), query_ctx(&table_info)) + .execute(plan, query_ctx(&table_info)) .await .context(ExecuteInternalStatementSnafu)?; let stream = match output.data { diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index d088961e73..54665b8c68 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -46,6 +46,7 @@ common-version = { workspace = true, features = ["codec"] } dashmap.workspace = true datafusion.workspace = true datafusion-common.workspace = true +datafusion-expr.workspace = true datatypes.workspace = true derive_builder.workspace = true futures = "0.3" diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index 43bb458bd5..0fde3b527c 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -398,13 +398,6 @@ pub enum Error { source: query::error::Error, }, - #[snafu(display("Failed to get param types"))] - GetPreparedStmtParams { - source: query::error::Error, - #[snafu(implicit)] - location: Location, - }, - #[snafu(display("{}", reason))] UnexpectedResult { reason: String, @@ -452,13 +445,6 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to replace params with values in prepared statement"))] - ReplacePreparedStmtParams { - source: query::error::Error, - #[snafu(implicit)] - location: Location, - }, - #[snafu(display("Failed to convert scalar value"))] ConvertScalarValue { source: datatypes::error::Error, @@ -635,9 +621,7 @@ impl ErrorExt for Error { InvalidUtf8Value { .. } => StatusCode::InvalidArguments, - ReplacePreparedStmtParams { source, .. } - | GetPreparedStmtParams { source, .. } - | ParsePromQL { source, .. } => source.status_code(), + ParsePromQL { source, .. } => source.status_code(), Other { source, .. } => source.status_code(), UnexpectedResult { .. } => StatusCode::Unexpected, diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 18388998e7..a2b72b548b 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -969,11 +969,11 @@ mod test { use axum::routing::get; use common_query::Output; use common_recordbatch::RecordBatches; + use datafusion_expr::LogicalPlan; use datatypes::prelude::*; use datatypes::schema::{ColumnSchema, Schema}; use datatypes::vectors::{StringVector, UInt32Vector}; use query::parser::PromQuery; - use query::plan::LogicalPlan; use query::query_engine::DescribeResult; use session::context::QueryContextRef; use tokio::sync::mpsc; diff --git a/src/servers/src/interceptor.rs b/src/servers/src/interceptor.rs index d3478a56ea..76ed4728e0 100644 --- a/src/servers/src/interceptor.rs +++ b/src/servers/src/interceptor.rs @@ -21,8 +21,8 @@ use api::v1::RowInsertRequests; use async_trait::async_trait; use common_error::ext::ErrorExt; use common_query::Output; +use datafusion_expr::LogicalPlan; use query::parser::PromQuery; -use query::plan::LogicalPlan; use serde_json::Value; use session::context::QueryContextRef; use sql::statements::statement::Statement; diff --git a/src/servers/src/lib.rs b/src/servers/src/lib.rs index ff1af967fa..ce6857c6d2 100644 --- a/src/servers/src/lib.rs +++ b/src/servers/src/lib.rs @@ -18,8 +18,8 @@ #![feature(let_chains)] #![feature(if_let_guard)] +use datafusion_expr::LogicalPlan; use datatypes::schema::Schema; -use query::plan::LogicalPlan; pub mod addrs; pub mod configurator; diff --git a/src/servers/src/mysql/handler.rs b/src/servers/src/mysql/handler.rs index 8e7b4630eb..3c0ac36f4c 100644 --- a/src/servers/src/mysql/handler.rs +++ b/src/servers/src/mysql/handler.rs @@ -25,6 +25,8 @@ use common_catalog::parse_optional_catalog_and_schema_from_db_string; use common_error::ext::ErrorExt; use common_query::Output; use common_telemetry::{debug, error, tracing, warn}; +use datafusion_common::ParamValues; +use datafusion_expr::LogicalPlan; use datatypes::prelude::ConcreteDataType; use itertools::Itertools; use opensrv_mysql::{ @@ -32,7 +34,6 @@ use opensrv_mysql::{ StatementMetaWriter, ValueInner, }; use parking_lot::RwLock; -use query::plan::LogicalPlan; use query::query_engine::DescribeResult; use rand::RngCore; use session::context::{Channel, QueryContextRef}; @@ -43,7 +44,7 @@ use sql::parser::{ParseOptions, ParserContext}; use sql::statements::statement::Statement; use tokio::io::AsyncWrite; -use crate::error::{self, InvalidPrepareStatementSnafu, Result}; +use crate::error::{self, DataFrameSnafu, InvalidPrepareStatementSnafu, Result}; use crate::metrics::METRIC_AUTH_FAILURE; use crate::mysql::helper::{ self, format_placeholder, replace_placeholders, transform_placeholders, @@ -175,8 +176,11 @@ impl MysqlInstanceShim { let params = if let Some(plan) = &plan { prepared_params( &plan - .get_param_types() - .context(error::GetPreparedStmtParamsSnafu)?, + .get_parameter_types() + .context(DataFrameSnafu)? + .into_iter() + .map(|(k, v)| (k, v.map(|v| ConcreteDataType::from_arrow_type(&v)))) + .collect(), )? } else { dummy_params(param_num)? @@ -323,8 +327,11 @@ impl AsyncMysqlShim for MysqlInstanceShi let outputs = match sql_plan.plan { Some(plan) => { let param_types = plan - .get_param_types() - .context(error::GetPreparedStmtParamsSnafu)?; + .get_parameter_types() + .context(DataFrameSnafu)? + .into_iter() + .map(|(k, v)| (k, v.map(|v| ConcreteDataType::from_arrow_type(&v)))) + .collect::>(); if params.len() != param_types.len() { return error::InternalSnafu { @@ -436,8 +443,11 @@ impl AsyncMysqlShim for MysqlInstanceShi let outputs = match sql_plan.plan { Some(plan) => { let param_types = plan - .get_param_types() - .context(error::GetPreparedStmtParamsSnafu)?; + .get_parameter_types() + .context(DataFrameSnafu)? + .into_iter() + .map(|(k, v)| (k, v.map(|v| ConcreteDataType::from_arrow_type(&v)))) + .collect::>(); if params.len() != param_types.len() { writer @@ -618,8 +628,9 @@ fn replace_params_with_values( } } - plan.replace_params_with_values(&values) - .context(error::ReplacePreparedStmtParamsSnafu) + plan.clone() + .replace_params_with_values(&ParamValues::List(values.clone())) + .context(DataFrameSnafu) } fn replace_params_with_exprs( @@ -645,8 +656,9 @@ fn replace_params_with_exprs( } } - plan.replace_params_with_values(&values) - .context(error::ReplacePreparedStmtParamsSnafu) + plan.clone() + .replace_params_with_values(&ParamValues::List(values.clone())) + .context(DataFrameSnafu) } async fn validate_query(query: &str) -> Result { diff --git a/src/servers/src/postgres/handler.rs b/src/servers/src/postgres/handler.rs index 53d907d814..e10a45ddec 100644 --- a/src/servers/src/postgres/handler.rs +++ b/src/servers/src/postgres/handler.rs @@ -20,6 +20,8 @@ use common_query::{Output, OutputData}; use common_recordbatch::error::Result as RecordBatchResult; use common_recordbatch::RecordBatch; use common_telemetry::{debug, error, tracing}; +use datafusion_common::ParamValues; +use datatypes::prelude::ConcreteDataType; use datatypes::schema::SchemaRef; use futures::{future, stream, Stream, StreamExt}; use pgwire::api::portal::{Format, Portal}; @@ -272,7 +274,10 @@ impl ExtendedQueryHandler for PostgresServerHandler { let output = if let Some(plan) = &sql_plan.plan { let plan = plan - .replace_params_with_values(parameters_to_scalar_values(plan, portal)?.as_ref()) + .clone() + .replace_params_with_values(&ParamValues::List(parameters_to_scalar_values( + plan, portal, + )?)) .map_err(|e| PgWireError::ApiError(Box::new(e)))?; self.query_handler .do_exec_plan(plan, query_ctx.clone()) @@ -306,8 +311,11 @@ impl ExtendedQueryHandler for PostgresServerHandler { let sql_plan = &stmt.statement; let (param_types, sql_plan, format) = if let Some(plan) = &sql_plan.plan { let param_types = plan - .get_param_types() - .map_err(|e| PgWireError::ApiError(Box::new(e)))?; + .get_parameter_types() + .map_err(|e| PgWireError::ApiError(Box::new(e)))? + .into_iter() + .map(|(k, v)| (k, v.map(|v| ConcreteDataType::from_arrow_type(&v)))) + .collect(); let types = param_types_to_pg_types(¶m_types) .map_err(|e| PgWireError::ApiError(Box::new(e)))?; diff --git a/src/servers/src/postgres/types.rs b/src/servers/src/postgres/types.rs index 9f9d94905e..85b626cb1a 100644 --- a/src/servers/src/postgres/types.rs +++ b/src/servers/src/postgres/types.rs @@ -23,6 +23,7 @@ use std::ops::Deref; use chrono::{NaiveDate, NaiveDateTime}; use common_time::Interval; use datafusion_common::ScalarValue; +use datafusion_expr::LogicalPlan; use datatypes::prelude::{ConcreteDataType, Value}; use datatypes::schema::Schema; use datatypes::types::TimestampType; @@ -30,7 +31,6 @@ use pgwire::api::portal::{Format, Portal}; use pgwire::api::results::{DataRowEncoder, FieldInfo}; use pgwire::api::Type; use pgwire::error::{PgWireError, PgWireResult}; -use query::plan::LogicalPlan; use session::context::QueryContextRef; use session::session_config::PGByteaOutputValue; @@ -276,8 +276,11 @@ pub(super) fn parameters_to_scalar_values( let client_param_types = &portal.statement.parameter_types; let param_types = plan - .get_param_types() - .map_err(|e| PgWireError::ApiError(Box::new(e)))?; + .get_parameter_types() + .map_err(|e| PgWireError::ApiError(Box::new(e)))? + .into_iter() + .map(|(k, v)| (k, v.map(|v| ConcreteDataType::from_arrow_type(&v)))) + .collect::>(); for idx in 0..param_count { let server_type = param_types diff --git a/src/servers/src/prom_store.rs b/src/servers/src/prom_store.rs index d548a843b5..34d332e81d 100644 --- a/src/servers/src/prom_store.rs +++ b/src/servers/src/prom_store.rs @@ -28,10 +28,10 @@ use common_telemetry::tracing; use common_time::timestamp::TimeUnit; use datafusion::prelude::{col, lit, regexp_match, Expr}; use datafusion_common::ScalarValue; +use datafusion_expr::LogicalPlan; use datatypes::prelude::{ConcreteDataType, Value}; use openmetrics_parser::{MetricsExposition, PrometheusType, PrometheusValue}; use query::dataframe::DataFrame; -use query::plan::LogicalPlan; use snafu::{ensure, OptionExt, ResultExt}; use snap::raw::{Decoder, Encoder}; @@ -123,7 +123,7 @@ pub fn query_to_plan(dataframe: DataFrame, q: &Query) -> Result { .filter(conditions) .context(error::DataFrameSnafu)?; - Ok(LogicalPlan::DfPlan(dataframe.into_parts().1)) + Ok(dataframe.into_parts().1) } #[inline] diff --git a/src/servers/src/query_handler/sql.rs b/src/servers/src/query_handler/sql.rs index 9b9148af88..eef443bbec 100644 --- a/src/servers/src/query_handler/sql.rs +++ b/src/servers/src/query_handler/sql.rs @@ -17,8 +17,8 @@ use std::sync::Arc; use async_trait::async_trait; use common_error::ext::{BoxedError, ErrorExt}; use common_query::Output; +use datafusion_expr::LogicalPlan; use query::parser::PromQuery; -use query::plan::LogicalPlan; use session::context::QueryContextRef; use snafu::ResultExt; use sql::statements::statement::Statement; diff --git a/src/servers/tests/http/influxdb_test.rs b/src/servers/tests/http/influxdb_test.rs index 3e6e743b5c..a0ef088050 100644 --- a/src/servers/tests/http/influxdb_test.rs +++ b/src/servers/tests/http/influxdb_test.rs @@ -21,8 +21,8 @@ use auth::tests::{DatabaseAuthInfo, MockUserProvider}; use axum::{http, Router}; use common_query::Output; use common_test_util::ports; +use datafusion_expr::LogicalPlan; use query::parser::PromQuery; -use query::plan::LogicalPlan; use query::query_engine::DescribeResult; use servers::error::{Error, Result}; use servers::http::header::constants::GREPTIME_DB_HEADER_NAME; diff --git a/src/servers/tests/http/opentsdb_test.rs b/src/servers/tests/http/opentsdb_test.rs index 1a719aa93d..635060261a 100644 --- a/src/servers/tests/http/opentsdb_test.rs +++ b/src/servers/tests/http/opentsdb_test.rs @@ -19,8 +19,8 @@ use async_trait::async_trait; use axum::Router; use common_query::Output; use common_test_util::ports; +use datafusion_expr::LogicalPlan; use query::parser::PromQuery; -use query::plan::LogicalPlan; use query::query_engine::DescribeResult; use servers::error::{self, Result}; use servers::http::test_helpers::TestClient; diff --git a/src/servers/tests/http/prom_store_test.rs b/src/servers/tests/http/prom_store_test.rs index fd2d01304e..bb800e5b7c 100644 --- a/src/servers/tests/http/prom_store_test.rs +++ b/src/servers/tests/http/prom_store_test.rs @@ -23,9 +23,9 @@ use async_trait::async_trait; use axum::Router; use common_query::Output; use common_test_util::ports; +use datafusion_expr::LogicalPlan; use prost::Message; use query::parser::PromQuery; -use query::plan::LogicalPlan; use query::query_engine::DescribeResult; use servers::error::{Error, Result}; use servers::http::header::{CONTENT_ENCODING_SNAPPY, CONTENT_TYPE_PROTOBUF}; diff --git a/src/servers/tests/mod.rs b/src/servers/tests/mod.rs index daf9382ec7..ca098546aa 100644 --- a/src/servers/tests/mod.rs +++ b/src/servers/tests/mod.rs @@ -21,8 +21,8 @@ use async_trait::async_trait; use catalog::memory::MemoryCatalogManager; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_query::Output; +use datafusion_expr::LogicalPlan; use query::parser::{PromQuery, QueryLanguageParser, QueryStatement}; -use query::plan::LogicalPlan; use query::query_engine::DescribeResult; use query::{QueryEngineFactory, QueryEngineRef}; use script::engine::{CompileContext, EvalContext, Script, ScriptEngine}; diff --git a/tests-integration/src/grpc.rs b/tests-integration/src/grpc.rs index fa88de07de..a4ef632b33 100644 --- a/tests-integration/src/grpc.rs +++ b/tests-integration/src/grpc.rs @@ -33,7 +33,6 @@ mod test { use common_recordbatch::RecordBatches; use frontend::instance::Instance; use query::parser::QueryLanguageParser; - use query::plan::LogicalPlan; use query::query_engine::DefaultSerializer; use servers::query_handler::grpc::GrpcQueryHandler; use session::context::QueryContext; @@ -540,7 +539,7 @@ CREATE TABLE {table_name} ( &QueryContext::arc(), ) .unwrap(); - let LogicalPlan::DfPlan(plan) = instance + let plan = instance .frontend() .statement_executor() .plan(stmt, QueryContext::arc()) diff --git a/tests-integration/src/instance.rs b/tests-integration/src/instance.rs index b3f966c811..096be44e0c 100644 --- a/tests-integration/src/instance.rs +++ b/tests-integration/src/instance.rs @@ -28,10 +28,10 @@ mod tests { use common_query::Output; use common_recordbatch::RecordBatches; use common_telemetry::debug; + use datafusion_expr::LogicalPlan; use frontend::error::{self, Error, Result}; use frontend::instance::Instance; use query::parser::QueryLanguageParser; - use query::plan::LogicalPlan; use query::query_engine::DefaultSerializer; use servers::interceptor::{SqlQueryInterceptor, SqlQueryInterceptorRef}; use servers::query_handler::sql::SqlQueryHandler; @@ -233,7 +233,7 @@ mod tests { &QueryContext::arc(), ) .unwrap(); - let LogicalPlan::DfPlan(plan) = instance + let plan = instance .frontend() .statement_executor() .plan(stmt, QueryContext::arc()) @@ -317,7 +317,7 @@ mod tests { fn pre_execute( &self, _statement: &Statement, - _plan: Option<&query::plan::LogicalPlan>, + _plan: Option<&LogicalPlan>, _query_ctx: QueryContextRef, ) -> Result<()> { let _ = self.c.fetch_add(1, std::sync::atomic::Ordering::Relaxed); From f02410c39bd0edce6e4786b2f6b0c6dfd7d03be4 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Fri, 20 Sep 2024 08:35:37 +0800 Subject: [PATCH 023/128] fix: disable field pruning in last non null mode (#4740) * fix: don't prune fields in last non null mode * test: add sqlness test for field pruning * test: add flush * refine implementation Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia Co-authored-by: Ruihang Xia --- src/mito2/src/read/scan_region.rs | 39 ++++++++++- .../common/select/prune_field.result | 69 +++++++++++++++++++ .../standalone/common/select/prune_field.sql | 24 +++++++ 3 files changed, 131 insertions(+), 1 deletion(-) create mode 100644 tests/cases/standalone/common/select/prune_field.result create mode 100644 tests/cases/standalone/common/select/prune_field.sql diff --git a/src/mito2/src/read/scan_region.rs b/src/mito2/src/read/scan_region.rs index 40f18c393b..d28562c554 100644 --- a/src/mito2/src/read/scan_region.rs +++ b/src/mito2/src/read/scan_region.rs @@ -14,6 +14,7 @@ //! Scans a region according to the scan request. +use std::collections::HashSet; use std::fmt; use std::sync::Arc; use std::time::{Duration, Instant}; @@ -24,6 +25,7 @@ use common_telemetry::{debug, error, tracing, warn}; use common_time::range::TimestampRange; use common_time::Timestamp; use datafusion::physical_plan::DisplayFormatType; +use datafusion_expr::utils::expr_to_columns; use smallvec::SmallVec; use store_api::region_engine::{PartitionRange, RegionScannerRef}; use store_api::storage::{ScanRequest, TimeSeriesRowSelector}; @@ -295,6 +297,9 @@ impl ScanRegion { self.version.options.append_mode, ); + // Remove field filters for LastNonNull mode after logging the request. + self.maybe_remove_field_filters(); + let inverted_index_applier = self.build_invereted_index_applier(); let fulltext_index_applier = self.build_fulltext_index_applier(); let predicate = Predicate::new(self.request.filters.clone()); @@ -321,7 +326,7 @@ impl ScanRegion { Ok(input) } - /// Build time range predicate from filters. + /// Build time range predicate from filters, also remove time filters from request. fn build_time_range_predicate(&mut self) -> TimestampRange { let time_index = self.version.metadata.time_index_column(); let unit = time_index @@ -337,6 +342,38 @@ impl ScanRegion { ) } + /// Remove field filters if the merge mode is [MergeMode::LastNonNull]. + fn maybe_remove_field_filters(&mut self) { + if self.version.options.merge_mode() != MergeMode::LastNonNull { + return; + } + + // TODO(yingwen): We can ignore field filters only when there are multiple sources in the same time window. + let field_columns = self + .version + .metadata + .field_columns() + .map(|col| &col.column_schema.name) + .collect::>(); + // Columns in the expr. + let mut columns = HashSet::new(); + + self.request.filters.retain(|expr| { + columns.clear(); + // `expr_to_columns` won't return error. + if expr_to_columns(expr, &mut columns).is_err() { + return false; + } + for column in &columns { + if field_columns.contains(&column.name) { + // This expr uses the field column. + return false; + } + } + true + }); + } + /// Use the latest schema to build the inveretd index applier. fn build_invereted_index_applier(&self) -> Option { if self.ignore_inverted_index { diff --git a/tests/cases/standalone/common/select/prune_field.result b/tests/cases/standalone/common/select/prune_field.result new file mode 100644 index 0000000000..d309d384be --- /dev/null +++ b/tests/cases/standalone/common/select/prune_field.result @@ -0,0 +1,69 @@ +CREATE TABLE IF NOT EXISTS prune_field ( + ts TIMESTAMP TIME INDEX, + tag UInt16, + a UInt8, + b UInt8, +PRIMARY KEY (tag)) ENGINE = mito WITH('merge_mode'='last_non_null'); + +Affected Rows: 0 + +insert into prune_field(ts, tag, a, b) values(0, 1, 1, null); + +Affected Rows: 1 + +admin flush_table('prune_field'); + ++----------------------------------+ +| ADMIN flush_table('prune_field') | ++----------------------------------+ +| 0 | ++----------------------------------+ + +insert into prune_field(ts, tag, a, b) values(0, 1, null, 1); + +Affected Rows: 1 + +admin flush_table('prune_field'); + ++----------------------------------+ +| ADMIN flush_table('prune_field') | ++----------------------------------+ +| 0 | ++----------------------------------+ + +select * from prune_field where a = 1; + ++---------------------+-----+---+---+ +| ts | tag | a | b | ++---------------------+-----+---+---+ +| 1970-01-01T00:00:00 | 1 | 1 | 1 | ++---------------------+-----+---+---+ + +select * from prune_field where b = 1; + ++---------------------+-----+---+---+ +| ts | tag | a | b | ++---------------------+-----+---+---+ +| 1970-01-01T00:00:00 | 1 | 1 | 1 | ++---------------------+-----+---+---+ + +select * from prune_field; + ++---------------------+-----+---+---+ +| ts | tag | a | b | ++---------------------+-----+---+---+ +| 1970-01-01T00:00:00 | 1 | 1 | 1 | ++---------------------+-----+---+---+ + +select * from prune_field where a = 1 and b = 1; + ++---------------------+-----+---+---+ +| ts | tag | a | b | ++---------------------+-----+---+---+ +| 1970-01-01T00:00:00 | 1 | 1 | 1 | ++---------------------+-----+---+---+ + +drop table prune_field; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/select/prune_field.sql b/tests/cases/standalone/common/select/prune_field.sql new file mode 100644 index 0000000000..6488175508 --- /dev/null +++ b/tests/cases/standalone/common/select/prune_field.sql @@ -0,0 +1,24 @@ +CREATE TABLE IF NOT EXISTS prune_field ( + ts TIMESTAMP TIME INDEX, + tag UInt16, + a UInt8, + b UInt8, +PRIMARY KEY (tag)) ENGINE = mito WITH('merge_mode'='last_non_null'); + +insert into prune_field(ts, tag, a, b) values(0, 1, 1, null); + +admin flush_table('prune_field'); + +insert into prune_field(ts, tag, a, b) values(0, 1, null, 1); + +admin flush_table('prune_field'); + +select * from prune_field where a = 1; + +select * from prune_field where b = 1; + +select * from prune_field; + +select * from prune_field where a = 1 and b = 1; + +drop table prune_field; From c4e52ebf918d4ae7f758c3336c07e6d60b022618 Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Fri, 20 Sep 2024 10:34:45 +0800 Subject: [PATCH 024/128] feat: use new image for gcc-10 (#4748) feat: use new image --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 697a31aed6..da9e8da5c0 100644 --- a/Makefile +++ b/Makefile @@ -8,7 +8,7 @@ CARGO_BUILD_OPTS := --locked IMAGE_REGISTRY ?= docker.io IMAGE_NAMESPACE ?= greptime IMAGE_TAG ?= latest -DEV_BUILDER_IMAGE_TAG ?= 2024-06-06-b4b105ad-20240827021230 +DEV_BUILDER_IMAGE_TAG ?= 2024-06-06-1acda74c-20240919113454 BUILDX_MULTI_PLATFORM_BUILD ?= false BUILDX_BUILDER_NAME ?= gtbuilder BASE_IMAGE ?= ubuntu From e12ffbeb2f807f83e1e15c6a921880d2ba2bdb4d Mon Sep 17 00:00:00 2001 From: Yingwen Date: Fri, 20 Sep 2024 10:55:31 +0800 Subject: [PATCH 025/128] feat: flush other workers if still need flush (#4746) --- src/mito2/src/engine/flush_test.rs | 69 ++++++++++++++++++++++++++++++ src/mito2/src/engine/listener.rs | 10 ++++- src/mito2/src/worker.rs | 5 +++ 3 files changed, 83 insertions(+), 1 deletion(-) diff --git a/src/mito2/src/engine/flush_test.rs b/src/mito2/src/engine/flush_test.rs index aac02db91e..c134def6aa 100644 --- a/src/mito2/src/engine/flush_test.rs +++ b/src/mito2/src/engine/flush_test.rs @@ -402,3 +402,72 @@ async fn test_auto_flush_engine() { +-------+---------+---------------------+"; assert_eq!(expected, batches.pretty_print().unwrap()); } + +#[tokio::test] +async fn test_flush_workers() { + let mut env = TestEnv::new(); + let write_buffer_manager = Arc::new(MockWriteBufferManager::default()); + let listener = Arc::new(FlushListener::default()); + let engine = env + .create_engine_with( + MitoConfig { + num_workers: 2, + ..Default::default() + }, + Some(write_buffer_manager.clone()), + Some(listener.clone()), + ) + .await; + + let region_id0 = RegionId::new(1, 0); + let region_id1 = RegionId::new(1, 1); + let request = CreateRequestBuilder::new().region_dir("r0").build(); + let column_schemas = rows_schema(&request); + engine + .handle_request(region_id0, RegionRequest::Create(request)) + .await + .unwrap(); + let request = CreateRequestBuilder::new().region_dir("r1").build(); + engine + .handle_request(region_id1, RegionRequest::Create(request.clone())) + .await + .unwrap(); + + // Prepares rows for flush. + let rows = Rows { + schema: column_schemas.clone(), + rows: build_rows_for_key("a", 0, 2, 0), + }; + put_rows(&engine, region_id0, rows.clone()).await; + put_rows(&engine, region_id1, rows).await; + + write_buffer_manager.set_should_flush(true); + + // Writes to the mutable memtable and triggers flush for region 0. + let rows = Rows { + schema: column_schemas.clone(), + rows: build_rows_for_key("b", 0, 2, 0), + }; + put_rows(&engine, region_id0, rows).await; + + // Waits until flush is finished. + while listener.success_count() < 2 { + listener.wait().await; + } + + // Scans region 1. + let request = ScanRequest::default(); + let scanner = engine.scanner(region_id1, request).unwrap(); + assert_eq!(0, scanner.num_memtables()); + assert_eq!(1, scanner.num_files()); + let stream = scanner.scan().await.unwrap(); + let batches = RecordBatches::try_collect(stream).await.unwrap(); + let expected = "\ ++-------+---------+---------------------+ +| tag_0 | field_0 | ts | ++-------+---------+---------------------+ +| a | 0.0 | 1970-01-01T00:00:00 | +| a | 1.0 | 1970-01-01T00:00:01 | ++-------+---------+---------------------+"; + assert_eq!(expected, batches.pretty_print().unwrap()); +} diff --git a/src/mito2/src/engine/listener.rs b/src/mito2/src/engine/listener.rs index a79cb6eafd..83679f96e4 100644 --- a/src/mito2/src/engine/listener.rs +++ b/src/mito2/src/engine/listener.rs @@ -14,6 +14,7 @@ //! Engine event listener for tests. +use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; use std::time::Duration; @@ -77,6 +78,7 @@ pub type EventListenerRef = Arc; #[derive(Default)] pub struct FlushListener { notify: Notify, + success_count: AtomicUsize, } impl FlushListener { @@ -84,6 +86,11 @@ impl FlushListener { pub async fn wait(&self) { self.notify.notified().await; } + + /// Returns the success count. + pub fn success_count(&self) -> usize { + self.success_count.load(Ordering::Relaxed) + } } #[async_trait] @@ -91,7 +98,8 @@ impl EventListener for FlushListener { fn on_flush_success(&self, region_id: RegionId) { info!("Region {} flush successfully", region_id); - self.notify.notify_one() + self.success_count.fetch_add(1, Ordering::Relaxed); + self.notify.notify_one(); } } diff --git a/src/mito2/src/worker.rs b/src/mito2/src/worker.rs index 0f872e24e4..c2fbc80982 100644 --- a/src/mito2/src/worker.rs +++ b/src/mito2/src/worker.rs @@ -670,6 +670,11 @@ impl RegionWorkerLoop { // The channel is disconnected. break; } else { + // Also flush this worker if other workers trigger flush as this worker may have + // a large memtable to flush. We may not have chance to flush that memtable if we + // never write to this worker. So only flushing other workers may not release enough + // memory. + self.maybe_flush_worker(); // A flush job is finished, handles stalled requests. self.handle_stalled_requests().await; continue; From 75c6fad1a3d409229ea88bb9e17ebff56d297d04 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Fri, 20 Sep 2024 12:19:50 +0800 Subject: [PATCH 026/128] feat: add more h3 scalar functions (#4707) * feat: add more h3 scalar functions * chore: comment up --- Cargo.lock | 22 + src/common/function/Cargo.toml | 1 + src/common/function/src/scalars/geo.rs | 16 +- src/common/function/src/scalars/geo/h3.rs | 718 +++++++++++++++++- .../standalone/common/function/geo.result | 174 +++-- .../cases/standalone/common/function/geo.sql | 41 +- 6 files changed, 857 insertions(+), 115 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 55e8ca8e4b..e7f180eb17 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1997,6 +1997,7 @@ dependencies = [ "common-version", "datafusion", "datatypes", + "derive_more", "geohash", "h3o", "jsonb", @@ -3413,6 +3414,27 @@ dependencies = [ "syn 2.0.66", ] +[[package]] +name = "derive_more" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4a9b99b9cbbe49445b21764dc0625032a89b145a2642e67603e1c936f5458d05" +dependencies = [ + "derive_more-impl", +] + +[[package]] +name = "derive_more-impl" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.66", + "unicode-xid", +] + [[package]] name = "diff" version = "0.1.13" diff --git a/src/common/function/Cargo.toml b/src/common/function/Cargo.toml index b2e9c5a98b..e9a3dd6b55 100644 --- a/src/common/function/Cargo.toml +++ b/src/common/function/Cargo.toml @@ -27,6 +27,7 @@ common-time.workspace = true common-version.workspace = true datafusion.workspace = true datatypes.workspace = true +derive_more = { version = "1", default-features = false, features = ["display"] } geohash = { version = "0.13", optional = true } h3o = { version = "0.6", optional = true } jsonb.workspace = true diff --git a/src/common/function/src/scalars/geo.rs b/src/common/function/src/scalars/geo.rs index 4b126f20f0..9e415a3fdd 100644 --- a/src/common/function/src/scalars/geo.rs +++ b/src/common/function/src/scalars/geo.rs @@ -17,7 +17,6 @@ mod geohash; mod h3; use geohash::GeohashFunction; -use h3::H3Function; use crate::function_registry::FunctionRegistry; @@ -25,7 +24,20 @@ pub(crate) struct GeoFunctions; impl GeoFunctions { pub fn register(registry: &FunctionRegistry) { + // geohash registry.register(Arc::new(GeohashFunction)); - registry.register(Arc::new(H3Function)); + // h3 family + registry.register(Arc::new(h3::H3LatLngToCell)); + registry.register(Arc::new(h3::H3LatLngToCellString)); + registry.register(Arc::new(h3::H3CellBase)); + registry.register(Arc::new(h3::H3CellCenterChild)); + registry.register(Arc::new(h3::H3CellCenterLat)); + registry.register(Arc::new(h3::H3CellCenterLng)); + registry.register(Arc::new(h3::H3CellIsPentagon)); + registry.register(Arc::new(h3::H3CellParent)); + registry.register(Arc::new(h3::H3CellResolution)); + registry.register(Arc::new(h3::H3CellToString)); + registry.register(Arc::new(h3::H3IsNeighbour)); + registry.register(Arc::new(h3::H3StringToCell)); } } diff --git a/src/common/function/src/scalars/geo/h3.rs b/src/common/function/src/scalars/geo/h3.rs index 26ec246997..672fbfd714 100644 --- a/src/common/function/src/scalars/geo/h3.rs +++ b/src/common/function/src/scalars/geo/h3.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt; +use std::str::FromStr; use common_error::ext::{BoxedError, PlainError}; use common_error::status_code::StatusCode; @@ -22,23 +22,118 @@ use datafusion::logical_expr::Volatility; use datatypes::prelude::ConcreteDataType; use datatypes::scalars::ScalarVectorBuilder; use datatypes::value::Value; -use datatypes::vectors::{MutableVector, StringVectorBuilder, VectorRef}; -use h3o::{LatLng, Resolution}; +use datatypes::vectors::{ + BooleanVectorBuilder, Float64VectorBuilder, MutableVector, StringVectorBuilder, + UInt64VectorBuilder, UInt8VectorBuilder, VectorRef, +}; +use derive_more::Display; +use h3o::{CellIndex, LatLng, Resolution}; use snafu::{ensure, ResultExt}; use crate::function::{Function, FunctionContext}; -/// Function that returns [h3] encoding string for a given geospatial coordinate. +/// Function that returns [h3] encoding cellid for a given geospatial coordinate. /// /// [h3]: https://h3geo.org/ -#[derive(Clone, Debug, Default)] -pub struct H3Function; +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3LatLngToCell; -const NAME: &str = "h3"; - -impl Function for H3Function { +impl Function for H3LatLngToCell { fn name(&self) -> &str { - NAME + "h3_latlng_to_cell" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint64_datatype()) + } + + fn signature(&self) -> Signature { + let mut signatures = Vec::new(); + for coord_type in &[ + ConcreteDataType::float32_datatype(), + ConcreteDataType::float64_datatype(), + ] { + for resolution_type in &[ + ConcreteDataType::int8_datatype(), + ConcreteDataType::int16_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int64_datatype(), + ConcreteDataType::uint8_datatype(), + ConcreteDataType::uint16_datatype(), + ConcreteDataType::uint32_datatype(), + ConcreteDataType::uint64_datatype(), + ] { + signatures.push(TypeSignature::Exact(vec![ + // latitude + coord_type.clone(), + // longitude + coord_type.clone(), + // resolution + resolution_type.clone(), + ])); + } + } + Signature::one_of(signatures, Volatility::Stable) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 3, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 3, provided : {}", + columns.len() + ), + } + ); + + let lat_vec = &columns[0]; + let lon_vec = &columns[1]; + let resolution_vec = &columns[2]; + + let size = lat_vec.len(); + let mut results = UInt64VectorBuilder::with_capacity(size); + + for i in 0..size { + let lat = lat_vec.get(i).as_f64_lossy(); + let lon = lon_vec.get(i).as_f64_lossy(); + let r = value_to_resolution(resolution_vec.get(i))?; + + let result = match (lat, lon) { + (Some(lat), Some(lon)) => { + let coord = LatLng::new(lat, lon) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("H3 error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?; + let encoded: u64 = coord.to_cell(r).into(); + Some(encoded) + } + _ => None, + }; + + results.push(result); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns [h3] encoding cellid in string form for a given +/// geospatial coordinate. +/// +/// [h3]: https://h3geo.org/ +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3LatLngToCellString; + +impl Function for H3LatLngToCellString { + fn name(&self) -> &str { + "h3_latlng_to_cell_string" } fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { @@ -95,17 +190,7 @@ impl Function for H3Function { for i in 0..size { let lat = lat_vec.get(i).as_f64_lossy(); let lon = lon_vec.get(i).as_f64_lossy(); - let r = match resolution_vec.get(i) { - Value::Int8(v) => v as u8, - Value::Int16(v) => v as u8, - Value::Int32(v) => v as u8, - Value::Int64(v) => v as u8, - Value::UInt8(v) => v, - Value::UInt16(v) => v as u8, - Value::UInt32(v) => v as u8, - Value::UInt64(v) => v as u8, - _ => unreachable!(), - }; + let r = value_to_resolution(resolution_vec.get(i))?; let result = match (lat, lon) { (Some(lat), Some(lon)) => { @@ -117,14 +202,6 @@ impl Function for H3Function { )) }) .context(error::ExecuteSnafu)?; - let r = Resolution::try_from(r) - .map_err(|e| { - BoxedError::new(PlainError::new( - format!("H3 error: {}", e), - StatusCode::EngineExecuteQuery, - )) - }) - .context(error::ExecuteSnafu)?; let encoded = coord.to_cell(r).to_string(); Some(encoded) } @@ -138,8 +215,585 @@ impl Function for H3Function { } } -impl fmt::Display for H3Function { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", NAME) +/// Function that converts cell id to its string form +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellToString; + +impl Function for H3CellToString { + fn name(&self) -> &str { + "h3_cell_to_string" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::string_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 1, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let size = cell_vec.len(); + let mut results = StringVectorBuilder::with_capacity(size); + + for i in 0..size { + let cell_id_string = cell_from_value(cell_vec.get(i))?.map(|c| c.to_string()); + + results.push(cell_id_string.as_deref()); + } + + Ok(results.to_vector()) } } + +/// Function that converts cell string id to uint64 number +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3StringToCell; + +impl Function for H3StringToCell { + fn name(&self) -> &str { + "h3_string_to_cell" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint64_datatype()) + } + + fn signature(&self) -> Signature { + Signature::new( + TypeSignature::Exact(vec![ConcreteDataType::string_datatype()]), + Volatility::Stable, + ) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 1, provided : {}", + columns.len() + ), + } + ); + + let string_vec = &columns[0]; + let size = string_vec.len(); + let mut results = UInt64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = string_vec.get(i); + + let cell_id = match cell { + Value::String(v) => Some( + CellIndex::from_str(v.as_utf8()) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("H3 error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)? + .into(), + ), + _ => None, + }; + + results.push(cell_id); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns centroid latitude of given cell id +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellCenterLat; + +impl Function for H3CellCenterLat { + fn name(&self) -> &str { + "h3_cell_center_lat" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::float64_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 1, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let size = cell_vec.len(); + let mut results = Float64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let lat = cell.map(|cell| LatLng::from(cell).lat()); + + results.push(lat); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns centroid longitude of given cell id +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellCenterLng; + +impl Function for H3CellCenterLng { + fn name(&self) -> &str { + "h3_cell_center_lng" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::float64_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 1, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let size = cell_vec.len(); + let mut results = Float64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let lat = cell.map(|cell| LatLng::from(cell).lng()); + + results.push(lat); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns resolution of given cell id +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellResolution; + +impl Function for H3CellResolution { + fn name(&self) -> &str { + "h3_cell_resolution" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint8_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 1, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let size = cell_vec.len(); + let mut results = UInt8VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = cell.map(|cell| cell.resolution().into()); + + results.push(res); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns base cell of given cell id +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellBase; + +impl Function for H3CellBase { + fn name(&self) -> &str { + "h3_cell_base" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint8_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 1, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let size = cell_vec.len(); + let mut results = UInt8VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = cell.map(|cell| cell.base_cell().into()); + + results.push(res); + } + + Ok(results.to_vector()) + } +} + +/// Function that check if given cell id is a pentagon +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellIsPentagon; + +impl Function for H3CellIsPentagon { + fn name(&self) -> &str { + "h3_cell_is_pentagon" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::boolean_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 1, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 1, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let size = cell_vec.len(); + let mut results = BooleanVectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = cell.map(|cell| cell.is_pentagon()); + + results.push(res); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns center child cell of given cell id +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellCenterChild; + +impl Function for H3CellCenterChild { + fn name(&self) -> &str { + "h3_cell_center_child" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint64_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell_and_resolution() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 2, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 2, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let res_vec = &columns[1]; + let size = cell_vec.len(); + let mut results = UInt64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = value_to_resolution(res_vec.get(i))?; + let result = cell + .and_then(|cell| cell.center_child(res)) + .map(|c| c.into()); + + results.push(result); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns parent cell of given cell id and resolution +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellParent; + +impl Function for H3CellParent { + fn name(&self) -> &str { + "h3_cell_parent" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint64_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell_and_resolution() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 2, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 2, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let res_vec = &columns[1]; + let size = cell_vec.len(); + let mut results = UInt64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = value_to_resolution(res_vec.get(i))?; + let result = cell.and_then(|cell| cell.parent(res)).map(|c| c.into()); + + results.push(result); + } + + Ok(results.to_vector()) + } +} + +/// Function that checks if two cells are neighbour +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3IsNeighbour; + +impl Function for H3IsNeighbour { + fn name(&self) -> &str { + "h3_is_neighbour" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::boolean_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_double_cell() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 2, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 2, provided : {}", + columns.len() + ), + } + ); + + let cell_vec = &columns[0]; + let cell2_vec = &columns[1]; + let size = cell_vec.len(); + let mut results = BooleanVectorBuilder::with_capacity(size); + + for i in 0..size { + let result = match ( + cell_from_value(cell_vec.get(i))?, + cell_from_value(cell2_vec.get(i))?, + ) { + (Some(cell_this), Some(cell_that)) => { + let is_neighbour = cell_this + .is_neighbor_with(cell_that) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("H3 error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?; + Some(is_neighbour) + } + _ => None, + }; + + results.push(result); + } + + Ok(results.to_vector()) + } +} + +fn value_to_resolution(v: Value) -> Result { + let r = match v { + Value::Int8(v) => v as u8, + Value::Int16(v) => v as u8, + Value::Int32(v) => v as u8, + Value::Int64(v) => v as u8, + Value::UInt8(v) => v, + Value::UInt16(v) => v as u8, + Value::UInt32(v) => v as u8, + Value::UInt64(v) => v as u8, + _ => unreachable!(), + }; + Resolution::try_from(r) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("H3 error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu) +} + +fn signature_of_cell() -> Signature { + let mut signatures = Vec::new(); + for cell_type in &[ + ConcreteDataType::uint64_datatype(), + ConcreteDataType::int64_datatype(), + ] { + signatures.push(TypeSignature::Exact(vec![cell_type.clone()])); + } + + Signature::one_of(signatures, Volatility::Stable) +} + +fn signature_of_double_cell() -> Signature { + let mut signatures = Vec::new(); + let cell_types = &[ + ConcreteDataType::uint64_datatype(), + ConcreteDataType::int64_datatype(), + ]; + for cell_type in cell_types { + for cell_type2 in cell_types { + signatures.push(TypeSignature::Exact(vec![ + cell_type.clone(), + cell_type2.clone(), + ])); + } + } + + Signature::one_of(signatures, Volatility::Stable) +} + +fn signature_of_cell_and_resolution() -> Signature { + let mut signatures = Vec::new(); + for cell_type in &[ + ConcreteDataType::uint64_datatype(), + ConcreteDataType::int64_datatype(), + ] { + for resolution_type in &[ + ConcreteDataType::int8_datatype(), + ConcreteDataType::int16_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int64_datatype(), + ConcreteDataType::uint8_datatype(), + ConcreteDataType::uint16_datatype(), + ConcreteDataType::uint32_datatype(), + ConcreteDataType::uint64_datatype(), + ] { + signatures.push(TypeSignature::Exact(vec![ + cell_type.clone(), + resolution_type.clone(), + ])); + } + } + Signature::one_of(signatures, Volatility::Stable) +} + +fn cell_from_value(v: Value) -> Result> { + let cell = match v { + Value::Int64(v) => Some( + CellIndex::try_from(v as u64) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("H3 error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?, + ), + Value::UInt64(v) => Some( + CellIndex::try_from(v) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("H3 error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?, + ), + _ => None, + }; + Ok(cell) +} diff --git a/tests/cases/standalone/common/function/geo.result b/tests/cases/standalone/common/function/geo.result index 6d44c3ac04..3a63b5890b 100644 --- a/tests/cases/standalone/common/function/geo.result +++ b/tests/cases/standalone/common/function/geo.result @@ -1,98 +1,136 @@ -SELECT h3(37.76938, -122.3889, 0); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 0), h3_latlng_to_cell_string(37.76938, -122.3889, 0);; -+---------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),Int64(0)) | -+---------------------------------------------------+ -| 8029fffffffffff | -+---------------------------------------------------+ ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),Int64(0)) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),Int64(0)) | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| 577199624117288959 | 8029fffffffffff | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 1); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 1), h3_latlng_to_cell_string(37.76938, -122.3889, 1); -+---------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),Int64(1)) | -+---------------------------------------------------+ -| 81283ffffffffff | -+---------------------------------------------------+ ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),Int64(1)) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),Int64(1)) | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| 581672437419081727 | 81283ffffffffff | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8), h3_latlng_to_cell_string(37.76938, -122.3889, 8); -+---------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),Int64(8)) | -+---------------------------------------------------+ -| 88283082e7fffff | -+---------------------------------------------------+ ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),Int64(8)) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),Int64(8)) | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 100); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 100), h3_latlng_to_cell_string(37.76938, -122.3889, 100); Error: 3001(EngineExecuteQuery), H3 error: invalid resolution (got Some(100)): out of range -SELECT h3(37.76938, -122.3889, -1); +SELECT h3_latlng_to_cell(37.76938, -122.3889, -1), h3_latlng_to_cell_string(37.76938, -122.3889, -1); Error: 3001(EngineExecuteQuery), H3 error: invalid resolution (got Some(255)): out of range -SELECT h3(37.76938, -122.3889, 8::Int8); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int8), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int8); -+---------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),Int64(8)) | -+---------------------------------------------------+ -| 88283082e7fffff | -+---------------------------------------------------+ ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),Int64(8)) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),Int64(8)) | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++------------------------------------------------------------------+-------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8::Int16); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int16), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int16); -+-----------------------------------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int16"))) | -+-----------------------------------------------------------------------------+ -| 88283082e7fffff | -+-----------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int16"))) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int16"))) | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8::Int32); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int32), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int32); -+-----------------------------------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int32"))) | -+-----------------------------------------------------------------------------+ -| 88283082e7fffff | -+-----------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int32"))) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int32"))) | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8::Int64); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int64), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int64); -+-----------------------------------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int64"))) | -+-----------------------------------------------------------------------------+ -| 88283082e7fffff | -+-----------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int64"))) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("Int64"))) | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8::UInt8); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt8), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt8); -+-----------------------------------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt8"))) | -+-----------------------------------------------------------------------------+ -| 88283082e7fffff | -+-----------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt8"))) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt8"))) | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++--------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8::UInt16); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt16), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt8); -+------------------------------------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt16"))) | -+------------------------------------------------------------------------------+ -| 88283082e7fffff | -+------------------------------------------------------------------------------+ ++---------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt16"))) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt8"))) | ++---------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++---------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8::UInt32); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt32), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt32); -+------------------------------------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt32"))) | -+------------------------------------------------------------------------------+ -| 88283082e7fffff | -+------------------------------------------------------------------------------+ ++---------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt32"))) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt32"))) | ++---------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++---------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+ -SELECT h3(37.76938, -122.3889, 8::UInt64); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt64); -+------------------------------------------------------------------------------+ -| h3(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt64"))) | -+------------------------------------------------------------------------------+ -| 88283082e7fffff | -+------------------------------------------------------------------------------+ ++---------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+ +| h3_latlng_to_cell(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt64"))) | h3_latlng_to_cell_string(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(8),Utf8("UInt64"))) | ++---------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+ +| 613196570438926335 | 88283082e7fffff | ++---------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+ + +SELECT h3_cell_to_string(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_str, h3_string_to_cell(h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt64)) AS cell_index; + ++-----------------+--------------------+ +| cell_str | cell_index | ++-----------------+--------------------+ +| 88283082e7fffff | 613196570438926335 | ++-----------------+--------------------+ + +SELECT h3_cell_center_lat(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lat, h3_cell_center_lng(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lng; + ++-------------------+---------------------+ +| cell_center_lat | cell_center_lng | ++-------------------+---------------------+ +| 37.77246152245501 | -122.39010997087324 | ++-------------------+---------------------+ + +SELECT + h3_cell_resolution(cell) AS resolution, + h3_cell_base(cell) AS base, + h3_cell_is_pentagon(cell) AS pentagon, + h3_cell_parent(cell, 6::UInt64) AS parent, +FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell); + ++------------+------+----------+--------------------+ +| resolution | base | pentagon | parent | ++------------+------+----------+--------------------+ +| 8 | 20 | false | 604189371209351167 | ++------------+------+----------+--------------------+ + +SELECT h3_is_neighbour(cell1, cell2) +FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell1, h3_latlng_to_cell(36.76938, -122.3889, 8::UInt64) AS cell2); + ++------------------------------+ +| h3_is_neighbour(cell1,cell2) | ++------------------------------+ +| false | ++------------------------------+ SELECT geohash(37.76938, -122.3889, 9); diff --git a/tests/cases/standalone/common/function/geo.sql b/tests/cases/standalone/common/function/geo.sql index 8f6f70f4a4..3a0e668acc 100644 --- a/tests/cases/standalone/common/function/geo.sql +++ b/tests/cases/standalone/common/function/geo.sql @@ -1,28 +1,43 @@ -SELECT h3(37.76938, -122.3889, 0); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 0), h3_latlng_to_cell_string(37.76938, -122.3889, 0);; -SELECT h3(37.76938, -122.3889, 1); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 1), h3_latlng_to_cell_string(37.76938, -122.3889, 1); -SELECT h3(37.76938, -122.3889, 8); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8), h3_latlng_to_cell_string(37.76938, -122.3889, 8); -SELECT h3(37.76938, -122.3889, 100); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 100), h3_latlng_to_cell_string(37.76938, -122.3889, 100); -SELECT h3(37.76938, -122.3889, -1); +SELECT h3_latlng_to_cell(37.76938, -122.3889, -1), h3_latlng_to_cell_string(37.76938, -122.3889, -1); -SELECT h3(37.76938, -122.3889, 8::Int8); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int8), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int8); -SELECT h3(37.76938, -122.3889, 8::Int16); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int16), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int16); -SELECT h3(37.76938, -122.3889, 8::Int32); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int32), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int32); -SELECT h3(37.76938, -122.3889, 8::Int64); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::Int64), h3_latlng_to_cell_string(37.76938, -122.3889, 8::Int64); -SELECT h3(37.76938, -122.3889, 8::UInt8); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt8), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt8); -SELECT h3(37.76938, -122.3889, 8::UInt16); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt16), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt8); -SELECT h3(37.76938, -122.3889, 8::UInt32); +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt32), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt32); + +SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64), h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt64); + +SELECT h3_cell_to_string(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_str, h3_string_to_cell(h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt64)) AS cell_index; + +SELECT h3_cell_center_lat(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lat, h3_cell_center_lng(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lng; + +SELECT + h3_cell_resolution(cell) AS resolution, + h3_cell_base(cell) AS base, + h3_cell_is_pentagon(cell) AS pentagon, + h3_cell_parent(cell, 6::UInt64) AS parent, +FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell); + +SELECT h3_is_neighbour(cell1, cell2) +FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell1, h3_latlng_to_cell(36.76938, -122.3889, 8::UInt64) AS cell2); -SELECT h3(37.76938, -122.3889, 8::UInt64); SELECT geohash(37.76938, -122.3889, 9); From 0c9b8eb0d2ff6dcb6c6c5f78aae3a3336f2090d7 Mon Sep 17 00:00:00 2001 From: taobo Date: Fri, 20 Sep 2024 14:07:53 +0800 Subject: [PATCH 027/128] feat: improve observability for procedure (#4675) * feat: improve observability for procedure * fix: test error * test: add sqlness test for information_schema.procedure_info * fix: sqlness test error * fix: cr comment * chore: update proto version * fix: apply cr comment * update version * fix: cr comment * optimize procedure type output format * upgrade dep version * fix: clippy error * fix: `procedure` borrowed error * fix: optimize code --- Cargo.lock | 4 +- Cargo.toml | 2 +- src/catalog/Cargo.toml | 1 + src/catalog/src/error.rs | 35 +- src/catalog/src/kvbackend/manager.rs | 8 + .../src/system_schema/information_schema.rs | 11 +- .../information_schema/procedure_info.rs | 310 ++++++++++++++++++ .../information_schema/table_names.rs | 1 + src/catalog/src/system_schema/utils.rs | 15 + src/catalog/src/table_source.rs | 1 + src/cmd/src/cli/repl.rs | 1 + src/cmd/src/flownode.rs | 1 + src/cmd/src/frontend.rs | 1 + src/cmd/src/standalone.rs | 1 + src/common/catalog/src/consts.rs | 2 + src/common/meta/src/ddl.rs | 3 + src/common/meta/src/ddl_manager.rs | 10 + src/common/meta/src/rpc/procedure.rs | 39 ++- src/common/procedure/Cargo.toml | 1 + src/common/procedure/src/lib.rs | 4 +- src/common/procedure/src/local.rs | 48 ++- src/common/procedure/src/local/runner.rs | 7 +- src/common/procedure/src/procedure.rs | 44 +++ src/meta-client/src/client.rs | 12 +- src/meta-client/src/client/procedure.rs | 29 +- src/meta-srv/src/service/procedure.rs | 18 +- tests-integration/src/cluster.rs | 1 + tests-integration/src/standalone.rs | 1 + .../information_schema/procedure_info.result | 40 +++ .../information_schema/procedure_info.sql | 20 ++ .../common/show/show_databases_tables.result | 3 + .../common/system/information_schema.result | 7 + .../standalone/common/view/create.result | 1 + 33 files changed, 662 insertions(+), 20 deletions(-) create mode 100644 src/catalog/src/system_schema/information_schema/procedure_info.rs create mode 100644 tests/cases/standalone/common/information_schema/procedure_info.result create mode 100644 tests/cases/standalone/common/information_schema/procedure_info.sql diff --git a/Cargo.lock b/Cargo.lock index e7f180eb17..2183cf593d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1379,6 +1379,7 @@ dependencies = [ "common-error", "common-macro", "common-meta", + "common-procedure", "common-query", "common-recordbatch", "common-runtime", @@ -2179,6 +2180,7 @@ dependencies = [ "common-runtime", "common-telemetry", "common-test-util", + "common-time", "futures", "futures-util", "humantime-serde", @@ -4394,7 +4396,7 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "greptime-proto" version = "0.1.0" -source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=973f49cde88a582fb65755cc572ebcf6fb93ccf7#973f49cde88a582fb65755cc572ebcf6fb93ccf7" +source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=796ce9b003c6689e853825f649e03543c81ede99#796ce9b003c6689e853825f649e03543c81ede99" dependencies = [ "prost 0.12.6", "serde", diff --git a/Cargo.toml b/Cargo.toml index 25cd5b5dd2..df70b98332 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -120,7 +120,7 @@ etcd-client = { version = "0.13" } fst = "0.4.7" futures = "0.3" futures-util = "0.3" -greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "973f49cde88a582fb65755cc572ebcf6fb93ccf7" } +greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "796ce9b003c6689e853825f649e03543c81ede99" } humantime = "2.1" humantime-serde = "1.1" itertools = "0.10" diff --git a/src/catalog/Cargo.toml b/src/catalog/Cargo.toml index cfea5e02c6..a5ad92e891 100644 --- a/src/catalog/Cargo.toml +++ b/src/catalog/Cargo.toml @@ -22,6 +22,7 @@ common-config.workspace = true common-error.workspace = true common-macro.workspace = true common-meta.workspace = true +common-procedure.workspace = true common-query.workspace = true common-recordbatch.workspace = true common-runtime.workspace = true diff --git a/src/catalog/src/error.rs b/src/catalog/src/error.rs index d44a5b7683..14748a4695 100644 --- a/src/catalog/src/error.rs +++ b/src/catalog/src/error.rs @@ -82,6 +82,33 @@ pub enum Error { location: Location, }, + #[snafu(display("Failed to get procedure client in {mode} mode"))] + GetProcedureClient { + mode: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to list procedures"))] + ListProcedures { + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + + #[snafu(display("Procedure id not found"))] + ProcedureIdNotFound { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("convert proto data error"))] + ConvertProtoData { + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Failed to re-compile script due to internal error"))] CompileScriptInternal { #[snafu(implicit)] @@ -266,7 +293,9 @@ impl ErrorExt for Error { | Error::FindRegionRoutes { .. } | Error::CacheNotFound { .. } | Error::CastManager { .. } - | Error::Json { .. } => StatusCode::Unexpected, + | Error::Json { .. } + | Error::GetProcedureClient { .. } + | Error::ProcedureIdNotFound { .. } => StatusCode::Unexpected, Error::ViewPlanColumnsChanged { .. } => StatusCode::InvalidArguments, @@ -283,7 +312,9 @@ impl ErrorExt for Error { | Error::ListNodes { source, .. } | Error::ListSchemas { source, .. } | Error::ListTables { source, .. } - | Error::ListFlows { source, .. } => source.status_code(), + | Error::ListFlows { source, .. } + | Error::ListProcedures { source, .. } + | Error::ConvertProtoData { source, .. } => source.status_code(), Error::CreateTable { source, .. } => source.status_code(), diff --git a/src/catalog/src/kvbackend/manager.rs b/src/catalog/src/kvbackend/manager.rs index 1559022514..2495a95255 100644 --- a/src/catalog/src/kvbackend/manager.rs +++ b/src/catalog/src/kvbackend/manager.rs @@ -31,6 +31,7 @@ use common_meta::key::table_info::TableInfoValue; use common_meta::key::table_name::TableNameKey; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; use common_meta::kv_backend::KvBackendRef; +use common_procedure::ProcedureManagerRef; use futures_util::stream::BoxStream; use futures_util::{StreamExt, TryStreamExt}; use meta_client::client::MetaClient; @@ -69,6 +70,7 @@ pub struct KvBackendCatalogManager { /// A sub-CatalogManager that handles system tables system_catalog: SystemCatalog, cache_registry: LayeredCacheRegistryRef, + procedure_manager: Option, } const CATALOG_CACHE_MAX_CAPACITY: u64 = 128; @@ -79,6 +81,7 @@ impl KvBackendCatalogManager { meta_client: Option>, backend: KvBackendRef, cache_registry: LayeredCacheRegistryRef, + procedure_manager: Option, ) -> Arc { Arc::new_cyclic(|me| Self { mode, @@ -106,6 +109,7 @@ impl KvBackendCatalogManager { backend, }, cache_registry, + procedure_manager, }) } @@ -132,6 +136,10 @@ impl KvBackendCatalogManager { pub fn table_metadata_manager_ref(&self) -> &TableMetadataManagerRef { &self.table_metadata_manager } + + pub fn procedure_manager(&self) -> Option { + self.procedure_manager.clone() + } } #[async_trait::async_trait] diff --git a/src/catalog/src/system_schema/information_schema.rs b/src/catalog/src/system_schema/information_schema.rs index 93dfaa75b5..83f2ff4926 100644 --- a/src/catalog/src/system_schema/information_schema.rs +++ b/src/catalog/src/system_schema/information_schema.rs @@ -18,6 +18,7 @@ pub mod flows; mod information_memory_table; pub mod key_column_usage; mod partitions; +mod procedure_info; mod region_peers; mod runtime_metrics; pub mod schemata; @@ -188,6 +189,11 @@ impl SystemSchemaProviderInner for InformationSchemaProvider { self.catalog_name.clone(), self.flow_metadata_manager.clone(), )) as _), + PROCEDURE_INFO => Some( + Arc::new(procedure_info::InformationSchemaProcedureInfo::new( + self.catalog_manager.clone(), + )) as _, + ), _ => None, } } @@ -250,7 +256,10 @@ impl InformationSchemaProvider { self.build_table(TABLE_CONSTRAINTS).unwrap(), ); tables.insert(FLOWS.to_string(), self.build_table(FLOWS).unwrap()); - + tables.insert( + PROCEDURE_INFO.to_string(), + self.build_table(PROCEDURE_INFO).unwrap(), + ); // Add memory tables for name in MEMORY_TABLES.iter() { tables.insert((*name).to_string(), self.build_table(name).expect(name)); diff --git a/src/catalog/src/system_schema/information_schema/procedure_info.rs b/src/catalog/src/system_schema/information_schema/procedure_info.rs new file mode 100644 index 0000000000..56c36c2210 --- /dev/null +++ b/src/catalog/src/system_schema/information_schema/procedure_info.rs @@ -0,0 +1,310 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::{Arc, Weak}; + +use api::v1::meta::{ProcedureMeta, ProcedureStatus}; +use arrow_schema::SchemaRef as ArrowSchemaRef; +use common_catalog::consts::INFORMATION_SCHEMA_PROCEDURE_INFO_TABLE_ID; +use common_config::Mode; +use common_error::ext::BoxedError; +use common_meta::ddl::{ExecutorContext, ProcedureExecutor}; +use common_meta::rpc::procedure; +use common_procedure::{ProcedureInfo, ProcedureState}; +use common_recordbatch::adapter::RecordBatchStreamAdapter; +use common_recordbatch::{RecordBatch, SendableRecordBatchStream}; +use common_time::timestamp::Timestamp; +use datafusion::execution::TaskContext; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter; +use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream; +use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream; +use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef}; +use datatypes::schema::{ColumnSchema, Schema, SchemaRef}; +use datatypes::timestamp::TimestampMillisecond; +use datatypes::value::Value; +use datatypes::vectors::{StringVectorBuilder, TimestampMillisecondVectorBuilder}; +use snafu::ResultExt; +use store_api::storage::{ScanRequest, TableId}; + +use super::PROCEDURE_INFO; +use crate::error::{ + ConvertProtoDataSnafu, CreateRecordBatchSnafu, GetProcedureClientSnafu, InternalSnafu, + ListProceduresSnafu, ProcedureIdNotFoundSnafu, Result, +}; +use crate::system_schema::information_schema::{InformationTable, Predicates}; +use crate::system_schema::utils; +use crate::CatalogManager; + +const PROCEDURE_ID: &str = "procedure_id"; +const PROCEDURE_TYPE: &str = "procedure_type"; +const START_TIME: &str = "start_time"; +const END_TIME: &str = "end_time"; +const STATUS: &str = "status"; +const LOCK_KEYS: &str = "lock_keys"; + +const INIT_CAPACITY: usize = 42; + +/// The `PROCEDURE_INFO` table provides information about the current procedure information of the cluster. +/// +/// - `procedure_id`: the unique identifier of the procedure. +/// - `procedure_name`: the name of the procedure. +/// - `start_time`: the starting execution time of the procedure. +/// - `end_time`: the ending execution time of the procedure. +/// - `status`: the status of the procedure. +/// - `lock_keys`: the lock keys of the procedure. +/// +pub(super) struct InformationSchemaProcedureInfo { + schema: SchemaRef, + catalog_manager: Weak, +} + +impl InformationSchemaProcedureInfo { + pub(super) fn new(catalog_manager: Weak) -> Self { + Self { + schema: Self::schema(), + catalog_manager, + } + } + + pub(crate) fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + ColumnSchema::new(PROCEDURE_ID, ConcreteDataType::string_datatype(), false), + ColumnSchema::new(PROCEDURE_TYPE, ConcreteDataType::string_datatype(), false), + ColumnSchema::new( + START_TIME, + ConcreteDataType::timestamp_millisecond_datatype(), + true, + ), + ColumnSchema::new( + END_TIME, + ConcreteDataType::timestamp_millisecond_datatype(), + true, + ), + ColumnSchema::new(STATUS, ConcreteDataType::string_datatype(), false), + ColumnSchema::new(LOCK_KEYS, ConcreteDataType::string_datatype(), true), + ])) + } + + fn builder(&self) -> InformationSchemaProcedureInfoBuilder { + InformationSchemaProcedureInfoBuilder::new( + self.schema.clone(), + self.catalog_manager.clone(), + ) + } +} + +impl InformationTable for InformationSchemaProcedureInfo { + fn table_id(&self) -> TableId { + INFORMATION_SCHEMA_PROCEDURE_INFO_TABLE_ID + } + + fn table_name(&self) -> &'static str { + PROCEDURE_INFO + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn to_stream(&self, request: ScanRequest) -> Result { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + let stream = Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_procedure_info(Some(request)) + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )); + Ok(Box::pin( + RecordBatchStreamAdapter::try_new(stream) + .map_err(BoxedError::new) + .context(InternalSnafu)?, + )) + } +} + +struct InformationSchemaProcedureInfoBuilder { + schema: SchemaRef, + catalog_manager: Weak, + + procedure_ids: StringVectorBuilder, + procedure_types: StringVectorBuilder, + start_times: TimestampMillisecondVectorBuilder, + end_times: TimestampMillisecondVectorBuilder, + statuses: StringVectorBuilder, + lock_keys: StringVectorBuilder, +} + +impl InformationSchemaProcedureInfoBuilder { + fn new(schema: SchemaRef, catalog_manager: Weak) -> Self { + Self { + schema, + catalog_manager, + procedure_ids: StringVectorBuilder::with_capacity(INIT_CAPACITY), + procedure_types: StringVectorBuilder::with_capacity(INIT_CAPACITY), + start_times: TimestampMillisecondVectorBuilder::with_capacity(INIT_CAPACITY), + end_times: TimestampMillisecondVectorBuilder::with_capacity(INIT_CAPACITY), + statuses: StringVectorBuilder::with_capacity(INIT_CAPACITY), + lock_keys: StringVectorBuilder::with_capacity(INIT_CAPACITY), + } + } + + /// Construct the `information_schema.procedure_info` virtual table + async fn make_procedure_info(&mut self, request: Option) -> Result { + let predicates = Predicates::from_scan_request(&request); + let mode = utils::running_mode(&self.catalog_manager)?.unwrap_or(Mode::Standalone); + match mode { + Mode::Standalone => { + if let Some(procedure_manager) = utils::procedure_manager(&self.catalog_manager)? { + let procedures = procedure_manager + .list_procedures() + .await + .map_err(BoxedError::new) + .context(ListProceduresSnafu)?; + for procedure in procedures { + self.add_procedure( + &predicates, + procedure.state.as_str_name().to_string(), + procedure, + ); + } + } else { + return GetProcedureClientSnafu { mode: "standalone" }.fail(); + } + } + Mode::Distributed => { + if let Some(meta_client) = utils::meta_client(&self.catalog_manager)? { + let procedures = meta_client + .list_procedures(&ExecutorContext::default()) + .await + .map_err(BoxedError::new) + .context(ListProceduresSnafu)?; + for procedure in procedures.procedures { + self.add_procedure_info(&predicates, procedure)?; + } + } else { + return GetProcedureClientSnafu { + mode: "distributed", + } + .fail(); + } + } + }; + + self.finish() + } + + fn add_procedure( + &mut self, + predicates: &Predicates, + status: String, + procedure_info: ProcedureInfo, + ) { + let ProcedureInfo { + id, + type_name, + start_time_ms, + end_time_ms, + lock_keys, + .. + } = procedure_info; + let pid = id.to_string(); + let start_time = TimestampMillisecond(Timestamp::new_millisecond(start_time_ms)); + let end_time = TimestampMillisecond(Timestamp::new_millisecond(end_time_ms)); + let lock_keys = lock_keys.join(","); + + let row = [ + (PROCEDURE_ID, &Value::from(pid.clone())), + (PROCEDURE_TYPE, &Value::from(type_name.clone())), + (START_TIME, &Value::from(start_time)), + (END_TIME, &Value::from(end_time)), + (STATUS, &Value::from(status.clone())), + (LOCK_KEYS, &Value::from(lock_keys.clone())), + ]; + if !predicates.eval(&row) { + return; + } + self.procedure_ids.push(Some(&pid)); + self.procedure_types.push(Some(&type_name)); + self.start_times.push(Some(start_time)); + self.end_times.push(Some(end_time)); + self.statuses.push(Some(&status)); + self.lock_keys.push(Some(&lock_keys)); + } + + fn add_procedure_info( + &mut self, + predicates: &Predicates, + procedure: ProcedureMeta, + ) -> Result<()> { + let pid = match procedure.id { + Some(pid) => pid, + None => return ProcedureIdNotFoundSnafu {}.fail(), + }; + let pid = procedure::pb_pid_to_pid(&pid) + .map_err(BoxedError::new) + .context(ConvertProtoDataSnafu)?; + let status = ProcedureStatus::try_from(procedure.status) + .map(|v| v.as_str_name()) + .unwrap_or("Unknown") + .to_string(); + let procedure_info = ProcedureInfo { + id: pid, + type_name: procedure.type_name, + start_time_ms: procedure.start_time_ms, + end_time_ms: procedure.end_time_ms, + state: ProcedureState::Running, + lock_keys: procedure.lock_keys, + }; + self.add_procedure(predicates, status, procedure_info); + Ok(()) + } + + fn finish(&mut self) -> Result { + let columns: Vec = vec![ + Arc::new(self.procedure_ids.finish()), + Arc::new(self.procedure_types.finish()), + Arc::new(self.start_times.finish()), + Arc::new(self.end_times.finish()), + Arc::new(self.statuses.finish()), + Arc::new(self.lock_keys.finish()), + ]; + RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu) + } +} + +impl DfPartitionStream for InformationSchemaProcedureInfo { + fn schema(&self) -> &ArrowSchemaRef { + self.schema.arrow_schema() + } + + fn execute(&self, _: Arc) -> DfSendableRecordBatchStream { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_procedure_info(None) + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )) + } +} diff --git a/src/catalog/src/system_schema/information_schema/table_names.rs b/src/catalog/src/system_schema/information_schema/table_names.rs index c2c9eeff24..a62f4ddb40 100644 --- a/src/catalog/src/system_schema/information_schema/table_names.rs +++ b/src/catalog/src/system_schema/information_schema/table_names.rs @@ -45,3 +45,4 @@ pub const TABLE_CONSTRAINTS: &str = "table_constraints"; pub const CLUSTER_INFO: &str = "cluster_info"; pub const VIEWS: &str = "views"; pub const FLOWS: &str = "flows"; +pub const PROCEDURE_INFO: &str = "procedure_info"; diff --git a/src/catalog/src/system_schema/utils.rs b/src/catalog/src/system_schema/utils.rs index 2a2e89516a..b9786bc260 100644 --- a/src/catalog/src/system_schema/utils.rs +++ b/src/catalog/src/system_schema/utils.rs @@ -18,6 +18,7 @@ use std::sync::{Arc, Weak}; use common_config::Mode; use common_meta::key::TableMetadataManagerRef; +use common_procedure::ProcedureManagerRef; use meta_client::client::MetaClient; use snafu::OptionExt; @@ -68,3 +69,17 @@ pub fn table_meta_manager( .downcast_ref::() .map(|manager| manager.table_metadata_manager_ref().clone())) } + +/// Try to get the `[ProcedureManagerRef]` from `[CatalogManager]` weak reference. +pub fn procedure_manager( + catalog_manager: &Weak, +) -> Result> { + let catalog_manager = catalog_manager + .upgrade() + .context(UpgradeWeakCatalogManagerRefSnafu)?; + + Ok(catalog_manager + .as_any() + .downcast_ref::() + .and_then(|manager| manager.procedure_manager())) +} diff --git a/src/catalog/src/table_source.rs b/src/catalog/src/table_source.rs index d6d81fa134..09d3d9d2fd 100644 --- a/src/catalog/src/table_source.rs +++ b/src/catalog/src/table_source.rs @@ -327,6 +327,7 @@ mod tests { None, backend.clone(), layered_cache_registry, + None, ); let table_metadata_manager = TableMetadataManager::new(backend); let mut view_info = common_meta::key::test_utils::new_test_table_info(1024, vec![]); diff --git a/src/cmd/src/cli/repl.rs b/src/cmd/src/cli/repl.rs index 3fc2184d37..b55b1c44d6 100644 --- a/src/cmd/src/cli/repl.rs +++ b/src/cmd/src/cli/repl.rs @@ -280,6 +280,7 @@ async fn create_query_engine(meta_addr: &str) -> Result { Some(meta_client.clone()), cached_meta_backend.clone(), layered_cache_registry, + None, ); let plugins: Plugins = Default::default(); let state = Arc::new(QueryEngineState::new( diff --git a/src/cmd/src/flownode.rs b/src/cmd/src/flownode.rs index e950d9bb43..78dfc90607 100644 --- a/src/cmd/src/flownode.rs +++ b/src/cmd/src/flownode.rs @@ -274,6 +274,7 @@ impl StartCommand { Some(meta_client.clone()), cached_meta_backend.clone(), layered_cache_registry.clone(), + None, ); let table_metadata_manager = diff --git a/src/cmd/src/frontend.rs b/src/cmd/src/frontend.rs index 6d8fd97070..320dc49c19 100644 --- a/src/cmd/src/frontend.rs +++ b/src/cmd/src/frontend.rs @@ -320,6 +320,7 @@ impl StartCommand { Some(meta_client.clone()), cached_meta_backend.clone(), layered_cache_registry.clone(), + None, ); let executor = HandlerGroupExecutor::new(vec![ diff --git a/src/cmd/src/standalone.rs b/src/cmd/src/standalone.rs index c8083c5f80..80b38ebaa7 100644 --- a/src/cmd/src/standalone.rs +++ b/src/cmd/src/standalone.rs @@ -482,6 +482,7 @@ impl StartCommand { None, kv_backend.clone(), layered_cache_registry.clone(), + Some(procedure_manager.clone()), ); let table_metadata_manager = diff --git a/src/common/catalog/src/consts.rs b/src/common/catalog/src/consts.rs index 6020a4bef8..2a8e2fc0e4 100644 --- a/src/common/catalog/src/consts.rs +++ b/src/common/catalog/src/consts.rs @@ -98,6 +98,8 @@ pub const INFORMATION_SCHEMA_CLUSTER_INFO_TABLE_ID: u32 = 31; pub const INFORMATION_SCHEMA_VIEW_TABLE_ID: u32 = 32; /// id for information_schema.FLOWS pub const INFORMATION_SCHEMA_FLOW_TABLE_ID: u32 = 33; +/// id for information_schema.procedure_info +pub const INFORMATION_SCHEMA_PROCEDURE_INFO_TABLE_ID: u32 = 34; /// ----- End of information_schema tables ----- /// ----- Begin of pg_catalog tables ----- diff --git a/src/common/meta/src/ddl.rs b/src/common/meta/src/ddl.rs index 7186997906..11654f04d6 100644 --- a/src/common/meta/src/ddl.rs +++ b/src/common/meta/src/ddl.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; +use api::v1::meta::ProcedureDetailResponse; use common_telemetry::tracing_context::W3cTrace; use store_api::storage::{RegionId, RegionNumber, TableId}; @@ -82,6 +83,8 @@ pub trait ProcedureExecutor: Send + Sync { ctx: &ExecutorContext, pid: &str, ) -> Result; + + async fn list_procedures(&self, ctx: &ExecutorContext) -> Result; } pub type ProcedureExecutorRef = Arc; diff --git a/src/common/meta/src/ddl_manager.rs b/src/common/meta/src/ddl_manager.rs index 152a4631e2..1ee148406d 100644 --- a/src/common/meta/src/ddl_manager.rs +++ b/src/common/meta/src/ddl_manager.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use api::v1::meta::ProcedureDetailResponse; use common_procedure::{ watcher, BoxedProcedureLoader, Output, ProcedureId, ProcedureManagerRef, ProcedureWithId, }; @@ -825,6 +826,15 @@ impl ProcedureExecutor for DdlManager { Ok(procedure::procedure_state_to_pb_response(&state)) } + + async fn list_procedures(&self, _ctx: &ExecutorContext) -> Result { + let metas = self + .procedure_manager + .list_procedures() + .await + .context(QueryProcedureSnafu)?; + Ok(procedure::procedure_details_to_pb_response(metas)) + } } #[cfg(test)] diff --git a/src/common/meta/src/rpc/procedure.rs b/src/common/meta/src/rpc/procedure.rs index 6dabe899a8..2c2a69b5b6 100644 --- a/src/common/meta/src/rpc/procedure.rs +++ b/src/common/meta/src/rpc/procedure.rs @@ -16,10 +16,11 @@ use std::time::Duration; pub use api::v1::meta::{MigrateRegionResponse, ProcedureStateResponse}; use api::v1::meta::{ - ProcedureId as PbProcedureId, ProcedureStateResponse as PbProcedureStateResponse, + ProcedureDetailResponse as PbProcedureDetailResponse, ProcedureId as PbProcedureId, + ProcedureMeta as PbProcedureMeta, ProcedureStateResponse as PbProcedureStateResponse, ProcedureStatus as PbProcedureStatus, }; -use common_procedure::{ProcedureId, ProcedureState}; +use common_procedure::{ProcedureId, ProcedureInfo, ProcedureState}; use snafu::ResultExt; use crate::error::{ParseProcedureIdSnafu, Result}; @@ -49,9 +50,9 @@ pub fn pid_to_pb_pid(pid: ProcedureId) -> PbProcedureId { } } -/// Cast the common [`ProcedureState`] to pb [`ProcedureStateResponse`]. -pub fn procedure_state_to_pb_response(state: &ProcedureState) -> PbProcedureStateResponse { - let (status, error) = match state { +/// Cast the [`ProcedureState`] to protobuf [`PbProcedureStatus`]. +pub fn procedure_state_to_pb_state(state: &ProcedureState) -> (PbProcedureStatus, String) { + match state { ProcedureState::Running => (PbProcedureStatus::Running, String::default()), ProcedureState::Done { .. } => (PbProcedureStatus::Done, String::default()), ProcedureState::Retrying { error } => (PbProcedureStatus::Retrying, error.to_string()), @@ -62,8 +63,12 @@ pub fn procedure_state_to_pb_response(state: &ProcedureState) -> PbProcedureStat ProcedureState::RollingBack { error } => { (PbProcedureStatus::RollingBack, error.to_string()) } - }; + } +} +/// Cast the common [`ProcedureState`] to pb [`ProcedureStateResponse`]. +pub fn procedure_state_to_pb_response(state: &ProcedureState) -> PbProcedureStateResponse { + let (status, error) = procedure_state_to_pb_state(state); PbProcedureStateResponse { status: status.into(), error, @@ -71,6 +76,28 @@ pub fn procedure_state_to_pb_response(state: &ProcedureState) -> PbProcedureStat } } +pub fn procedure_details_to_pb_response(metas: Vec) -> PbProcedureDetailResponse { + let procedures = metas + .into_iter() + .map(|meta| { + let (status, error) = procedure_state_to_pb_state(&meta.state); + PbProcedureMeta { + id: Some(pid_to_pb_pid(meta.id)), + type_name: meta.type_name.to_string(), + status: status.into(), + start_time_ms: meta.start_time_ms, + end_time_ms: meta.end_time_ms, + lock_keys: meta.lock_keys, + error, + } + }) + .collect(); + PbProcedureDetailResponse { + procedures, + ..Default::default() + } +} + #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/src/common/procedure/Cargo.toml b/src/common/procedure/Cargo.toml index b56aa46a91..1d8c6736e3 100644 --- a/src/common/procedure/Cargo.toml +++ b/src/common/procedure/Cargo.toml @@ -19,6 +19,7 @@ common-error.workspace = true common-macro.workspace = true common-runtime.workspace = true common-telemetry.workspace = true +common-time.workspace = true futures.workspace = true humantime-serde.workspace = true object-store.workspace = true diff --git a/src/common/procedure/src/lib.rs b/src/common/procedure/src/lib.rs index ece2ce4189..269ad2c529 100644 --- a/src/common/procedure/src/lib.rs +++ b/src/common/procedure/src/lib.rs @@ -26,7 +26,7 @@ pub mod watcher; pub use crate::error::{Error, Result}; pub use crate::procedure::{ BoxedProcedure, BoxedProcedureLoader, Context, ContextProvider, LockKey, Output, ParseIdError, - Procedure, ProcedureId, ProcedureManager, ProcedureManagerRef, ProcedureState, ProcedureWithId, - Status, StringKey, + Procedure, ProcedureId, ProcedureInfo, ProcedureManager, ProcedureManagerRef, ProcedureState, + ProcedureWithId, Status, StringKey, }; pub use crate::watcher::Watcher; diff --git a/src/common/procedure/src/local.rs b/src/common/procedure/src/local.rs index 63fc06270a..da456fb37a 100644 --- a/src/common/procedure/src/local.rs +++ b/src/common/procedure/src/local.rs @@ -16,7 +16,7 @@ mod runner; mod rwlock; use std::collections::{HashMap, VecDeque}; -use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::atomic::{AtomicBool, AtomicI64, Ordering}; use std::sync::{Arc, Mutex, RwLock}; use std::time::{Duration, Instant}; @@ -35,7 +35,7 @@ use crate::error::{ StartRemoveOutdatedMetaTaskSnafu, StopRemoveOutdatedMetaTaskSnafu, }; use crate::local::runner::Runner; -use crate::procedure::{BoxedProcedureLoader, InitProcedureState}; +use crate::procedure::{BoxedProcedureLoader, InitProcedureState, ProcedureInfo}; use crate::store::{ProcedureMessage, ProcedureMessages, ProcedureStore, StateStoreRef}; use crate::{ BoxedProcedure, ContextProvider, LockKey, ProcedureId, ProcedureManager, ProcedureState, @@ -57,6 +57,8 @@ const META_TTL: Duration = Duration::from_secs(60 * 10); pub(crate) struct ProcedureMeta { /// Id of this procedure. id: ProcedureId, + /// Type name of this procedure. + type_name: String, /// Parent procedure id. parent_id: Option, /// Notify to wait for subprocedures. @@ -69,6 +71,10 @@ pub(crate) struct ProcedureMeta { state_receiver: Receiver, /// Id of child procedures. children: Mutex>, + /// Start execution time of this procedure. + start_time_ms: AtomicI64, + /// End execution time of this procedure. + end_time_ms: AtomicI64, } impl ProcedureMeta { @@ -77,6 +83,7 @@ impl ProcedureMeta { procedure_state: ProcedureState, parent_id: Option, lock_key: LockKey, + type_name: &str, ) -> ProcedureMeta { let (state_sender, state_receiver) = watch::channel(procedure_state); ProcedureMeta { @@ -87,6 +94,9 @@ impl ProcedureMeta { state_sender, state_receiver, children: Mutex::new(Vec::new()), + start_time_ms: AtomicI64::new(0), + end_time_ms: AtomicI64::new(0), + type_name: type_name.to_string(), } } @@ -117,6 +127,18 @@ impl ProcedureMeta { fn num_children(&self) -> usize { self.children.lock().unwrap().len() } + + /// update the start time of the procedure. + fn set_start_time_ms(&self) { + self.start_time_ms + .store(common_time::util::current_time_millis(), Ordering::Relaxed); + } + + /// update the end time of the procedure. + fn set_end_time_ms(&self) { + self.end_time_ms + .store(common_time::util::current_time_millis(), Ordering::Relaxed); + } } /// Reference counted pointer to [ProcedureMeta]. @@ -210,6 +232,22 @@ impl ManagerContext { procedures.get(&procedure_id).map(|meta| meta.state()) } + /// Returns the [ProcedureMeta] of all procedures. + fn list_procedure(&self) -> Vec { + let procedures = self.procedures.read().unwrap(); + procedures + .values() + .map(|meta| ProcedureInfo { + id: meta.id, + type_name: meta.type_name.clone(), + start_time_ms: meta.start_time_ms.load(Ordering::Relaxed), + end_time_ms: meta.end_time_ms.load(Ordering::Relaxed), + state: meta.state(), + lock_keys: meta.lock_key.get_keys(), + }) + .collect() + } + /// Returns the [Watcher] of specific `procedure_id`. fn watcher(&self, procedure_id: ProcedureId) -> Option { let procedures = self.procedures.read().unwrap(); @@ -438,6 +476,7 @@ impl LocalManager { procedure_state, None, procedure.lock_key(), + procedure.type_name(), )); let runner = Runner { meta: meta.clone(), @@ -641,6 +680,10 @@ impl ProcedureManager for LocalManager { fn procedure_watcher(&self, procedure_id: ProcedureId) -> Option { self.manager_ctx.watcher(procedure_id) } + + async fn list_procedures(&self) -> Result> { + Ok(self.manager_ctx.list_procedure()) + } } struct RemoveOutdatedMetaFunction { @@ -675,6 +718,7 @@ pub(crate) mod test_util { ProcedureState::Running, None, LockKey::default(), + "ProcedureAdapter", ) } diff --git a/src/common/procedure/src/local/runner.rs b/src/common/procedure/src/local/runner.rs index 2f38ae135d..c2d15001fb 100644 --- a/src/common/procedure/src/local/runner.rs +++ b/src/common/procedure/src/local/runner.rs @@ -27,7 +27,9 @@ use crate::error::{self, ProcedurePanicSnafu, Result, RollbackTimesExceededSnafu use crate::local::{ManagerContext, ProcedureMeta, ProcedureMetaRef}; use crate::procedure::{Output, StringKey}; use crate::store::{ProcedureMessage, ProcedureStore}; -use crate::{BoxedProcedure, Context, Error, ProcedureId, ProcedureState, ProcedureWithId, Status}; +use crate::{ + BoxedProcedure, Context, Error, Procedure, ProcedureId, ProcedureState, ProcedureWithId, Status, +}; /// A guard to cleanup procedure state. struct ProcedureGuard { @@ -129,7 +131,9 @@ impl Runner { // Execute the procedure. We need to release the lock whenever the execution // is successful or fail. + self.meta.set_start_time_ms(); self.execute_procedure_in_loop().await; + self.meta.set_end_time_ms(); // We can't remove the metadata of the procedure now as users and its parent might // need to query its state. @@ -368,6 +372,7 @@ impl Runner { procedure_state, Some(self.meta.id), procedure.lock_key(), + procedure.type_name(), )); let runner = Runner { meta: meta.clone(), diff --git a/src/common/procedure/src/procedure.rs b/src/common/procedure/src/procedure.rs index 6c694315e9..ddf5dc74a3 100644 --- a/src/common/procedure/src/procedure.rs +++ b/src/common/procedure/src/procedure.rs @@ -159,6 +159,14 @@ impl Procedure for Box { (**self).execute(ctx).await } + async fn rollback(&mut self, ctx: &Context) -> Result<()> { + (**self).rollback(ctx).await + } + + fn rollback_supported(&self) -> bool { + (**self).rollback_supported() + } + fn dump(&self) -> Result { (**self).dump() } @@ -227,6 +235,11 @@ impl LockKey { pub fn keys_to_lock(&self) -> impl Iterator { self.0.iter() } + + /// Returns the keys to lock. + pub fn get_keys(&self) -> Vec { + self.0.iter().map(|key| format!("{:?}", key)).collect() + } } /// Boxed [Procedure]. @@ -374,6 +387,18 @@ impl ProcedureState { _ => None, } } + + /// Return the string values of the enum field names. + pub fn as_str_name(&self) -> &str { + match self { + ProcedureState::Running => "Running", + ProcedureState::Done { .. } => "Done", + ProcedureState::Retrying { .. } => "Retrying", + ProcedureState::Failed { .. } => "Failed", + ProcedureState::PrepareRollback { .. } => "PrepareRollback", + ProcedureState::RollingBack { .. } => "RollingBack", + } + } } /// The initial procedure state. @@ -412,11 +437,30 @@ pub trait ProcedureManager: Send + Sync + 'static { /// Returns a [Watcher] to watch [ProcedureState] of specific procedure. fn procedure_watcher(&self, procedure_id: ProcedureId) -> Option; + + /// Returns the details of the procedure. + async fn list_procedures(&self) -> Result>; } /// Ref-counted pointer to the [ProcedureManager]. pub type ProcedureManagerRef = Arc; +#[derive(Debug, Clone)] +pub struct ProcedureInfo { + /// Id of this procedure. + pub id: ProcedureId, + /// Type name of this procedure. + pub type_name: String, + /// Start execution time of this procedure. + pub start_time_ms: i64, + /// End execution time of this procedure. + pub end_time_ms: i64, + /// status of this procedure. + pub state: ProcedureState, + /// Lock keys of this procedure. + pub lock_keys: Vec, +} + #[cfg(test)] mod tests { use common_error::mock::MockError; diff --git a/src/meta-client/src/client.rs b/src/meta-client/src/client.rs index 7c2a6ed923..0a9df7293f 100644 --- a/src/meta-client/src/client.rs +++ b/src/meta-client/src/client.rs @@ -22,7 +22,7 @@ mod cluster; mod store; mod util; -use api::v1::meta::Role; +use api::v1::meta::{ProcedureDetailResponse, Role}; use cluster::Client as ClusterClient; use common_error::ext::BoxedError; use common_grpc::channel_manager::{ChannelConfig, ChannelManager}; @@ -259,6 +259,16 @@ impl ProcedureExecutor for MetaClient { .map_err(BoxedError::new) .context(meta_error::ExternalSnafu) } + + async fn list_procedures(&self, _ctx: &ExecutorContext) -> MetaResult { + self.procedure_client() + .map_err(BoxedError::new) + .context(meta_error::ExternalSnafu)? + .list_procedures() + .await + .map_err(BoxedError::new) + .context(meta_error::ExternalSnafu) + } } #[async_trait::async_trait] diff --git a/src/meta-client/src/client/procedure.rs b/src/meta-client/src/client/procedure.rs index 32049dbabd..f45cfb7879 100644 --- a/src/meta-client/src/client/procedure.rs +++ b/src/meta-client/src/client/procedure.rs @@ -18,8 +18,9 @@ use std::time::Duration; use api::v1::meta::procedure_service_client::ProcedureServiceClient; use api::v1::meta::{ - DdlTaskRequest, DdlTaskResponse, MigrateRegionRequest, MigrateRegionResponse, ProcedureId, - ProcedureStateResponse, QueryProcedureRequest, ResponseHeader, Role, + DdlTaskRequest, DdlTaskResponse, MigrateRegionRequest, MigrateRegionResponse, + ProcedureDetailRequest, ProcedureDetailResponse, ProcedureId, ProcedureStateResponse, + QueryProcedureRequest, ResponseHeader, Role, }; use common_grpc::channel_manager::ChannelManager; use common_telemetry::tracing_context::TracingContext; @@ -89,6 +90,11 @@ impl Client { .migrate_region(region_id, from_peer, to_peer, replay_timeout) .await } + + pub async fn list_procedures(&self) -> Result { + let inner = self.inner.read().await; + inner.list_procedures().await + } } #[derive(Debug)] @@ -279,4 +285,23 @@ impl Inner { ) .await } + + async fn list_procedures(&self) -> Result { + let mut req = ProcedureDetailRequest::default(); + req.set_header( + self.id, + self.role, + TracingContext::from_current_span().to_w3c(), + ); + + self.with_retry( + "list procedure", + move |mut client| { + let req = req.clone(); + async move { client.details(req).await.map(|res| res.into_inner()) } + }, + |resp: &ProcedureDetailResponse| &resp.header, + ) + .await + } } diff --git a/src/meta-srv/src/service/procedure.rs b/src/meta-srv/src/service/procedure.rs index 05e6d98fa4..d19d0902ae 100644 --- a/src/meta-srv/src/service/procedure.rs +++ b/src/meta-srv/src/service/procedure.rs @@ -18,7 +18,7 @@ use std::time::Duration; use api::v1::meta::{ procedure_service_server, DdlTaskRequest as PbDdlTaskRequest, DdlTaskResponse as PbDdlTaskResponse, MigrateRegionRequest, MigrateRegionResponse, - ProcedureStateResponse, QueryProcedureRequest, + ProcedureDetailRequest, ProcedureDetailResponse, ProcedureStateResponse, QueryProcedureRequest, }; use common_meta::ddl::ExecutorContext; use common_meta::rpc::ddl::{DdlTask, SubmitDdlTaskRequest}; @@ -146,4 +146,20 @@ impl procedure_service_server::ProcedureService for Metasrv { Ok(Response::new(resp)) } + + async fn details( + &self, + request: Request, + ) -> GrpcResult { + let ProcedureDetailRequest { header } = request.into_inner(); + let _header = header.context(error::MissingRequestHeaderSnafu)?; + let metas = self + .procedure_manager() + .list_procedures() + .await + .context(error::QueryProcedureSnafu)?; + Ok(Response::new(procedure::procedure_details_to_pb_response( + metas, + ))) + } } diff --git a/tests-integration/src/cluster.rs b/tests-integration/src/cluster.rs index 19cb36a5b9..ce2803996a 100644 --- a/tests-integration/src/cluster.rs +++ b/tests-integration/src/cluster.rs @@ -370,6 +370,7 @@ impl GreptimeDbClusterBuilder { Some(meta_client.clone()), cached_meta_backend.clone(), cache_registry.clone(), + None, ); let handlers_executor = HandlerGroupExecutor::new(vec![ diff --git a/tests-integration/src/standalone.rs b/tests-integration/src/standalone.rs index 9f7188568f..fa6e8f2a9e 100644 --- a/tests-integration/src/standalone.rs +++ b/tests-integration/src/standalone.rs @@ -149,6 +149,7 @@ impl GreptimeDbStandaloneBuilder { None, kv_backend.clone(), cache_registry.clone(), + Some(procedure_manager.clone()), ); let flow_builder = FlownodeBuilder::new( diff --git a/tests/cases/standalone/common/information_schema/procedure_info.result b/tests/cases/standalone/common/information_schema/procedure_info.result new file mode 100644 index 0000000000..965e02b8ba --- /dev/null +++ b/tests/cases/standalone/common/information_schema/procedure_info.result @@ -0,0 +1,40 @@ +--- test information_schema.procedure_info ---- +USE public; + +Affected Rows: 0 + +CREATE TABLE procedure_info_for_sql_test1( + ts TIMESTAMP TIME INDEX, + temperature DOUBLE DEFAULT 10, +) engine=mito with('append_mode'='true'); + +Affected Rows: 0 + +CREATE TABLE procedure_info_for_sql_test2( + ts TIMESTAMP TIME INDEX, + temperature DOUBLE DEFAULT 10, +) engine=mito with('append_mode'='true'); + +Affected Rows: 0 + +use INFORMATION_SCHEMA; + +Affected Rows: 0 + +select procedure_type from procedure_info where lock_keys like '%procedure_info_for_sql_test%'; + ++--------------------------------+ +| procedure_type | ++--------------------------------+ +| metasrv-procedure::CreateTable | +| metasrv-procedure::CreateTable | ++--------------------------------+ + +use public; + +Affected Rows: 0 + +DROP TABLE procedure_info_for_sql_test1, procedure_info_for_sql_test2; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/information_schema/procedure_info.sql b/tests/cases/standalone/common/information_schema/procedure_info.sql new file mode 100644 index 0000000000..763dde12ef --- /dev/null +++ b/tests/cases/standalone/common/information_schema/procedure_info.sql @@ -0,0 +1,20 @@ +--- test information_schema.procedure_info ---- +USE public; + +CREATE TABLE procedure_info_for_sql_test1( + ts TIMESTAMP TIME INDEX, + temperature DOUBLE DEFAULT 10, +) engine=mito with('append_mode'='true'); + +CREATE TABLE procedure_info_for_sql_test2( + ts TIMESTAMP TIME INDEX, + temperature DOUBLE DEFAULT 10, +) engine=mito with('append_mode'='true'); + +use INFORMATION_SCHEMA; + +select procedure_type from procedure_info where lock_keys like '%procedure_info_for_sql_test%'; + +use public; + +DROP TABLE procedure_info_for_sql_test1, procedure_info_for_sql_test2; diff --git a/tests/cases/standalone/common/show/show_databases_tables.result b/tests/cases/standalone/common/show/show_databases_tables.result index fa50fb2aab..c92a5b33c8 100644 --- a/tests/cases/standalone/common/show/show_databases_tables.result +++ b/tests/cases/standalone/common/show/show_databases_tables.result @@ -45,6 +45,7 @@ SHOW TABLES; | optimizer_trace | | parameters | | partitions | +| procedure_info | | profiling | | referential_constraints | | region_peers | @@ -91,6 +92,7 @@ SHOW FULL TABLES; | optimizer_trace | LOCAL TEMPORARY | | parameters | LOCAL TEMPORARY | | partitions | LOCAL TEMPORARY | +| procedure_info | LOCAL TEMPORARY | | profiling | LOCAL TEMPORARY | | referential_constraints | LOCAL TEMPORARY | | region_peers | LOCAL TEMPORARY | @@ -131,6 +133,7 @@ SHOW TABLE STATUS; |optimizer_trace||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |parameters||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |partitions||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| +|procedure_info||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |profiling||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |referential_constraints||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |region_peers||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index cbb0c12b6f..03b65cf390 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -32,6 +32,7 @@ order by table_schema, table_name; |greptime|information_schema|optimizer_trace|LOCALTEMPORARY|17|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|parameters|LOCALTEMPORARY|18|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|partitions|LOCALTEMPORARY|28|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| +|greptime|information_schema|procedure_info|LOCALTEMPORARY|34|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|profiling|LOCALTEMPORARY|19|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|referential_constraints|LOCALTEMPORARY|20|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|region_peers|LOCALTEMPORARY|29|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| @@ -256,6 +257,12 @@ select * from information_schema.columns order by table_schema, table_name, colu | greptime | information_schema | partitions | table_schema | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | | greptime | information_schema | partitions | tablespace_name | 25 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | | greptime | information_schema | partitions | update_time | 20 | | | | | 3 | | | | | select,insert | | DateTime | datetime | FIELD | | Yes | datetime | | | +| greptime | information_schema | procedure_info | end_time | 4 | | | | | 3 | | | | | select,insert | | TimestampMillisecond | timestamp(3) | FIELD | | Yes | timestamp(3) | | | +| greptime | information_schema | procedure_info | lock_keys | 6 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | procedure_info | procedure_id | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | +| greptime | information_schema | procedure_info | procedure_type | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | +| greptime | information_schema | procedure_info | start_time | 3 | | | | | 3 | | | | | select,insert | | TimestampMillisecond | timestamp(3) | FIELD | | Yes | timestamp(3) | | | +| greptime | information_schema | procedure_info | status | 5 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | | greptime | information_schema | profiling | block_ops_in | 9 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | | | greptime | information_schema | profiling | block_ops_out | 10 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | | | greptime | information_schema | profiling | context_involuntary | 8 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | | diff --git a/tests/cases/standalone/common/view/create.result b/tests/cases/standalone/common/view/create.result index 4cf8084cd3..4fd2f59356 100644 --- a/tests/cases/standalone/common/view/create.result +++ b/tests/cases/standalone/common/view/create.result @@ -106,6 +106,7 @@ SELECT * FROM INFORMATION_SCHEMA.TABLES ORDER BY TABLE_NAME, TABLE_TYPE; |greptime|information_schema|optimizer_trace|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|parameters|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|partitions|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| +|greptime|information_schema|procedure_info|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|profiling|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|referential_constraints|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|region_peers|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| From 163cea81c26ded94b0780840eabbaae022c3b6d4 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 20 Sep 2024 17:27:20 +0900 Subject: [PATCH 028/128] feat: migrate local WAL regions (#4715) * feat: allow to flush region before migrating * fix: fix unit tests * feat: allow to set `flush_timeout` * feat: skip to replay memtable * fix: fix unit tests * test: add more tests * refactor: simplify timeout logical * test: add unit tests * test: add unit tests * chore: update comments * fix: fix unit tests * fix: fmt and clippy * feat: change default timeout to 30s * fix: throw `ExceededDeadline` error * test: add tests for `downgrade_region_with_retry` * chore: apply suggestions from CR * chore: apply suggestions from CR * chore: apply suggestions from CR * chore: update proto to `3633474` * refactor: refactor `upgrade_region_with_retry` * chore: apply suggestions from CR --- Cargo.lock | 2 +- Cargo.toml | 2 +- .../function/src/table/migrate_region.rs | 21 +- src/common/meta/src/instruction.rs | 13 +- src/common/meta/src/rpc/procedure.rs | 2 +- src/datanode/src/heartbeat/handler.rs | 24 +- .../src/heartbeat/handler/downgrade_region.rs | 401 +++++++++++++++++- .../src/heartbeat/handler/upgrade_region.rs | 57 +-- src/datanode/src/tests.rs | 15 +- src/meta-client/src/client.rs | 2 +- src/meta-client/src/client/procedure.rs | 10 +- src/meta-srv/src/error.rs | 10 +- .../src/procedure/region_migration.rs | 28 +- .../downgrade_leader_region.rs | 178 +++++--- .../src/procedure/region_migration/manager.rs | 24 +- .../procedure/region_migration/test_util.rs | 2 +- .../upgrade_candidate_region.rs | 124 ++++-- src/meta-srv/src/region/supervisor.rs | 2 +- src/meta-srv/src/service/procedure.rs | 5 +- src/mito2/src/engine/catchup_test.rs | 120 +++++- src/mito2/src/test_util.rs | 6 + src/mito2/src/worker/handle_catchup.rs | 67 +-- src/store-api/src/logstore/provider.rs | 5 + 23 files changed, 848 insertions(+), 272 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2183cf593d..000d6316ec 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4396,7 +4396,7 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "greptime-proto" version = "0.1.0" -source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=796ce9b003c6689e853825f649e03543c81ede99#796ce9b003c6689e853825f649e03543c81ede99" +source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=36334744c7020734dcb4a6b8d24d52ae7ed53fe1#36334744c7020734dcb4a6b8d24d52ae7ed53fe1" dependencies = [ "prost 0.12.6", "serde", diff --git a/Cargo.toml b/Cargo.toml index df70b98332..f0b93ac653 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -120,7 +120,7 @@ etcd-client = { version = "0.13" } fst = "0.4.7" futures = "0.3" futures-util = "0.3" -greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "796ce9b003c6689e853825f649e03543c81ede99" } +greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "36334744c7020734dcb4a6b8d24d52ae7ed53fe1" } humantime = "2.1" humantime-serde = "1.1" itertools = "0.10" diff --git a/src/common/function/src/table/migrate_region.rs b/src/common/function/src/table/migrate_region.rs index b46231eb45..0a487973d3 100644 --- a/src/common/function/src/table/migrate_region.rs +++ b/src/common/function/src/table/migrate_region.rs @@ -25,13 +25,13 @@ use session::context::QueryContextRef; use crate::handlers::ProcedureServiceHandlerRef; use crate::helper::cast_u64; -const DEFAULT_REPLAY_TIMEOUT_SECS: u64 = 10; +const DEFAULT_TIMEOUT_SECS: u64 = 30; /// A function to migrate a region from source peer to target peer. /// Returns the submitted procedure id if success. Only available in cluster mode. /// -/// - `migrate_region(region_id, from_peer, to_peer)`, with default replay WAL timeout(10 seconds). -/// - `migrate_region(region_id, from_peer, to_peer, timeout(secs))` +/// - `migrate_region(region_id, from_peer, to_peer)`, with timeout(30 seconds). +/// - `migrate_region(region_id, from_peer, to_peer, timeout(secs))`. /// /// The parameters: /// - `region_id`: the region id @@ -48,18 +48,13 @@ pub(crate) async fn migrate_region( _ctx: &QueryContextRef, params: &[ValueRef<'_>], ) -> Result { - let (region_id, from_peer, to_peer, replay_timeout) = match params.len() { + let (region_id, from_peer, to_peer, timeout) = match params.len() { 3 => { let region_id = cast_u64(¶ms[0])?; let from_peer = cast_u64(¶ms[1])?; let to_peer = cast_u64(¶ms[2])?; - ( - region_id, - from_peer, - to_peer, - Some(DEFAULT_REPLAY_TIMEOUT_SECS), - ) + (region_id, from_peer, to_peer, Some(DEFAULT_TIMEOUT_SECS)) } 4 => { @@ -82,14 +77,14 @@ pub(crate) async fn migrate_region( } }; - match (region_id, from_peer, to_peer, replay_timeout) { - (Some(region_id), Some(from_peer), Some(to_peer), Some(replay_timeout)) => { + match (region_id, from_peer, to_peer, timeout) { + (Some(region_id), Some(from_peer), Some(to_peer), Some(timeout)) => { let pid = procedure_service_handler .migrate_region(MigrateRegionRequest { region_id, from_peer, to_peer, - replay_timeout: Duration::from_secs(replay_timeout), + timeout: Duration::from_secs(timeout), }) .await?; diff --git a/src/common/meta/src/instruction.rs b/src/common/meta/src/instruction.rs index d620cc3449..61e2811e72 100644 --- a/src/common/meta/src/instruction.rs +++ b/src/common/meta/src/instruction.rs @@ -132,11 +132,20 @@ impl OpenRegion { pub struct DowngradeRegion { /// The [RegionId]. pub region_id: RegionId, + /// The timeout of waiting for flush the region. + /// + /// `None` stands for don't flush before downgrading the region. + #[serde(default)] + pub flush_timeout: Option, } impl Display for DowngradeRegion { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - write!(f, "DowngradeRegion(region_id={})", self.region_id) + write!( + f, + "DowngradeRegion(region_id={}, flush_timeout={:?})", + self.region_id, self.flush_timeout, + ) } } @@ -152,7 +161,7 @@ pub struct UpgradeRegion { /// `None` stands for no wait, /// it's helpful to verify whether the leader region is ready. #[serde(with = "humantime_serde")] - pub wait_for_replay_timeout: Option, + pub replay_timeout: Option, /// The hint for replaying memtable. #[serde(default)] pub location_id: Option, diff --git a/src/common/meta/src/rpc/procedure.rs b/src/common/meta/src/rpc/procedure.rs index 2c2a69b5b6..2e25a4aa5d 100644 --- a/src/common/meta/src/rpc/procedure.rs +++ b/src/common/meta/src/rpc/procedure.rs @@ -31,7 +31,7 @@ pub struct MigrateRegionRequest { pub region_id: u64, pub from_peer: u64, pub to_peer: u64, - pub replay_timeout: Duration, + pub timeout: Duration, } /// Cast the protobuf [`ProcedureId`] to common [`ProcedureId`]. diff --git a/src/datanode/src/heartbeat/handler.rs b/src/datanode/src/heartbeat/handler.rs index 573b94cb11..d23615eb13 100644 --- a/src/datanode/src/heartbeat/handler.rs +++ b/src/datanode/src/heartbeat/handler.rs @@ -37,6 +37,7 @@ use crate::region_server::RegionServer; pub struct RegionHeartbeatResponseHandler { region_server: RegionServer, catchup_tasks: TaskTracker<()>, + downgrade_tasks: TaskTracker<()>, } /// Handler of the instruction. @@ -47,12 +48,22 @@ pub type InstructionHandler = pub struct HandlerContext { region_server: RegionServer, catchup_tasks: TaskTracker<()>, + downgrade_tasks: TaskTracker<()>, } impl HandlerContext { fn region_ident_to_region_id(region_ident: &RegionIdent) -> RegionId { RegionId::new(region_ident.table_id, region_ident.region_number) } + + #[cfg(test)] + pub fn new_for_test(region_server: RegionServer) -> Self { + Self { + region_server, + catchup_tasks: TaskTracker::new(), + downgrade_tasks: TaskTracker::new(), + } + } } impl RegionHeartbeatResponseHandler { @@ -61,6 +72,7 @@ impl RegionHeartbeatResponseHandler { Self { region_server, catchup_tasks: TaskTracker::new(), + downgrade_tasks: TaskTracker::new(), } } @@ -107,11 +119,13 @@ impl HeartbeatResponseHandler for RegionHeartbeatResponseHandler { let mailbox = ctx.mailbox.clone(); let region_server = self.region_server.clone(); let catchup_tasks = self.catchup_tasks.clone(); + let downgrade_tasks = self.downgrade_tasks.clone(); let handler = Self::build_handler(instruction)?; let _handle = common_runtime::spawn_global(async move { let reply = handler(HandlerContext { region_server, catchup_tasks, + downgrade_tasks, }) .await; @@ -129,6 +143,7 @@ mod tests { use std::assert_matches::assert_matches; use std::collections::HashMap; use std::sync::Arc; + use std::time::Duration; use common_meta::heartbeat::mailbox::{ HeartbeatMailbox, IncomingMessage, MailboxRef, MessageMeta, @@ -197,6 +212,7 @@ mod tests { // Downgrade region let instruction = Instruction::DowngradeRegion(DowngradeRegion { region_id: RegionId::new(2048, 1), + flush_timeout: Some(Duration::from_secs(1)), }); assert!(heartbeat_handler .is_acceptable(&heartbeat_env.create_handler_ctx((meta.clone(), instruction)))); @@ -205,7 +221,7 @@ mod tests { let instruction = Instruction::UpgradeRegion(UpgradeRegion { region_id, last_entry_id: None, - wait_for_replay_timeout: None, + replay_timeout: None, location_id: None, }); assert!( @@ -392,7 +408,10 @@ mod tests { // Should be ok, if we try to downgrade it twice. for _ in 0..2 { let meta = MessageMeta::new_test(1, "test", "dn-1", "me-0"); - let instruction = Instruction::DowngradeRegion(DowngradeRegion { region_id }); + let instruction = Instruction::DowngradeRegion(DowngradeRegion { + region_id, + flush_timeout: Some(Duration::from_secs(1)), + }); let mut ctx = heartbeat_env.create_handler_ctx((meta, instruction)); let control = heartbeat_handler.handle(&mut ctx).await.unwrap(); @@ -413,6 +432,7 @@ mod tests { let meta = MessageMeta::new_test(1, "test", "dn-1", "me-0"); let instruction = Instruction::DowngradeRegion(DowngradeRegion { region_id: RegionId::new(2048, 1), + flush_timeout: Some(Duration::from_secs(1)), }); let mut ctx = heartbeat_env.create_handler_ctx((meta, instruction)); let control = heartbeat_handler.handle(&mut ctx).await.unwrap(); diff --git a/src/datanode/src/heartbeat/handler/downgrade_region.rs b/src/datanode/src/heartbeat/handler/downgrade_region.rs index 4dccdc26aa..ac11792803 100644 --- a/src/datanode/src/heartbeat/handler/downgrade_region.rs +++ b/src/datanode/src/heartbeat/handler/downgrade_region.rs @@ -13,38 +13,399 @@ // limitations under the License. use common_meta::instruction::{DowngradeRegion, DowngradeRegionReply, InstructionReply}; +use common_telemetry::tracing::info; +use common_telemetry::warn; use futures_util::future::BoxFuture; use store_api::region_engine::SetReadonlyResponse; +use store_api::region_request::{RegionFlushRequest, RegionRequest}; +use store_api::storage::RegionId; use crate::heartbeat::handler::HandlerContext; +use crate::heartbeat::task_tracker::WaitResult; impl HandlerContext { + async fn set_readonly_gracefully(&self, region_id: RegionId) -> InstructionReply { + match self.region_server.set_readonly_gracefully(region_id).await { + Ok(SetReadonlyResponse::Success { last_entry_id }) => { + InstructionReply::DowngradeRegion(DowngradeRegionReply { + last_entry_id, + exists: true, + error: None, + }) + } + Ok(SetReadonlyResponse::NotFound) => { + InstructionReply::DowngradeRegion(DowngradeRegionReply { + last_entry_id: None, + exists: false, + error: None, + }) + } + Err(err) => InstructionReply::DowngradeRegion(DowngradeRegionReply { + last_entry_id: None, + exists: true, + error: Some(format!("{err:?}")), + }), + } + } + pub(crate) fn handle_downgrade_region_instruction( self, - DowngradeRegion { region_id }: DowngradeRegion, + DowngradeRegion { + region_id, + flush_timeout, + }: DowngradeRegion, ) -> BoxFuture<'static, InstructionReply> { Box::pin(async move { - match self.region_server.set_readonly_gracefully(region_id).await { - Ok(SetReadonlyResponse::Success { last_entry_id }) => { - InstructionReply::DowngradeRegion(DowngradeRegionReply { - last_entry_id, - exists: true, - error: None, - }) - } - Ok(SetReadonlyResponse::NotFound) => { - InstructionReply::DowngradeRegion(DowngradeRegionReply { - last_entry_id: None, - exists: false, - error: None, - }) - } - Err(err) => InstructionReply::DowngradeRegion(DowngradeRegionReply { + let Some(writable) = self.region_server.is_writable(region_id) else { + return InstructionReply::DowngradeRegion(DowngradeRegionReply { last_entry_id: None, - exists: true, - error: Some(format!("{err:?}")), - }), + exists: false, + error: None, + }); + }; + + // Ignores flush request + if !writable { + return self.set_readonly_gracefully(region_id).await; + } + + let region_server_moved = self.region_server.clone(); + if let Some(flush_timeout) = flush_timeout { + let register_result = self + .downgrade_tasks + .try_register( + region_id, + Box::pin(async move { + info!("Flush region: {region_id} before downgrading region"); + region_server_moved + .handle_request( + region_id, + RegionRequest::Flush(RegionFlushRequest { + row_group_size: None, + }), + ) + .await?; + + Ok(()) + }), + ) + .await; + + if register_result.is_busy() { + warn!("Another flush task is running for the region: {region_id}"); + } + + let mut watcher = register_result.into_watcher(); + let result = self.catchup_tasks.wait(&mut watcher, flush_timeout).await; + + match result { + WaitResult::Timeout => { + InstructionReply::DowngradeRegion(DowngradeRegionReply { + last_entry_id: None, + exists: true, + error: Some(format!( + "Flush region: {region_id} before downgrading region is timeout" + )), + }) + } + WaitResult::Finish(Ok(_)) => self.set_readonly_gracefully(region_id).await, + WaitResult::Finish(Err(err)) => { + InstructionReply::DowngradeRegion(DowngradeRegionReply { + last_entry_id: None, + exists: true, + error: Some(format!("{err:?}")), + }) + } + } + } else { + self.set_readonly_gracefully(region_id).await } }) } } + +#[cfg(test)] +mod tests { + use std::assert_matches::assert_matches; + use std::time::Duration; + + use common_meta::instruction::{DowngradeRegion, InstructionReply}; + use mito2::engine::MITO_ENGINE_NAME; + use store_api::region_engine::{RegionRole, SetReadonlyResponse}; + use store_api::region_request::RegionRequest; + use store_api::storage::RegionId; + use tokio::time::Instant; + + use crate::error; + use crate::heartbeat::handler::HandlerContext; + use crate::tests::{mock_region_server, MockRegionEngine}; + + #[tokio::test] + async fn test_region_not_exist() { + let mut mock_region_server = mock_region_server(); + let (mock_engine, _) = MockRegionEngine::new(MITO_ENGINE_NAME); + mock_region_server.register_engine(mock_engine); + let handler_context = HandlerContext::new_for_test(mock_region_server); + let region_id = RegionId::new(1024, 1); + let waits = vec![None, Some(Duration::from_millis(100u64))]; + + for flush_timeout in waits { + let reply = handler_context + .clone() + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout, + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(!reply.exists); + assert!(reply.error.is_none()); + assert!(reply.last_entry_id.is_none()); + } + } + } + + #[tokio::test] + async fn test_region_readonly() { + let mock_region_server = mock_region_server(); + let region_id = RegionId::new(1024, 1); + let (mock_engine, _) = + MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { + region_engine.mock_role = Some(Some(RegionRole::Follower)); + region_engine.handle_request_mock_fn = Some(Box::new(|_, req| { + if let RegionRequest::Flush(_) = req { + // Should be unreachable. + unreachable!(); + }; + + Ok(0) + })); + region_engine.handle_set_readonly_gracefully_mock_fn = + Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + }); + mock_region_server.register_test_region(region_id, mock_engine); + let handler_context = HandlerContext::new_for_test(mock_region_server); + + let waits = vec![None, Some(Duration::from_millis(100u64))]; + for flush_timeout in waits { + let reply = handler_context + .clone() + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout, + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(reply.exists); + assert!(reply.error.is_none()); + assert_eq!(reply.last_entry_id.unwrap(), 1024); + } + } + } + + #[tokio::test] + async fn test_region_flush_timeout() { + let mock_region_server = mock_region_server(); + let region_id = RegionId::new(1024, 1); + let (mock_engine, _) = + MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { + region_engine.mock_role = Some(Some(RegionRole::Leader)); + region_engine.handle_request_delay = Some(Duration::from_secs(100)); + region_engine.handle_set_readonly_gracefully_mock_fn = + Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + }); + mock_region_server.register_test_region(region_id, mock_engine); + let handler_context = HandlerContext::new_for_test(mock_region_server); + + let flush_timeout = Duration::from_millis(100); + let reply = handler_context + .clone() + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout: Some(flush_timeout), + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(reply.exists); + assert!(reply.error.unwrap().contains("timeout")); + assert!(reply.last_entry_id.is_none()); + } + } + + #[tokio::test] + async fn test_region_flush_timeout_and_retry() { + let mock_region_server = mock_region_server(); + let region_id = RegionId::new(1024, 1); + let (mock_engine, _) = + MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { + region_engine.mock_role = Some(Some(RegionRole::Leader)); + region_engine.handle_request_delay = Some(Duration::from_millis(300)); + region_engine.handle_set_readonly_gracefully_mock_fn = + Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + }); + mock_region_server.register_test_region(region_id, mock_engine); + let handler_context = HandlerContext::new_for_test(mock_region_server); + + let waits = vec![ + Some(Duration::from_millis(100u64)), + Some(Duration::from_millis(100u64)), + ]; + + for flush_timeout in waits { + let reply = handler_context + .clone() + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout, + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(reply.exists); + assert!(reply.error.unwrap().contains("timeout")); + assert!(reply.last_entry_id.is_none()); + } + } + let timer = Instant::now(); + let reply = handler_context + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout: Some(Duration::from_millis(500)), + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + // Must less than 300 ms. + assert!(timer.elapsed().as_millis() < 300); + + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(reply.exists); + assert!(reply.error.is_none()); + assert_eq!(reply.last_entry_id.unwrap(), 1024); + } + } + + #[tokio::test] + async fn test_region_flush_timeout_and_retry_error() { + let mock_region_server = mock_region_server(); + let region_id = RegionId::new(1024, 1); + let (mock_engine, _) = + MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { + region_engine.mock_role = Some(Some(RegionRole::Leader)); + region_engine.handle_request_delay = Some(Duration::from_millis(300)); + region_engine.handle_request_mock_fn = Some(Box::new(|_, _| { + error::UnexpectedSnafu { + violated: "mock flush failed", + } + .fail() + })); + region_engine.handle_set_readonly_gracefully_mock_fn = + Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + }); + mock_region_server.register_test_region(region_id, mock_engine); + let handler_context = HandlerContext::new_for_test(mock_region_server); + + let waits = vec![ + Some(Duration::from_millis(100u64)), + Some(Duration::from_millis(100u64)), + ]; + + for flush_timeout in waits { + let reply = handler_context + .clone() + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout, + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(reply.exists); + assert!(reply.error.unwrap().contains("timeout")); + assert!(reply.last_entry_id.is_none()); + } + } + let timer = Instant::now(); + let reply = handler_context + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout: Some(Duration::from_millis(500)), + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + // Must less than 300 ms. + assert!(timer.elapsed().as_millis() < 300); + + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(reply.exists); + assert!(reply.error.unwrap().contains("flush failed")); + assert!(reply.last_entry_id.is_none()); + } + } + + #[tokio::test] + async fn test_set_region_readonly_not_found() { + let mock_region_server = mock_region_server(); + let region_id = RegionId::new(1024, 1); + let (mock_engine, _) = + MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { + region_engine.mock_role = Some(Some(RegionRole::Leader)); + region_engine.handle_set_readonly_gracefully_mock_fn = + Some(Box::new(|_| Ok(SetReadonlyResponse::NotFound))); + }); + mock_region_server.register_test_region(region_id, mock_engine); + let handler_context = HandlerContext::new_for_test(mock_region_server); + let reply = handler_context + .clone() + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout: None, + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(!reply.exists); + assert!(reply.error.is_none()); + assert!(reply.last_entry_id.is_none()); + } + } + + #[tokio::test] + async fn test_set_region_readonly_error() { + let mock_region_server = mock_region_server(); + let region_id = RegionId::new(1024, 1); + let (mock_engine, _) = + MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { + region_engine.mock_role = Some(Some(RegionRole::Leader)); + region_engine.handle_set_readonly_gracefully_mock_fn = Some(Box::new(|_| { + error::UnexpectedSnafu { + violated: "Failed to set region to readonly", + } + .fail() + })); + }); + mock_region_server.register_test_region(region_id, mock_engine); + let handler_context = HandlerContext::new_for_test(mock_region_server); + let reply = handler_context + .clone() + .handle_downgrade_region_instruction(DowngradeRegion { + region_id, + flush_timeout: None, + }) + .await; + assert_matches!(reply, InstructionReply::DowngradeRegion(_)); + if let InstructionReply::DowngradeRegion(reply) = reply { + assert!(reply.exists); + assert!(reply + .error + .unwrap() + .contains("Failed to set region to readonly")); + assert!(reply.last_entry_id.is_none()); + } + } +} diff --git a/src/datanode/src/heartbeat/handler/upgrade_region.rs b/src/datanode/src/heartbeat/handler/upgrade_region.rs index 6143731663..0d1ef0476c 100644 --- a/src/datanode/src/heartbeat/handler/upgrade_region.rs +++ b/src/datanode/src/heartbeat/handler/upgrade_region.rs @@ -26,7 +26,7 @@ impl HandlerContext { UpgradeRegion { region_id, last_entry_id, - wait_for_replay_timeout, + replay_timeout, location_id, }: UpgradeRegion, ) -> BoxFuture<'static, InstructionReply> { @@ -78,7 +78,7 @@ impl HandlerContext { } // Returns immediately - let Some(wait_for_replay_timeout) = wait_for_replay_timeout else { + let Some(replay_timeout) = replay_timeout else { return InstructionReply::UpgradeRegion(UpgradeRegionReply { ready: false, exists: true, @@ -88,10 +88,7 @@ impl HandlerContext { // We don't care that it returns a newly registered or running task. let mut watcher = register_result.into_watcher(); - let result = self - .catchup_tasks - .wait(&mut watcher, wait_for_replay_timeout) - .await; + let result = self.catchup_tasks.wait(&mut watcher, replay_timeout).await; match result { WaitResult::Timeout => InstructionReply::UpgradeRegion(UpgradeRegionReply { @@ -129,7 +126,6 @@ mod tests { use crate::error; use crate::heartbeat::handler::HandlerContext; - use crate::heartbeat::task_tracker::TaskTracker; use crate::tests::{mock_region_server, MockRegionEngine}; #[tokio::test] @@ -138,21 +134,18 @@ mod tests { let (mock_engine, _) = MockRegionEngine::new(MITO_ENGINE_NAME); mock_region_server.register_engine(mock_engine); - let handler_context = HandlerContext { - region_server: mock_region_server, - catchup_tasks: TaskTracker::new(), - }; + let handler_context = HandlerContext::new_for_test(mock_region_server); let region_id = RegionId::new(1024, 1); let waits = vec![None, Some(Duration::from_millis(100u64))]; - for wait_for_replay_timeout in waits { + for replay_timeout in waits { let reply = handler_context .clone() .handle_upgrade_region_instruction(UpgradeRegion { region_id, last_entry_id: None, - wait_for_replay_timeout, + replay_timeout, location_id: None, }) .await; @@ -180,20 +173,17 @@ mod tests { }); mock_region_server.register_test_region(region_id, mock_engine); - let handler_context = HandlerContext { - region_server: mock_region_server, - catchup_tasks: TaskTracker::new(), - }; + let handler_context = HandlerContext::new_for_test(mock_region_server); let waits = vec![None, Some(Duration::from_millis(100u64))]; - for wait_for_replay_timeout in waits { + for replay_timeout in waits { let reply = handler_context .clone() .handle_upgrade_region_instruction(UpgradeRegion { region_id, last_entry_id: None, - wait_for_replay_timeout, + replay_timeout, location_id: None, }) .await; @@ -222,20 +212,17 @@ mod tests { }); mock_region_server.register_test_region(region_id, mock_engine); - let handler_context = HandlerContext { - region_server: mock_region_server, - catchup_tasks: TaskTracker::new(), - }; + let handler_context = HandlerContext::new_for_test(mock_region_server); let waits = vec![None, Some(Duration::from_millis(100u64))]; - for wait_for_replay_timeout in waits { + for replay_timeout in waits { let reply = handler_context .clone() .handle_upgrade_region_instruction(UpgradeRegion { region_id, last_entry_id: None, - wait_for_replay_timeout, + replay_timeout, location_id: None, }) .await; @@ -269,17 +256,14 @@ mod tests { Some(Duration::from_millis(100u64)), ]; - let handler_context = HandlerContext { - region_server: mock_region_server, - catchup_tasks: TaskTracker::new(), - }; + let handler_context = HandlerContext::new_for_test(mock_region_server); - for wait_for_replay_timeout in waits { + for replay_timeout in waits { let reply = handler_context .clone() .handle_upgrade_region_instruction(UpgradeRegion { region_id, - wait_for_replay_timeout, + replay_timeout, last_entry_id: None, location_id: None, }) @@ -298,7 +282,7 @@ mod tests { .handle_upgrade_region_instruction(UpgradeRegion { region_id, last_entry_id: None, - wait_for_replay_timeout: Some(Duration::from_millis(500)), + replay_timeout: Some(Duration::from_millis(500)), location_id: None, }) .await; @@ -333,17 +317,14 @@ mod tests { }); mock_region_server.register_test_region(region_id, mock_engine); - let handler_context = HandlerContext { - region_server: mock_region_server, - catchup_tasks: TaskTracker::new(), - }; + let handler_context = HandlerContext::new_for_test(mock_region_server); let reply = handler_context .clone() .handle_upgrade_region_instruction(UpgradeRegion { region_id, last_entry_id: None, - wait_for_replay_timeout: None, + replay_timeout: None, location_id: None, }) .await; @@ -361,7 +342,7 @@ mod tests { .handle_upgrade_region_instruction(UpgradeRegion { region_id, last_entry_id: None, - wait_for_replay_timeout: Some(Duration::from_millis(200)), + replay_timeout: Some(Duration::from_millis(200)), location_id: None, }) .await; diff --git a/src/datanode/src/tests.rs b/src/datanode/src/tests.rs index 89be76511d..8966dc4932 100644 --- a/src/datanode/src/tests.rs +++ b/src/datanode/src/tests.rs @@ -103,10 +103,14 @@ pub fn mock_region_server() -> RegionServer { pub type MockRequestHandler = Box Result + Send + Sync>; +pub type MockSetReadonlyGracefullyHandler = + Box Result + Send + Sync>; + pub struct MockRegionEngine { sender: Sender<(RegionId, RegionRequest)>, pub(crate) handle_request_delay: Option, pub(crate) handle_request_mock_fn: Option, + pub(crate) handle_set_readonly_gracefully_mock_fn: Option, pub(crate) mock_role: Option>, engine: String, } @@ -120,6 +124,7 @@ impl MockRegionEngine { handle_request_delay: None, sender: tx, handle_request_mock_fn: None, + handle_set_readonly_gracefully_mock_fn: None, mock_role: None, engine: engine.to_string(), }), @@ -138,6 +143,7 @@ impl MockRegionEngine { handle_request_delay: None, sender: tx, handle_request_mock_fn: Some(mock_fn), + handle_set_readonly_gracefully_mock_fn: None, mock_role: None, engine: engine.to_string(), }), @@ -157,6 +163,7 @@ impl MockRegionEngine { handle_request_delay: None, sender: tx, handle_request_mock_fn: None, + handle_set_readonly_gracefully_mock_fn: None, mock_role: None, engine: engine.to_string(), }; @@ -217,9 +224,13 @@ impl RegionEngine for MockRegionEngine { async fn set_readonly_gracefully( &self, - _region_id: RegionId, + region_id: RegionId, ) -> Result { - unimplemented!() + if let Some(mock_fn) = &self.handle_set_readonly_gracefully_mock_fn { + return mock_fn(region_id).map_err(BoxedError::new); + }; + + unreachable!() } fn role(&self, _region_id: RegionId) -> Option { diff --git a/src/meta-client/src/client.rs b/src/meta-client/src/client.rs index 0a9df7293f..d5f7c6aaaf 100644 --- a/src/meta-client/src/client.rs +++ b/src/meta-client/src/client.rs @@ -483,7 +483,7 @@ impl MetaClient { request.region_id, request.from_peer, request.to_peer, - request.replay_timeout, + request.timeout, ) .await } diff --git a/src/meta-client/src/client/procedure.rs b/src/meta-client/src/client/procedure.rs index f45cfb7879..eccfe90dff 100644 --- a/src/meta-client/src/client/procedure.rs +++ b/src/meta-client/src/client/procedure.rs @@ -77,17 +77,17 @@ impl Client { /// - `region_id`: the migrated region id /// - `from_peer`: the source datanode id /// - `to_peer`: the target datanode id - /// - `replay_timeout`: replay WAL timeout after migration. + /// - `timeout`: timeout for downgrading region and upgrading region operations pub async fn migrate_region( &self, region_id: u64, from_peer: u64, to_peer: u64, - replay_timeout: Duration, + timeout: Duration, ) -> Result { let inner = self.inner.read().await; inner - .migrate_region(region_id, from_peer, to_peer, replay_timeout) + .migrate_region(region_id, from_peer, to_peer, timeout) .await } @@ -216,13 +216,13 @@ impl Inner { region_id: u64, from_peer: u64, to_peer: u64, - replay_timeout: Duration, + timeout: Duration, ) -> Result { let mut req = MigrateRegionRequest { region_id, from_peer, to_peer, - replay_timeout_secs: replay_timeout.as_secs() as u32, + timeout_secs: timeout.as_secs() as u32, ..Default::default() }; diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index d0a58d688e..728f326871 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -32,6 +32,13 @@ use crate::pubsub::Message; #[snafu(visibility(pub))] #[stack_trace_debug] pub enum Error { + #[snafu(display("Exceeded deadline, operation: {}", operation))] + ExceededDeadline { + #[snafu(implicit)] + location: Location, + operation: String, + }, + #[snafu(display("The target peer is unavailable temporally: {}", peer_id))] PeerUnavailable { #[snafu(implicit)] @@ -783,7 +790,8 @@ impl ErrorExt for Error { | Error::Join { .. } | Error::WeightArray { .. } | Error::NotSetWeightArray { .. } - | Error::PeerUnavailable { .. } => StatusCode::Internal, + | Error::PeerUnavailable { .. } + | Error::ExceededDeadline { .. } => StatusCode::Internal, Error::Unsupported { .. } => StatusCode::Unsupported, diff --git a/src/meta-srv/src/procedure/region_migration.rs b/src/meta-srv/src/procedure/region_migration.rs index e56afdbf11..9b5bdfc006 100644 --- a/src/meta-srv/src/procedure/region_migration.rs +++ b/src/meta-srv/src/procedure/region_migration.rs @@ -73,13 +73,13 @@ pub struct PersistentContext { to_peer: Peer, /// The [RegionId] of migration region. region_id: RegionId, - /// The timeout of waiting for a candidate to replay the WAL. - #[serde(with = "humantime_serde", default = "default_replay_timeout")] - replay_timeout: Duration, + /// The timeout for downgrading leader region and upgrading candidate region operations. + #[serde(with = "humantime_serde", default = "default_timeout")] + timeout: Duration, } -fn default_replay_timeout() -> Duration { - Duration::from_secs(1) +fn default_timeout() -> Duration { + Duration::from_secs(10) } impl PersistentContext { @@ -123,6 +123,8 @@ pub struct VolatileContext { leader_region_lease_deadline: Option, /// The last_entry_id of leader region. leader_region_last_entry_id: Option, + /// Elapsed time of downgrading region and upgrading region. + operations_elapsed: Duration, } impl VolatileContext { @@ -211,6 +213,18 @@ pub struct Context { } impl Context { + /// Returns the next operation's timeout. + pub fn next_operation_timeout(&self) -> Option { + self.persistent_ctx + .timeout + .checked_sub(self.volatile_ctx.operations_elapsed) + } + + /// Updates operations elapsed. + pub fn update_operations_elapsed(&mut self, instant: Instant) { + self.volatile_ctx.operations_elapsed += instant.elapsed(); + } + /// Returns address of meta server. pub fn server_addr(&self) -> &str { &self.server_addr @@ -441,7 +455,7 @@ impl RegionMigrationProcedure { region_id: persistent_ctx.region_id, from_peer: persistent_ctx.from_peer.clone(), to_peer: persistent_ctx.to_peer.clone(), - replay_timeout: persistent_ctx.replay_timeout, + timeout: persistent_ctx.timeout, }); let context = context_factory.new_context(persistent_ctx); @@ -537,7 +551,7 @@ mod tests { let procedure = RegionMigrationProcedure::new(persistent_context, context, None); let serialized = procedure.dump().unwrap(); - let expected = r#"{"persistent_ctx":{"catalog":"greptime","schema":"public","cluster_id":0,"from_peer":{"id":1,"addr":""},"to_peer":{"id":2,"addr":""},"region_id":4398046511105,"replay_timeout":"1s"},"state":{"region_migration_state":"RegionMigrationStart"}}"#; + let expected = r#"{"persistent_ctx":{"catalog":"greptime","schema":"public","cluster_id":0,"from_peer":{"id":1,"addr":""},"to_peer":{"id":2,"addr":""},"region_id":4398046511105,"timeout":"10s"},"state":{"region_migration_state":"RegionMigrationStart"}}"#; assert_eq!(expected, serialized); } diff --git a/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs b/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs index 17c5762906..836ca4c532 100644 --- a/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs +++ b/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs @@ -16,24 +16,23 @@ use std::any::Any; use std::time::Duration; use api::v1::meta::MailboxMessage; -use common_meta::distributed_time_constants::{MAILBOX_RTT_SECS, REGION_LEASE_SECS}; +use common_meta::distributed_time_constants::REGION_LEASE_SECS; use common_meta::instruction::{ DowngradeRegion, DowngradeRegionReply, Instruction, InstructionReply, }; use common_procedure::Status; -use common_telemetry::{info, warn}; +use common_telemetry::{error, info, warn}; use serde::{Deserialize, Serialize}; -use snafu::ResultExt; -use tokio::time::sleep; +use snafu::{OptionExt, ResultExt}; +use tokio::time::{sleep, Instant}; +use super::update_metadata::UpdateMetadata; use super::upgrade_candidate_region::UpgradeCandidateRegion; use crate::error::{self, Result}; use crate::handler::HeartbeatMailbox; use crate::procedure::region_migration::{Context, State}; use crate::service::mailbox::Channel; -const DOWNGRADE_LEADER_REGION_TIMEOUT: Duration = Duration::from_secs(MAILBOX_RTT_SECS); - #[derive(Debug, Serialize, Deserialize)] pub struct DowngradeLeaderRegion { // The optimistic retry times. @@ -55,25 +54,32 @@ impl Default for DowngradeLeaderRegion { #[typetag::serde] impl State for DowngradeLeaderRegion { async fn next(&mut self, ctx: &mut Context) -> Result<(Box, Status)> { - let replay_timeout = ctx.persistent_ctx.replay_timeout; // Ensures the `leader_region_lease_deadline` must exist after recovering. ctx.volatile_ctx .set_leader_region_lease_deadline(Duration::from_secs(REGION_LEASE_SECS)); - self.downgrade_region_with_retry(ctx).await; - if let Some(deadline) = ctx.volatile_ctx.leader_region_lease_deadline.as_ref() { - info!( - "Running into the downgrade leader slow path, sleep until {:?}", - deadline - ); - tokio::time::sleep_until(*deadline).await; + match self.downgrade_region_with_retry(ctx).await { + Ok(_) => { + // Do nothing + } + Err(error::Error::ExceededDeadline { .. }) => { + // Rollbacks the metadata if procedure is timeout + return Ok((Box::new(UpdateMetadata::Rollback), Status::executing(false))); + } + Err(err) => { + error!(err; "Occurs non-retryable error"); + if let Some(deadline) = ctx.volatile_ctx.leader_region_lease_deadline.as_ref() { + info!( + "Running into the downgrade leader slow path, sleep until {:?}", + deadline + ); + tokio::time::sleep_until(*deadline).await; + } + } } Ok(( - Box::new(UpgradeCandidateRegion { - replay_timeout, - ..Default::default() - }), + Box::new(UpgradeCandidateRegion::default()), Status::executing(false), )) } @@ -85,10 +91,17 @@ impl State for DowngradeLeaderRegion { impl DowngradeLeaderRegion { /// Builds downgrade region instruction. - fn build_downgrade_region_instruction(&self, ctx: &Context) -> Instruction { + fn build_downgrade_region_instruction( + &self, + ctx: &Context, + flush_timeout: Duration, + ) -> Instruction { let pc = &ctx.persistent_ctx; let region_id = pc.region_id; - Instruction::DowngradeRegion(DowngradeRegion { region_id }) + Instruction::DowngradeRegion(DowngradeRegion { + region_id, + flush_timeout: Some(flush_timeout), + }) } /// Tries to downgrade a leader region. @@ -102,32 +115,32 @@ impl DowngradeLeaderRegion { /// - [PushMessage](error::Error::PushMessage), The receiver is dropped. /// - [MailboxReceiver](error::Error::MailboxReceiver), The sender is dropped without sending (impossible). /// - [UnexpectedInstructionReply](error::Error::UnexpectedInstructionReply). + /// - [ExceededDeadline](error::Error::ExceededDeadline) /// - Invalid JSON. - async fn downgrade_region( - &self, - ctx: &mut Context, - downgrade_instruction: &Instruction, - ) -> Result<()> { + async fn downgrade_region(&self, ctx: &mut Context) -> Result<()> { let pc = &ctx.persistent_ctx; let region_id = pc.region_id; let leader = &pc.from_peer; + let operation_timeout = + ctx.next_operation_timeout() + .context(error::ExceededDeadlineSnafu { + operation: "Downgrade region", + })?; + let downgrade_instruction = self.build_downgrade_region_instruction(ctx, operation_timeout); let msg = MailboxMessage::json_message( &format!("Downgrade leader region: {}", region_id), &format!("Meta@{}", ctx.server_addr()), &format!("Datanode-{}@{}", leader.id, leader.addr), common_time::util::current_time_millis(), - downgrade_instruction, + &downgrade_instruction, ) .with_context(|_| error::SerializeToJsonSnafu { input: downgrade_instruction.to_string(), })?; let ch = Channel::Datanode(leader.id); - let receiver = ctx - .mailbox - .send(&ch, msg, DOWNGRADE_LEADER_REGION_TIMEOUT) - .await?; + let receiver = ctx.mailbox.send(&ch, msg, operation_timeout).await?; match receiver.await? { Ok(msg) => { @@ -191,26 +204,36 @@ impl DowngradeLeaderRegion { /// /// Slow path: /// - Waits for the lease of the leader region expired. - async fn downgrade_region_with_retry(&self, ctx: &mut Context) { - let instruction = self.build_downgrade_region_instruction(ctx); - + /// + /// Abort: + /// - ExceededDeadline + async fn downgrade_region_with_retry(&self, ctx: &mut Context) -> Result<()> { let mut retry = 0; loop { - if let Err(err) = self.downgrade_region(ctx, &instruction).await { + let timer = Instant::now(); + if let Err(err) = self.downgrade_region(ctx).await { + ctx.update_operations_elapsed(timer); retry += 1; - if err.is_retryable() && retry < self.optimistic_retry { - warn!("Failed to downgrade region, error: {err:?}, retry later"); + // Throws the error immediately if the procedure exceeded the deadline. + if matches!(err, error::Error::ExceededDeadline { .. }) { + return Err(err); + } else if err.is_retryable() && retry < self.optimistic_retry { + error!("Failed to downgrade region, error: {err:?}, retry later"); sleep(self.retry_initial_interval).await; } else { + error!("Failed to downgrade region, error: {err:?}"); break; } } else { + ctx.update_operations_elapsed(timer); // Resets the deadline. ctx.volatile_ctx.reset_leader_region_lease_deadline(); break; } } + + Ok(()) } } @@ -237,7 +260,7 @@ mod tests { to_peer: Peer::empty(2), region_id: RegionId::new(1024, 1), cluster_id: 0, - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), } } @@ -248,11 +271,7 @@ mod tests { let env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); - let instruction = &state.build_downgrade_region_instruction(&ctx); - let err = state - .downgrade_region(&mut ctx, instruction) - .await - .unwrap_err(); + let err = state.downgrade_region(&mut ctx).await.unwrap_err(); assert_matches!(err, Error::PusherNotFound { .. }); assert!(!err.is_retryable()); @@ -276,16 +295,33 @@ mod tests { drop(rx); - let instruction = &state.build_downgrade_region_instruction(&ctx); - let err = state - .downgrade_region(&mut ctx, instruction) - .await - .unwrap_err(); + let err = state.downgrade_region(&mut ctx).await.unwrap_err(); assert_matches!(err, Error::PushMessage { .. }); assert!(!err.is_retryable()); } + #[tokio::test] + async fn test_procedure_exceeded_deadline() { + let state = DowngradeLeaderRegion::default(); + let persistent_context = new_persistent_context(); + let env = TestingEnv::new(); + let mut ctx = env.context_factory().new_context(persistent_context); + ctx.volatile_ctx.operations_elapsed = ctx.persistent_ctx.timeout + Duration::from_secs(1); + + let err = state.downgrade_region(&mut ctx).await.unwrap_err(); + + assert_matches!(err, Error::ExceededDeadline { .. }); + assert!(!err.is_retryable()); + + let err = state + .downgrade_region_with_retry(&mut ctx) + .await + .unwrap_err(); + assert_matches!(err, Error::ExceededDeadline { .. }); + assert!(!err.is_retryable()); + } + #[tokio::test] async fn test_unexpected_instruction_reply() { let state = DowngradeLeaderRegion::default(); @@ -306,11 +342,7 @@ mod tests { // Sends an incorrect reply. send_mock_reply(mailbox, rx, |id| Ok(new_close_region_reply(id))); - let instruction = &state.build_downgrade_region_instruction(&ctx); - let err = state - .downgrade_region(&mut ctx, instruction) - .await - .unwrap_err(); + let err = state.downgrade_region(&mut ctx).await.unwrap_err(); assert_matches!(err, Error::UnexpectedInstructionReply { .. }); assert!(!err.is_retryable()); @@ -337,11 +369,7 @@ mod tests { Err(error::MailboxTimeoutSnafu { id }.build()) }); - let instruction = &state.build_downgrade_region_instruction(&ctx); - let err = state - .downgrade_region(&mut ctx, instruction) - .await - .unwrap_err(); + let err = state.downgrade_region(&mut ctx).await.unwrap_err(); assert_matches!(err, Error::RetryLater { .. }); assert!(err.is_retryable()); @@ -373,11 +401,7 @@ mod tests { )) }); - let instruction = &state.build_downgrade_region_instruction(&ctx); - let err = state - .downgrade_region(&mut ctx, instruction) - .await - .unwrap_err(); + let err = state.downgrade_region(&mut ctx).await.unwrap_err(); assert_matches!(err, Error::RetryLater { .. }); assert!(err.is_retryable()); @@ -425,7 +449,7 @@ mod tests { .unwrap(); }); - state.downgrade_region_with_retry(&mut ctx).await; + state.downgrade_region_with_retry(&mut ctx).await.unwrap(); assert_eq!(ctx.volatile_ctx.leader_region_last_entry_id, Some(1)); assert!(ctx.volatile_ctx.leader_region_lease_deadline.is_none()); } @@ -467,7 +491,7 @@ mod tests { ctx.volatile_ctx .set_leader_region_lease_deadline(Duration::from_secs(5)); let expected_deadline = ctx.volatile_ctx.leader_region_lease_deadline.unwrap(); - state.downgrade_region_with_retry(&mut ctx).await; + state.downgrade_region_with_retry(&mut ctx).await.unwrap(); assert_eq!(ctx.volatile_ctx.leader_region_last_entry_id, None); // Should remain no change. assert_eq!( @@ -509,4 +533,30 @@ mod tests { .downcast_ref::() .unwrap(); } + + #[tokio::test] + async fn test_downgrade_region_procedure_exceeded_deadline() { + let mut state = Box::::default(); + state.retry_initial_interval = Duration::from_millis(100); + let persistent_context = new_persistent_context(); + let to_peer_id = persistent_context.to_peer.id; + + let mut env = TestingEnv::new(); + let mut ctx = env.context_factory().new_context(persistent_context); + let mailbox_ctx = env.mailbox_context(); + let mailbox = mailbox_ctx.mailbox().clone(); + ctx.volatile_ctx.operations_elapsed = ctx.persistent_ctx.timeout + Duration::from_secs(1); + + let (tx, rx) = tokio::sync::mpsc::channel(1); + mailbox_ctx + .insert_heartbeat_response_receiver(Channel::Datanode(to_peer_id), tx) + .await; + + send_mock_reply(mailbox, rx, |id| { + Ok(new_downgrade_region_reply(id, None, true, None)) + }); + let (next, _) = state.next(&mut ctx).await.unwrap(); + let update_metadata = next.as_any().downcast_ref::().unwrap(); + assert_matches!(update_metadata, UpdateMetadata::Rollback); + } } diff --git a/src/meta-srv/src/procedure/region_migration/manager.rs b/src/meta-srv/src/procedure/region_migration/manager.rs index 73aa4371f4..bb3eff80c0 100644 --- a/src/meta-srv/src/procedure/region_migration/manager.rs +++ b/src/meta-srv/src/procedure/region_migration/manager.rs @@ -104,7 +104,7 @@ pub struct RegionMigrationProcedureTask { pub(crate) region_id: RegionId, pub(crate) from_peer: Peer, pub(crate) to_peer: Peer, - pub(crate) replay_timeout: Duration, + pub(crate) timeout: Duration, } impl RegionMigrationProcedureTask { @@ -113,14 +113,14 @@ impl RegionMigrationProcedureTask { region_id: RegionId, from_peer: Peer, to_peer: Peer, - replay_timeout: Duration, + timeout: Duration, ) -> Self { Self { cluster_id, region_id, from_peer, to_peer, - replay_timeout, + timeout, } } } @@ -328,7 +328,7 @@ impl RegionMigrationManager { region_id, from_peer, to_peer, - replay_timeout, + timeout, } = task.clone(); let procedure = RegionMigrationProcedure::new( PersistentContext { @@ -338,7 +338,7 @@ impl RegionMigrationManager { region_id, from_peer, to_peer, - replay_timeout, + timeout, }, self.context_factory.clone(), Some(guard), @@ -390,7 +390,7 @@ mod test { region_id, from_peer: Peer::empty(2), to_peer: Peer::empty(1), - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), }; // Inserts one manager @@ -415,7 +415,7 @@ mod test { region_id, from_peer: Peer::empty(1), to_peer: Peer::empty(1), - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), }; let err = manager.submit_procedure(task).await.unwrap_err(); @@ -433,7 +433,7 @@ mod test { region_id, from_peer: Peer::empty(1), to_peer: Peer::empty(2), - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), }; let err = manager.submit_procedure(task).await.unwrap_err(); @@ -451,7 +451,7 @@ mod test { region_id, from_peer: Peer::empty(1), to_peer: Peer::empty(2), - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), }; let table_info = new_test_table_info(1024, vec![1]).into(); @@ -479,7 +479,7 @@ mod test { region_id, from_peer: Peer::empty(1), to_peer: Peer::empty(2), - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), }; let table_info = new_test_table_info(1024, vec![1]).into(); @@ -511,7 +511,7 @@ mod test { region_id, from_peer: Peer::empty(1), to_peer: Peer::empty(2), - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), }; let table_info = new_test_table_info(1024, vec![1]).into(); @@ -538,7 +538,7 @@ mod test { region_id, from_peer: Peer::empty(1), to_peer: Peer::empty(2), - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), }; let err = manager diff --git a/src/meta-srv/src/procedure/region_migration/test_util.rs b/src/meta-srv/src/procedure/region_migration/test_util.rs index edfb89515f..65e33ab3d9 100644 --- a/src/meta-srv/src/procedure/region_migration/test_util.rs +++ b/src/meta-srv/src/procedure/region_migration/test_util.rs @@ -316,7 +316,7 @@ pub fn new_persistent_context(from: u64, to: u64, region_id: RegionId) -> Persis to_peer: Peer::empty(to), region_id, cluster_id: 0, - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_secs(10), } } diff --git a/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs index 88caf5f08d..49100e92f3 100644 --- a/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs @@ -16,13 +16,12 @@ use std::any::Any; use std::time::Duration; use api::v1::meta::MailboxMessage; -use common_meta::distributed_time_constants::MAILBOX_RTT_SECS; use common_meta::instruction::{Instruction, InstructionReply, UpgradeRegion, UpgradeRegionReply}; use common_procedure::Status; -use common_telemetry::warn; +use common_telemetry::error; use serde::{Deserialize, Serialize}; -use snafu::{ensure, ResultExt}; -use tokio::time::sleep; +use snafu::{ensure, OptionExt, ResultExt}; +use tokio::time::{sleep, Instant}; use super::update_metadata::UpdateMetadata; use crate::error::{self, Result}; @@ -36,8 +35,6 @@ pub struct UpgradeCandidateRegion { pub(crate) optimistic_retry: usize, // The retry initial interval. pub(crate) retry_initial_interval: Duration, - // The replay timeout of a instruction. - pub(crate) replay_timeout: Duration, // If it's true it requires the candidate region MUST replay the WAL to the latest entry id. // Otherwise, it will rollback to the old leader region. pub(crate) require_ready: bool, @@ -48,7 +45,6 @@ impl Default for UpgradeCandidateRegion { Self { optimistic_retry: 3, retry_initial_interval: Duration::from_millis(500), - replay_timeout: Duration::from_millis(1000), require_ready: true, } } @@ -71,17 +67,12 @@ impl State for UpgradeCandidateRegion { } impl UpgradeCandidateRegion { - const UPGRADE_CANDIDATE_REGION_RTT: Duration = Duration::from_secs(MAILBOX_RTT_SECS); - - /// Returns the timeout of the upgrade candidate region. - /// - /// Equals `replay_timeout` + RTT - fn send_upgrade_candidate_region_timeout(&self) -> Duration { - self.replay_timeout + UpgradeCandidateRegion::UPGRADE_CANDIDATE_REGION_RTT - } - /// Builds upgrade region instruction. - fn build_upgrade_region_instruction(&self, ctx: &Context) -> Instruction { + fn build_upgrade_region_instruction( + &self, + ctx: &Context, + replay_timeout: Duration, + ) -> Instruction { let pc = &ctx.persistent_ctx; let region_id = pc.region_id; let last_entry_id = ctx.volatile_ctx.leader_region_last_entry_id; @@ -89,7 +80,7 @@ impl UpgradeCandidateRegion { Instruction::UpgradeRegion(UpgradeRegion { region_id, last_entry_id, - wait_for_replay_timeout: Some(self.replay_timeout), + replay_timeout: Some(replay_timeout), location_id: Some(ctx.persistent_ctx.from_peer.id), }) } @@ -106,28 +97,32 @@ impl UpgradeCandidateRegion { /// - [PushMessage](error::Error::PushMessage), The receiver is dropped. /// - [MailboxReceiver](error::Error::MailboxReceiver), The sender is dropped without sending (impossible). /// - [UnexpectedInstructionReply](error::Error::UnexpectedInstructionReply) (impossible). + /// - [ExceededDeadline](error::Error::ExceededDeadline) /// - Invalid JSON (impossible). - async fn upgrade_region(&self, ctx: &Context, upgrade_instruction: &Instruction) -> Result<()> { + async fn upgrade_region(&self, ctx: &Context) -> Result<()> { let pc = &ctx.persistent_ctx; let region_id = pc.region_id; let candidate = &pc.to_peer; + let operation_timeout = + ctx.next_operation_timeout() + .context(error::ExceededDeadlineSnafu { + operation: "Upgrade region", + })?; + let upgrade_instruction = self.build_upgrade_region_instruction(ctx, operation_timeout); let msg = MailboxMessage::json_message( &format!("Upgrade candidate region: {}", region_id), &format!("Meta@{}", ctx.server_addr()), &format!("Datanode-{}@{}", candidate.id, candidate.addr), common_time::util::current_time_millis(), - upgrade_instruction, + &upgrade_instruction, ) .with_context(|_| error::SerializeToJsonSnafu { input: upgrade_instruction.to_string(), })?; let ch = Channel::Datanode(candidate.id); - let receiver = ctx - .mailbox - .send(&ch, msg, self.send_upgrade_candidate_region_timeout()) - .await?; + let receiver = ctx.mailbox.send(&ch, msg, operation_timeout).await?; match receiver.await? { Ok(msg) => { @@ -192,22 +187,27 @@ impl UpgradeCandidateRegion { /// Upgrades a candidate region. /// /// Returns true if the candidate region is upgraded successfully. - async fn upgrade_region_with_retry(&self, ctx: &Context) -> bool { - let upgrade_instruction = self.build_upgrade_region_instruction(ctx); - + async fn upgrade_region_with_retry(&self, ctx: &mut Context) -> bool { let mut retry = 0; let mut upgraded = false; loop { - if let Err(err) = self.upgrade_region(ctx, &upgrade_instruction).await { + let timer = Instant::now(); + if let Err(err) = self.upgrade_region(ctx).await { retry += 1; - if err.is_retryable() && retry < self.optimistic_retry { - warn!("Failed to upgrade region, error: {err:?}, retry later"); + ctx.update_operations_elapsed(timer); + if matches!(err, error::Error::ExceededDeadline { .. }) { + error!("Failed to upgrade region, exceeded deadline"); + break; + } else if err.is_retryable() && retry < self.optimistic_retry { + error!("Failed to upgrade region, error: {err:?}, retry later"); sleep(self.retry_initial_interval).await; } else { + error!("Failed to upgrade region, error: {err:?}"); break; } } else { + ctx.update_operations_elapsed(timer); upgraded = true; break; } @@ -239,7 +239,7 @@ mod tests { to_peer: Peer::empty(2), region_id: RegionId::new(1024, 1), cluster_id: 0, - replay_timeout: Duration::from_millis(1000), + timeout: Duration::from_millis(1000), } } @@ -250,8 +250,7 @@ mod tests { let env = TestingEnv::new(); let ctx = env.context_factory().new_context(persistent_context); - let instruction = &state.build_upgrade_region_instruction(&ctx); - let err = state.upgrade_region(&ctx, instruction).await.unwrap_err(); + let err = state.upgrade_region(&ctx).await.unwrap_err(); assert_matches!(err, Error::PusherNotFound { .. }); assert!(!err.is_retryable()); @@ -275,13 +274,26 @@ mod tests { drop(rx); - let instruction = &state.build_upgrade_region_instruction(&ctx); - let err = state.upgrade_region(&ctx, instruction).await.unwrap_err(); + let err = state.upgrade_region(&ctx).await.unwrap_err(); assert_matches!(err, Error::PushMessage { .. }); assert!(!err.is_retryable()); } + #[tokio::test] + async fn test_procedure_exceeded_deadline() { + let state = UpgradeCandidateRegion::default(); + let persistent_context = new_persistent_context(); + let env = TestingEnv::new(); + let mut ctx = env.context_factory().new_context(persistent_context); + ctx.volatile_ctx.operations_elapsed = ctx.persistent_ctx.timeout + Duration::from_secs(1); + + let err = state.upgrade_region(&ctx).await.unwrap_err(); + + assert_matches!(err, Error::ExceededDeadline { .. }); + assert!(!err.is_retryable()); + } + #[tokio::test] async fn test_unexpected_instruction_reply() { let state = UpgradeCandidateRegion::default(); @@ -301,8 +313,7 @@ mod tests { send_mock_reply(mailbox, rx, |id| Ok(new_close_region_reply(id))); - let instruction = &state.build_upgrade_region_instruction(&ctx); - let err = state.upgrade_region(&ctx, instruction).await.unwrap_err(); + let err = state.upgrade_region(&ctx).await.unwrap_err(); assert_matches!(err, Error::UnexpectedInstructionReply { .. }); assert!(!err.is_retryable()); } @@ -334,8 +345,7 @@ mod tests { )) }); - let instruction = &state.build_upgrade_region_instruction(&ctx); - let err = state.upgrade_region(&ctx, instruction).await.unwrap_err(); + let err = state.upgrade_region(&ctx).await.unwrap_err(); assert_matches!(err, Error::RetryLater { .. }); assert!(err.is_retryable()); @@ -363,8 +373,7 @@ mod tests { Ok(new_upgrade_region_reply(id, true, false, None)) }); - let instruction = &state.build_upgrade_region_instruction(&ctx); - let err = state.upgrade_region(&ctx, instruction).await.unwrap_err(); + let err = state.upgrade_region(&ctx).await.unwrap_err(); assert_matches!(err, Error::Unexpected { .. }); assert!(!err.is_retryable()); @@ -396,8 +405,7 @@ mod tests { Ok(new_upgrade_region_reply(id, false, true, None)) }); - let instruction = &state.build_upgrade_region_instruction(&ctx); - let err = state.upgrade_region(&ctx, instruction).await.unwrap_err(); + let err = state.upgrade_region(&ctx).await.unwrap_err(); assert_matches!(err, Error::RetryLater { .. }); assert!(err.is_retryable()); @@ -417,8 +425,7 @@ mod tests { Ok(new_upgrade_region_reply(id, false, true, None)) }); - let instruction = &state.build_upgrade_region_instruction(&ctx); - state.upgrade_region(&ctx, instruction).await.unwrap(); + state.upgrade_region(&ctx).await.unwrap(); } #[tokio::test] @@ -537,4 +544,31 @@ mod tests { let update_metadata = next.as_any().downcast_ref::().unwrap(); assert_matches!(update_metadata, UpdateMetadata::Rollback); } + + #[tokio::test] + async fn test_upgrade_region_procedure_exceeded_deadline() { + let mut state = Box::::default(); + state.retry_initial_interval = Duration::from_millis(100); + let persistent_context = new_persistent_context(); + let to_peer_id = persistent_context.to_peer.id; + + let mut env = TestingEnv::new(); + let mut ctx = env.context_factory().new_context(persistent_context); + let mailbox_ctx = env.mailbox_context(); + let mailbox = mailbox_ctx.mailbox().clone(); + ctx.volatile_ctx.operations_elapsed = ctx.persistent_ctx.timeout + Duration::from_secs(1); + + let (tx, rx) = tokio::sync::mpsc::channel(1); + mailbox_ctx + .insert_heartbeat_response_receiver(Channel::Datanode(to_peer_id), tx) + .await; + + send_mock_reply(mailbox, rx, |id| { + Ok(new_upgrade_region_reply(id, false, true, None)) + }); + + let (next, _) = state.next(&mut ctx).await.unwrap(); + let update_metadata = next.as_any().downcast_ref::().unwrap(); + assert_matches!(update_metadata, UpdateMetadata::Rollback); + } } diff --git a/src/meta-srv/src/region/supervisor.rs b/src/meta-srv/src/region/supervisor.rs index 8c7dff9b3d..79b305bb80 100644 --- a/src/meta-srv/src/region/supervisor.rs +++ b/src/meta-srv/src/region/supervisor.rs @@ -402,7 +402,7 @@ impl RegionSupervisor { region_id, from_peer, to_peer, - replay_timeout: Duration::from_secs(60), + timeout: Duration::from_secs(60), }; if let Err(err) = self.region_migration_manager.submit_procedure(task).await { diff --git a/src/meta-srv/src/service/procedure.rs b/src/meta-srv/src/service/procedure.rs index d19d0902ae..e20bb2c4db 100644 --- a/src/meta-srv/src/service/procedure.rs +++ b/src/meta-srv/src/service/procedure.rs @@ -111,8 +111,7 @@ impl procedure_service_server::ProcedureService for Metasrv { region_id, from_peer, to_peer, - replay_timeout_secs, - .. + timeout_secs, } = request.into_inner(); let header = header.context(error::MissingRequestHeaderSnafu)?; @@ -134,7 +133,7 @@ impl procedure_service_server::ProcedureService for Metasrv { region_id: region_id.into(), from_peer, to_peer, - replay_timeout: Duration::from_secs(replay_timeout_secs.into()), + timeout: Duration::from_secs(timeout_secs.into()), }) .await? .map(procedure::pid_to_pb_pid); diff --git a/src/mito2/src/engine/catchup_test.rs b/src/mito2/src/engine/catchup_test.rs index de72bb6128..5f4dd3b15a 100644 --- a/src/mito2/src/engine/catchup_test.rs +++ b/src/mito2/src/engine/catchup_test.rs @@ -19,6 +19,9 @@ use api::v1::Rows; use common_error::ext::ErrorExt; use common_error::status_code::StatusCode; use common_recordbatch::RecordBatches; +use common_wal::options::{KafkaWalOptions, WalOptions, WAL_OPTIONS_KEY}; +use rstest::rstest; +use rstest_reuse::{self, apply}; use store_api::region_engine::{RegionEngine, SetReadonlyResponse}; use store_api::region_request::{RegionCatchupRequest, RegionOpenRequest, RegionRequest}; use store_api::storage::{RegionId, ScanRequest}; @@ -26,7 +29,8 @@ use store_api::storage::{RegionId, ScanRequest}; use crate::config::MitoConfig; use crate::error::{self, Error}; use crate::test_util::{ - build_rows, flush_region, put_rows, rows_schema, CreateRequestBuilder, TestEnv, + build_rows, flush_region, kafka_log_store_factory, prepare_test_for_kafka_log_store, put_rows, + rows_schema, single_kafka_log_store_factory, CreateRequestBuilder, LogStoreFactory, TestEnv, }; use crate::wal::EntryId; @@ -38,15 +42,23 @@ fn get_last_entry_id(resp: SetReadonlyResponse) -> Option { } } -#[tokio::test] -async fn test_catchup_with_last_entry_id() { +#[apply(single_kafka_log_store_factory)] + +async fn test_catchup_with_last_entry_id(factory: Option) { common_telemetry::init_default_ut_logging(); - let mut env = TestEnv::with_prefix("last_entry_id"); + let Some(factory) = factory else { + return; + }; + + let mut env = TestEnv::with_prefix("last_entry_id").with_log_store_factory(factory.clone()); + let topic = prepare_test_for_kafka_log_store(&factory).await; let leader_engine = env.create_engine(MitoConfig::default()).await; let follower_engine = env.create_follower_engine(MitoConfig::default()).await; let region_id = RegionId::new(1, 1); - let request = CreateRequestBuilder::new().build(); + let request = CreateRequestBuilder::new() + .kafka_topic(topic.clone()) + .build(); let region_dir = request.region_dir.clone(); let column_schemas = rows_schema(&request); @@ -55,13 +67,23 @@ async fn test_catchup_with_last_entry_id() { .await .unwrap(); + let mut options = HashMap::new(); + if let Some(topic) = &topic { + options.insert( + WAL_OPTIONS_KEY.to_string(), + serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions { + topic: topic.to_string(), + })) + .unwrap(), + ); + }; follower_engine .handle_request( region_id, RegionRequest::Open(RegionOpenRequest { engine: String::new(), region_dir, - options: HashMap::default(), + options, skip_wal_replay: false, }), ) @@ -135,15 +157,23 @@ async fn test_catchup_with_last_entry_id() { assert!(resp.is_ok()); } -#[tokio::test] -async fn test_catchup_with_incorrect_last_entry_id() { +#[apply(single_kafka_log_store_factory)] +async fn test_catchup_with_incorrect_last_entry_id(factory: Option) { common_telemetry::init_default_ut_logging(); - let mut env = TestEnv::with_prefix("incorrect_last_entry_id"); + let Some(factory) = factory else { + return; + }; + + let mut env = + TestEnv::with_prefix("incorrect_last_entry_id").with_log_store_factory(factory.clone()); + let topic = prepare_test_for_kafka_log_store(&factory).await; let leader_engine = env.create_engine(MitoConfig::default()).await; let follower_engine = env.create_follower_engine(MitoConfig::default()).await; let region_id = RegionId::new(1, 1); - let request = CreateRequestBuilder::new().build(); + let request = CreateRequestBuilder::new() + .kafka_topic(topic.clone()) + .build(); let region_dir = request.region_dir.clone(); let column_schemas = rows_schema(&request); @@ -152,13 +182,23 @@ async fn test_catchup_with_incorrect_last_entry_id() { .await .unwrap(); + let mut options = HashMap::new(); + if let Some(topic) = &topic { + options.insert( + WAL_OPTIONS_KEY.to_string(), + serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions { + topic: topic.to_string(), + })) + .unwrap(), + ); + }; follower_engine .handle_request( region_id, RegionRequest::Open(RegionOpenRequest { engine: String::new(), region_dir, - options: HashMap::default(), + options, skip_wal_replay: false, }), ) @@ -217,14 +257,23 @@ async fn test_catchup_with_incorrect_last_entry_id() { assert!(resp.is_ok()); } -#[tokio::test] -async fn test_catchup_without_last_entry_id() { - let mut env = TestEnv::with_prefix("without_last_entry_id"); +#[apply(single_kafka_log_store_factory)] +async fn test_catchup_without_last_entry_id(factory: Option) { + common_telemetry::init_default_ut_logging(); + let Some(factory) = factory else { + return; + }; + + let mut env = + TestEnv::with_prefix("without_last_entry_id").with_log_store_factory(factory.clone()); + let topic = prepare_test_for_kafka_log_store(&factory).await; let leader_engine = env.create_engine(MitoConfig::default()).await; let follower_engine = env.create_follower_engine(MitoConfig::default()).await; let region_id = RegionId::new(1, 1); - let request = CreateRequestBuilder::new().build(); + let request = CreateRequestBuilder::new() + .kafka_topic(topic.clone()) + .build(); let region_dir = request.region_dir.clone(); let column_schemas = rows_schema(&request); @@ -233,13 +282,23 @@ async fn test_catchup_without_last_entry_id() { .await .unwrap(); + let mut options = HashMap::new(); + if let Some(topic) = &topic { + options.insert( + WAL_OPTIONS_KEY.to_string(), + serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions { + topic: topic.to_string(), + })) + .unwrap(), + ); + }; follower_engine .handle_request( region_id, RegionRequest::Open(RegionOpenRequest { engine: String::new(), region_dir, - options: HashMap::default(), + options, skip_wal_replay: false, }), ) @@ -299,14 +358,23 @@ async fn test_catchup_without_last_entry_id() { assert!(region.is_writable()); } -#[tokio::test] -async fn test_catchup_with_manifest_update() { - let mut env = TestEnv::with_prefix("without_manifest_update"); +#[apply(single_kafka_log_store_factory)] +async fn test_catchup_with_manifest_update(factory: Option) { + common_telemetry::init_default_ut_logging(); + let Some(factory) = factory else { + return; + }; + + let mut env = + TestEnv::with_prefix("without_manifest_update").with_log_store_factory(factory.clone()); + let topic = prepare_test_for_kafka_log_store(&factory).await; let leader_engine = env.create_engine(MitoConfig::default()).await; let follower_engine = env.create_follower_engine(MitoConfig::default()).await; let region_id = RegionId::new(1, 1); - let request = CreateRequestBuilder::new().build(); + let request = CreateRequestBuilder::new() + .kafka_topic(topic.clone()) + .build(); let region_dir = request.region_dir.clone(); let column_schemas = rows_schema(&request); @@ -315,13 +383,23 @@ async fn test_catchup_with_manifest_update() { .await .unwrap(); + let mut options = HashMap::new(); + if let Some(topic) = &topic { + options.insert( + WAL_OPTIONS_KEY.to_string(), + serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions { + topic: topic.to_string(), + })) + .unwrap(), + ); + }; follower_engine .handle_request( region_id, RegionRequest::Open(RegionOpenRequest { engine: String::new(), region_dir, - options: HashMap::default(), + options, skip_wal_replay: false, }), ) diff --git a/src/mito2/src/test_util.rs b/src/mito2/src/test_util.rs index da0451bc1a..cd449e53fa 100644 --- a/src/mito2/src/test_util.rs +++ b/src/mito2/src/test_util.rs @@ -112,6 +112,12 @@ pub(crate) fn kafka_log_store_factory() -> Option { #[tokio::test] pub(crate) fn multiple_log_store_factories(#[case] factory: Option) {} +#[template] +#[rstest] +#[case::with_kafka(kafka_log_store_factory())] +#[tokio::test] +pub(crate) fn single_kafka_log_store_factory(#[case] factory: Option) {} + #[derive(Clone)] pub(crate) struct RaftEngineLogStoreFactory; diff --git a/src/mito2/src/worker/handle_catchup.rs b/src/mito2/src/worker/handle_catchup.rs index bee00fae5e..505c994d36 100644 --- a/src/mito2/src/worker/handle_catchup.rs +++ b/src/mito2/src/worker/handle_catchup.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use common_telemetry::info; +use common_telemetry::tracing::warn; use snafu::ensure; use store_api::logstore::LogStore; use store_api::region_request::{AffectedRows, RegionCatchupRequest}; @@ -72,38 +73,42 @@ impl RegionWorkerLoop { region }; - let flushed_entry_id = region.version_control.current().last_entry_id; - info!("Trying to replay memtable for region: {region_id}, flushed entry id: {flushed_entry_id}"); - let timer = Instant::now(); - let wal_entry_reader = - self.wal - .wal_entry_reader(®ion.provider, region_id, request.location_id); - let on_region_opened = self.wal.on_region_opened(); - let last_entry_id = replay_memtable( - ®ion.provider, - wal_entry_reader, - region_id, - flushed_entry_id, - ®ion.version_control, - self.config.allow_stale_entries, - on_region_opened, - ) - .await?; - info!( - "Elapsed: {:?}, region: {region_id} catchup finished. last entry id: {last_entry_id}, expected: {:?}.", - timer.elapsed(), - request.entry_id - ); - if let Some(expected_last_entry_id) = request.entry_id { - ensure!( - // The replayed last entry id may be greater than the `expected_last_entry_id`. - last_entry_id >= expected_last_entry_id, - error::UnexpectedReplaySnafu { - region_id, - expected_last_entry_id, - replayed_last_entry_id: last_entry_id, - } + if region.provider.is_remote_wal() { + let flushed_entry_id = region.version_control.current().last_entry_id; + info!("Trying to replay memtable for region: {region_id}, flushed entry id: {flushed_entry_id}"); + let timer = Instant::now(); + let wal_entry_reader = + self.wal + .wal_entry_reader(®ion.provider, region_id, request.location_id); + let on_region_opened = self.wal.on_region_opened(); + let last_entry_id = replay_memtable( + ®ion.provider, + wal_entry_reader, + region_id, + flushed_entry_id, + ®ion.version_control, + self.config.allow_stale_entries, + on_region_opened, ) + .await?; + info!( + "Elapsed: {:?}, region: {region_id} catchup finished. last entry id: {last_entry_id}, expected: {:?}.", + timer.elapsed(), + request.entry_id + ); + if let Some(expected_last_entry_id) = request.entry_id { + ensure!( + // The replayed last entry id may be greater than the `expected_last_entry_id`. + last_entry_id >= expected_last_entry_id, + error::UnexpectedReplaySnafu { + region_id, + expected_last_entry_id, + replayed_last_entry_id: last_entry_id, + } + ) + } + } else { + warn!("Skips to replay memtable for region: {}", region.region_id); } if request.set_writable { diff --git a/src/store-api/src/logstore/provider.rs b/src/store-api/src/logstore/provider.rs index 16f907f3b4..b37b2de907 100644 --- a/src/store-api/src/logstore/provider.rs +++ b/src/store-api/src/logstore/provider.rs @@ -84,6 +84,11 @@ impl Provider { Provider::Kafka(Arc::new(KafkaProvider { topic })) } + /// Returns true if it's remote WAL. + pub fn is_remote_wal(&self) -> bool { + matches!(self, Provider::Kafka(_)) + } + /// Returns the type name. pub fn type_name(&self) -> &'static str { match self { From 0f992183867c145330ec5602b6f50c13a3b9840d Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Sun, 22 Sep 2024 10:39:38 +0800 Subject: [PATCH 029/128] feat: list/array/timezone support for postgres output (#4727) * feat: list/array support for postgres output * fix: implement time zone support for postgrsql * feat: add a geohash function that returns array * fix: typo * fix: lint warnings * test: add sqlness test * refactor: check resolution range before convert value * fix: test result for sqlness * feat: upgrade pgwire apis --- Cargo.lock | 42 +- src/common/function/src/scalars/geo.rs | 3 +- .../function/src/scalars/geo/geohash.rs | 198 +++++++- src/servers/Cargo.toml | 2 +- src/servers/src/postgres.rs | 35 +- src/servers/src/postgres/auth_handler.rs | 4 +- src/servers/src/postgres/handler.rs | 6 +- src/servers/src/postgres/server.rs | 10 +- src/servers/src/postgres/types.rs | 448 ++++++++++++++++-- src/servers/src/postgres/types/datetime.rs | 61 ++- tests-integration/tests/sql.rs | 32 ++ .../standalone/common/function/geo.result | 12 +- .../cases/standalone/common/function/geo.sql | 2 + 13 files changed, 742 insertions(+), 113 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 000d6316ec..013b0a205a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3303,9 +3303,9 @@ dependencies = [ [[package]] name = "derive-new" -version = "0.6.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d150dea618e920167e5973d70ae6ece4385b7164e0d799fe7c122dd0a5d912ad" +checksum = "2cdc8d50f426189eef89dac62fabfa0abb27d5cc008f25bf4156a0203325becc" dependencies = [ "proc-macro2", "quote", @@ -5760,6 +5760,29 @@ dependencies = [ "serde", ] +[[package]] +name = "lazy-regex" +version = "3.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8d8e41c97e6bc7ecb552016274b99fbb5d035e8de288c582d9b933af6677bfda" +dependencies = [ + "lazy-regex-proc_macros", + "once_cell", + "regex-lite", +] + +[[package]] +name = "lazy-regex-proc_macros" +version = "3.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76e1d8b05d672c53cb9c7b920bbba8783845ae4f0b076e02a3db1d02c81b4163" +dependencies = [ + "proc-macro2", + "quote", + "regex", + "syn 2.0.66", +] + [[package]] name = "lazy_static" version = "1.4.0" @@ -7856,20 +7879,22 @@ dependencies = [ [[package]] name = "pgwire" -version = "0.22.0" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3770f56e1e8a608c6de40011b9a00c6b669c14d121024411701b4bc3b2a5be99" +checksum = "ed4ca46dd335b3a030d977be54dfe121b1b9fe22aa8bbd69161ac2434524fc68" dependencies = [ "async-trait", "bytes", "chrono", - "derive-new 0.6.0", + "derive-new 0.7.0", "futures", "hex", + "lazy-regex", "md5", "postgres-types", "rand", "ring 0.17.8", + "rust_decimal", "thiserror", "tokio", "tokio-rustls 0.26.0", @@ -9075,6 +9100,12 @@ dependencies = [ "regex-syntax 0.8.4", ] +[[package]] +name = "regex-lite" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53a49587ad06b26609c52e423de037e7f57f20d53535d66e08c695f347df952a" + [[package]] name = "regex-syntax" version = "0.6.29" @@ -9550,6 +9581,7 @@ dependencies = [ "borsh", "bytes", "num-traits", + "postgres-types", "rand", "rkyv", "serde", diff --git a/src/common/function/src/scalars/geo.rs b/src/common/function/src/scalars/geo.rs index 9e415a3fdd..8ad6a7aef2 100644 --- a/src/common/function/src/scalars/geo.rs +++ b/src/common/function/src/scalars/geo.rs @@ -16,7 +16,7 @@ use std::sync::Arc; mod geohash; mod h3; -use geohash::GeohashFunction; +use geohash::{GeohashFunction, GeohashNeighboursFunction}; use crate::function_registry::FunctionRegistry; @@ -26,6 +26,7 @@ impl GeoFunctions { pub fn register(registry: &FunctionRegistry) { // geohash registry.register(Arc::new(GeohashFunction)); + registry.register(Arc::new(GeohashNeighboursFunction)); // h3 family registry.register(Arc::new(h3::H3LatLngToCell)); registry.register(Arc::new(h3::H3LatLngToCellString)); diff --git a/src/common/function/src/scalars/geo/geohash.rs b/src/common/function/src/scalars/geo/geohash.rs index 2daa8223cc..d35a6a06ff 100644 --- a/src/common/function/src/scalars/geo/geohash.rs +++ b/src/common/function/src/scalars/geo/geohash.rs @@ -20,23 +20,69 @@ use common_query::error::{self, InvalidFuncArgsSnafu, Result}; use common_query::prelude::{Signature, TypeSignature}; use datafusion::logical_expr::Volatility; use datatypes::prelude::ConcreteDataType; -use datatypes::scalars::ScalarVectorBuilder; -use datatypes::value::Value; -use datatypes::vectors::{MutableVector, StringVectorBuilder, VectorRef}; +use datatypes::scalars::{Scalar, ScalarVectorBuilder}; +use datatypes::value::{ListValue, Value}; +use datatypes::vectors::{ListVectorBuilder, MutableVector, StringVectorBuilder, VectorRef}; use geohash::Coord; use snafu::{ensure, ResultExt}; use crate::function::{Function, FunctionContext}; +macro_rules! ensure_resolution_usize { + ($v: ident) => { + if !($v > 0 && $v <= 12) { + Err(BoxedError::new(PlainError::new( + format!("Invalid geohash resolution {}, expect value: [1, 12]", $v), + StatusCode::EngineExecuteQuery, + ))) + .context(error::ExecuteSnafu) + } else { + Ok($v as usize) + } + }; +} + +fn try_into_resolution(v: Value) -> Result { + match v { + Value::Int8(v) => { + ensure_resolution_usize!(v) + } + Value::Int16(v) => { + ensure_resolution_usize!(v) + } + Value::Int32(v) => { + ensure_resolution_usize!(v) + } + Value::Int64(v) => { + ensure_resolution_usize!(v) + } + Value::UInt8(v) => { + ensure_resolution_usize!(v) + } + Value::UInt16(v) => { + ensure_resolution_usize!(v) + } + Value::UInt32(v) => { + ensure_resolution_usize!(v) + } + Value::UInt64(v) => { + ensure_resolution_usize!(v) + } + _ => unreachable!(), + } +} + /// Function that return geohash string for a given geospatial coordinate. #[derive(Clone, Debug, Default)] pub struct GeohashFunction; -const NAME: &str = "geohash"; +impl GeohashFunction { + const NAME: &'static str = "geohash"; +} impl Function for GeohashFunction { fn name(&self) -> &str { - NAME + Self::NAME } fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { @@ -93,17 +139,7 @@ impl Function for GeohashFunction { for i in 0..size { let lat = lat_vec.get(i).as_f64_lossy(); let lon = lon_vec.get(i).as_f64_lossy(); - let r = match resolution_vec.get(i) { - Value::Int8(v) => v as usize, - Value::Int16(v) => v as usize, - Value::Int32(v) => v as usize, - Value::Int64(v) => v as usize, - Value::UInt8(v) => v as usize, - Value::UInt16(v) => v as usize, - Value::UInt32(v) => v as usize, - Value::UInt64(v) => v as usize, - _ => unreachable!(), - }; + let r = try_into_resolution(resolution_vec.get(i))?; let result = match (lat, lon) { (Some(lat), Some(lon)) => { @@ -130,6 +166,134 @@ impl Function for GeohashFunction { impl fmt::Display for GeohashFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", NAME) + write!(f, "{}", Self::NAME) + } +} + +/// Function that return geohash string for a given geospatial coordinate. +#[derive(Clone, Debug, Default)] +pub struct GeohashNeighboursFunction; + +impl GeohashNeighboursFunction { + const NAME: &'static str = "geohash_neighbours"; +} + +impl Function for GeohashNeighboursFunction { + fn name(&self) -> &str { + GeohashNeighboursFunction::NAME + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::list_datatype( + ConcreteDataType::string_datatype(), + )) + } + + fn signature(&self) -> Signature { + let mut signatures = Vec::new(); + for coord_type in &[ + ConcreteDataType::float32_datatype(), + ConcreteDataType::float64_datatype(), + ] { + for resolution_type in &[ + ConcreteDataType::int8_datatype(), + ConcreteDataType::int16_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int64_datatype(), + ConcreteDataType::uint8_datatype(), + ConcreteDataType::uint16_datatype(), + ConcreteDataType::uint32_datatype(), + ConcreteDataType::uint64_datatype(), + ] { + signatures.push(TypeSignature::Exact(vec![ + // latitude + coord_type.clone(), + // longitude + coord_type.clone(), + // resolution + resolution_type.clone(), + ])); + } + } + Signature::one_of(signatures, Volatility::Stable) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 3, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect 3, provided : {}", + columns.len() + ), + } + ); + + let lat_vec = &columns[0]; + let lon_vec = &columns[1]; + let resolution_vec = &columns[2]; + + let size = lat_vec.len(); + let mut results = + ListVectorBuilder::with_type_capacity(ConcreteDataType::string_datatype(), size); + + for i in 0..size { + let lat = lat_vec.get(i).as_f64_lossy(); + let lon = lon_vec.get(i).as_f64_lossy(); + let r = try_into_resolution(resolution_vec.get(i))?; + + let result = match (lat, lon) { + (Some(lat), Some(lon)) => { + let coord = Coord { x: lon, y: lat }; + let encoded = geohash::encode(coord, r) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("Geohash error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?; + let neighbours = geohash::neighbors(&encoded) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("Geohash error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?; + Some(ListValue::new( + vec![ + neighbours.n, + neighbours.nw, + neighbours.w, + neighbours.sw, + neighbours.s, + neighbours.se, + neighbours.e, + neighbours.ne, + ] + .into_iter() + .map(Value::from) + .collect(), + ConcreteDataType::string_datatype(), + )) + } + _ => None, + }; + + if let Some(list_value) = result { + results.push(Some(list_value.as_scalar_ref())); + } else { + results.push(None); + } + } + + Ok(results.to_vector()) + } +} + +impl fmt::Display for GeohashNeighboursFunction { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", GeohashNeighboursFunction::NAME) } } diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 54665b8c68..b94fa17d44 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -71,7 +71,7 @@ openmetrics-parser = "0.4" opensrv-mysql = { git = "https://github.com/datafuselabs/opensrv", rev = "6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb" } opentelemetry-proto.workspace = true parking_lot = "0.12" -pgwire = { version = "0.22", default-features = false, features = ["server-api-ring"] } +pgwire = { version = "0.24.2", default-features = false, features = ["server-api-ring"] } pin-project = "1.0" pipeline.workspace = true postgres-types = { version = "0.2", features = ["with-chrono-0_4", "with-serde_json-1"] } diff --git a/src/servers/src/postgres.rs b/src/servers/src/postgres.rs index 36f6730b43..5e8de2294e 100644 --- a/src/servers/src/postgres.rs +++ b/src/servers/src/postgres.rs @@ -32,7 +32,8 @@ use std::sync::Arc; use ::auth::UserProviderRef; use derive_builder::Builder; use pgwire::api::auth::ServerParameterProvider; -use pgwire::api::ClientInfo; +use pgwire::api::copy::NoopCopyHandler; +use pgwire::api::{ClientInfo, PgWireHandlerFactory}; pub use server::PostgresServer; use session::context::Channel; use session::Session; @@ -68,7 +69,7 @@ impl ServerParameterProvider for GreptimeDBStartupParameters { } } -pub struct PostgresServerHandler { +pub struct PostgresServerHandlerInner { query_handler: ServerSqlQueryHandlerRef, login_verifier: PgLoginVerifier, force_tls: bool, @@ -87,10 +88,35 @@ pub(crate) struct MakePostgresServerHandler { force_tls: bool, } +pub(crate) struct PostgresServerHandler(Arc); + +impl PgWireHandlerFactory for PostgresServerHandler { + type StartupHandler = PostgresServerHandlerInner; + type SimpleQueryHandler = PostgresServerHandlerInner; + type ExtendedQueryHandler = PostgresServerHandlerInner; + type CopyHandler = NoopCopyHandler; + + fn simple_query_handler(&self) -> Arc { + self.0.clone() + } + + fn extended_query_handler(&self) -> Arc { + self.0.clone() + } + + fn startup_handler(&self) -> Arc { + self.0.clone() + } + + fn copy_handler(&self) -> Arc { + Arc::new(NoopCopyHandler) + } +} + impl MakePostgresServerHandler { fn make(&self, addr: Option) -> PostgresServerHandler { let session = Arc::new(Session::new(addr, Channel::Postgres, Default::default())); - PostgresServerHandler { + let handler = PostgresServerHandlerInner { query_handler: self.query_handler.clone(), login_verifier: PgLoginVerifier::new(self.user_provider.clone()), force_tls: self.force_tls, @@ -98,6 +124,7 @@ impl MakePostgresServerHandler { session: session.clone(), query_parser: Arc::new(DefaultQueryParser::new(self.query_handler.clone(), session)), - } + }; + PostgresServerHandler(Arc::new(handler)) } } diff --git a/src/servers/src/postgres/auth_handler.rs b/src/servers/src/postgres/auth_handler.rs index 83d508215d..3f33603858 100644 --- a/src/servers/src/postgres/auth_handler.rs +++ b/src/servers/src/postgres/auth_handler.rs @@ -29,7 +29,7 @@ use pgwire::messages::{PgWireBackendMessage, PgWireFrontendMessage}; use session::Session; use snafu::IntoError; -use super::PostgresServerHandler; +use super::PostgresServerHandlerInner; use crate::error::{AuthSnafu, Result}; use crate::metrics::METRIC_AUTH_FAILURE; use crate::postgres::types::PgErrorCode; @@ -127,7 +127,7 @@ where } #[async_trait] -impl StartupHandler for PostgresServerHandler { +impl StartupHandler for PostgresServerHandlerInner { async fn on_startup( &self, client: &mut C, diff --git a/src/servers/src/postgres/handler.rs b/src/servers/src/postgres/handler.rs index e10a45ddec..158e2cab4d 100644 --- a/src/servers/src/postgres/handler.rs +++ b/src/servers/src/postgres/handler.rs @@ -39,13 +39,13 @@ use sql::dialect::PostgreSqlDialect; use sql::parser::{ParseOptions, ParserContext}; use super::types::*; -use super::{fixtures, PostgresServerHandler}; +use super::{fixtures, PostgresServerHandlerInner}; use crate::error::Result; use crate::query_handler::sql::ServerSqlQueryHandlerRef; use crate::SqlPlan; #[async_trait] -impl SimpleQueryHandler for PostgresServerHandler { +impl SimpleQueryHandler for PostgresServerHandlerInner { #[tracing::instrument(skip_all, fields(protocol = "postgres"))] async fn do_query<'a, C>( &self, @@ -237,7 +237,7 @@ impl QueryParser for DefaultQueryParser { } #[async_trait] -impl ExtendedQueryHandler for PostgresServerHandler { +impl ExtendedQueryHandler for PostgresServerHandlerInner { type Statement = SqlPlan; type QueryParser = DefaultQueryParser; diff --git a/src/servers/src/postgres/server.rs b/src/servers/src/postgres/server.rs index cca9c43181..e904845547 100644 --- a/src/servers/src/postgres/server.rs +++ b/src/servers/src/postgres/server.rs @@ -94,14 +94,8 @@ impl PostgresServer { let _handle = io_runtime.spawn(async move { crate::metrics::METRIC_POSTGRES_CONNECTIONS.inc(); let pg_handler = Arc::new(handler_maker.make(addr)); - let r = process_socket( - io_stream, - tls_acceptor.clone(), - pg_handler.clone(), - pg_handler.clone(), - pg_handler, - ) - .await; + let r = + process_socket(io_stream, tls_acceptor.clone(), pg_handler).await; crate::metrics::METRIC_POSTGRES_CONNECTIONS.dec(); r }); diff --git a/src/servers/src/postgres/types.rs b/src/servers/src/postgres/types.rs index 85b626cb1a..2e4a805ef0 100644 --- a/src/servers/src/postgres/types.rs +++ b/src/servers/src/postgres/types.rs @@ -20,13 +20,14 @@ mod interval; use std::collections::HashMap; use std::ops::Deref; -use chrono::{NaiveDate, NaiveDateTime}; +use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; use common_time::Interval; use datafusion_common::ScalarValue; use datafusion_expr::LogicalPlan; use datatypes::prelude::{ConcreteDataType, Value}; use datatypes::schema::Schema; use datatypes::types::TimestampType; +use datatypes::value::ListValue; use pgwire::api::portal::{Format, Portal}; use pgwire::api::results::{DataRowEncoder, FieldInfo}; use pgwire::api::Type; @@ -58,6 +59,317 @@ pub(super) fn schema_to_pg(origin: &Schema, field_formats: &Format) -> Result>>() } +fn encode_array( + query_ctx: &QueryContextRef, + value_list: &ListValue, + builder: &mut DataRowEncoder, +) -> PgWireResult<()> { + match value_list.datatype() { + &ConcreteDataType::Boolean(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Boolean(v) => Ok(Some(*v)), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected bool",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Int8(_) | &ConcreteDataType::UInt8(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Int8(v) => Ok(Some(*v)), + Value::UInt8(v) => Ok(Some(*v as i8)), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "Invalid list item type, find {v:?}, expected int8 or uint8", + ), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Int16(_) | &ConcreteDataType::UInt16(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Int16(v) => Ok(Some(*v)), + Value::UInt16(v) => Ok(Some(*v as i16)), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "Invalid list item type, find {v:?}, expected int16 or uint16", + ), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Int32(_) | &ConcreteDataType::UInt32(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Int32(v) => Ok(Some(*v)), + Value::UInt32(v) => Ok(Some(*v as i32)), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "Invalid list item type, find {v:?}, expected int32 or uint32", + ), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Int64(_) | &ConcreteDataType::UInt64(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Int64(v) => Ok(Some(*v)), + Value::UInt64(v) => Ok(Some(*v as i64)), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "Invalid list item type, find {v:?}, expected int64 or uint64", + ), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Float32(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Float32(v) => Ok(Some(v.0)), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected float32",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Float64(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Float64(v) => Ok(Some(v.0)), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected float64",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Binary(_) => { + let bytea_output = query_ctx.configuration_parameter().postgres_bytea_output(); + + match *bytea_output { + PGByteaOutputValue::ESCAPE => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Binary(v) => Ok(Some(EscapeOutputBytea(v.deref()))), + + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "Invalid list item type, find {v:?}, expected binary", + ), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + PGByteaOutputValue::HEX => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Binary(v) => Ok(Some(HexOutputBytea(v.deref()))), + + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "Invalid list item type, find {v:?}, expected binary", + ), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + } + } + &ConcreteDataType::String(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::String(v) => Ok(Some(v.as_utf8())), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected string",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Date(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Date(v) => { + if let Some(date) = v.to_chrono_date() { + let (style, order) = + *query_ctx.configuration_parameter().pg_datetime_style(); + Ok(Some(StylingDate(date, style, order))) + } else { + Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Failed to convert date to postgres type {v:?}",), + }))) + } + } + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected date",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::DateTime(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::DateTime(v) => { + if let Some(datetime) = + v.to_chrono_datetime_with_timezone(Some(&query_ctx.timezone())) + { + let (style, order) = + *query_ctx.configuration_parameter().pg_datetime_style(); + Ok(Some(StylingDateTime(datetime, style, order))) + } else { + Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Failed to convert date to postgres type {v:?}",), + }))) + } + } + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected date",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Timestamp(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Timestamp(v) => { + if let Some(datetime) = + v.to_chrono_datetime_with_timezone(Some(&query_ctx.timezone())) + { + let (style, order) = + *query_ctx.configuration_parameter().pg_datetime_style(); + Ok(Some(StylingDateTime(datetime, style, order))) + } else { + Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Failed to convert date to postgres type {v:?}",), + }))) + } + } + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected timestamp",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Time(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Time(v) => Ok(v.to_chrono_time()), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected time",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Interval(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Interval(v) => Ok(Some(PgInterval::from(*v))), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected interval",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Decimal128(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Decimal128(v) => Ok(Some(v.to_string())), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected decimal",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + &ConcreteDataType::Json(_) => { + let array = value_list + .items() + .iter() + .map(|v| match v { + Value::Null => Ok(None), + Value::Binary(v) => Ok(Some(jsonb::to_string(v))), + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!("Invalid list item type, find {v:?}, expected json",), + }))), + }) + .collect::>>>()?; + builder.encode_field(&array) + } + _ => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "cannot write array type {:?} in postgres protocol: unimplemented", + value_list.datatype() + ), + }))), + } +} + pub(super) fn encode_value( query_ctx: &QueryContextRef, value: &Value, @@ -93,7 +405,7 @@ pub(super) fn encode_value( Value::Date(v) => { if let Some(date) = v.to_chrono_date() { let (style, order) = *query_ctx.configuration_parameter().pg_datetime_style(); - builder.encode_field(&StylingDate(&date, style, order)) + builder.encode_field(&StylingDate(date, style, order)) } else { Err(PgWireError::ApiError(Box::new(Error::Internal { err_msg: format!("Failed to convert date to postgres type {v:?}",), @@ -101,9 +413,10 @@ pub(super) fn encode_value( } } Value::DateTime(v) => { - if let Some(datetime) = v.to_chrono_datetime() { + if let Some(datetime) = v.to_chrono_datetime_with_timezone(Some(&query_ctx.timezone())) + { let (style, order) = *query_ctx.configuration_parameter().pg_datetime_style(); - builder.encode_field(&StylingDateTime(&datetime, style, order)) + builder.encode_field(&StylingDateTime(datetime, style, order)) } else { Err(PgWireError::ApiError(Box::new(Error::Internal { err_msg: format!("Failed to convert date to postgres type {v:?}",), @@ -111,9 +424,10 @@ pub(super) fn encode_value( } } Value::Timestamp(v) => { - if let Some(datetime) = v.to_chrono_datetime() { + if let Some(datetime) = v.to_chrono_datetime_with_timezone(Some(&query_ctx.timezone())) + { let (style, order) = *query_ctx.configuration_parameter().pg_datetime_style(); - builder.encode_field(&StylingDateTime(&datetime, style, order)) + builder.encode_field(&StylingDateTime(datetime, style, order)) } else { Err(PgWireError::ApiError(Box::new(Error::Internal { err_msg: format!("Failed to convert date to postgres type {v:?}",), @@ -131,14 +445,13 @@ pub(super) fn encode_value( } Value::Interval(v) => builder.encode_field(&PgInterval::from(*v)), Value::Decimal128(v) => builder.encode_field(&v.to_string()), - Value::List(_) | Value::Duration(_) => { - Err(PgWireError::ApiError(Box::new(Error::Internal { - err_msg: format!( - "cannot write value {:?} in postgres protocol: unimplemented", - &value - ), - }))) - } + Value::List(values) => encode_array(query_ctx, values, builder), + Value::Duration(_) => Err(PgWireError::ApiError(Box::new(Error::Internal { + err_msg: format!( + "cannot write value {:?} in postgres protocol: unimplemented", + &value + ), + }))), } } @@ -155,19 +468,45 @@ pub(super) fn type_gt_to_pg(origin: &ConcreteDataType) -> Result { &ConcreteDataType::Binary(_) => Ok(Type::BYTEA), &ConcreteDataType::String(_) => Ok(Type::VARCHAR), &ConcreteDataType::Date(_) => Ok(Type::DATE), - &ConcreteDataType::DateTime(_) => Ok(Type::TIMESTAMP), - &ConcreteDataType::Timestamp(_) => Ok(Type::TIMESTAMP), + &ConcreteDataType::DateTime(_) | &ConcreteDataType::Timestamp(_) => Ok(Type::TIMESTAMP), &ConcreteDataType::Time(_) => Ok(Type::TIME), &ConcreteDataType::Interval(_) => Ok(Type::INTERVAL), &ConcreteDataType::Decimal128(_) => Ok(Type::NUMERIC), &ConcreteDataType::Json(_) => Ok(Type::JSON), - &ConcreteDataType::Duration(_) - | &ConcreteDataType::List(_) - | &ConcreteDataType::Dictionary(_) => server_error::UnsupportedDataTypeSnafu { - data_type: origin, - reason: "not implemented", + ConcreteDataType::List(list) => match list.item_type() { + &ConcreteDataType::Null(_) => Ok(Type::UNKNOWN), + &ConcreteDataType::Boolean(_) => Ok(Type::BOOL_ARRAY), + &ConcreteDataType::Int8(_) | &ConcreteDataType::UInt8(_) => Ok(Type::CHAR_ARRAY), + &ConcreteDataType::Int16(_) | &ConcreteDataType::UInt16(_) => Ok(Type::INT2_ARRAY), + &ConcreteDataType::Int32(_) | &ConcreteDataType::UInt32(_) => Ok(Type::INT4_ARRAY), + &ConcreteDataType::Int64(_) | &ConcreteDataType::UInt64(_) => Ok(Type::INT8_ARRAY), + &ConcreteDataType::Float32(_) => Ok(Type::FLOAT4_ARRAY), + &ConcreteDataType::Float64(_) => Ok(Type::FLOAT8_ARRAY), + &ConcreteDataType::Binary(_) => Ok(Type::BYTEA_ARRAY), + &ConcreteDataType::String(_) => Ok(Type::VARCHAR_ARRAY), + &ConcreteDataType::Date(_) => Ok(Type::DATE_ARRAY), + &ConcreteDataType::DateTime(_) | &ConcreteDataType::Timestamp(_) => { + Ok(Type::TIMESTAMP_ARRAY) + } + &ConcreteDataType::Time(_) => Ok(Type::TIME_ARRAY), + &ConcreteDataType::Interval(_) => Ok(Type::INTERVAL_ARRAY), + &ConcreteDataType::Decimal128(_) => Ok(Type::NUMERIC_ARRAY), + &ConcreteDataType::Json(_) => Ok(Type::JSON_ARRAY), + &ConcreteDataType::Duration(_) + | &ConcreteDataType::Dictionary(_) + | &ConcreteDataType::List(_) => server_error::UnsupportedDataTypeSnafu { + data_type: origin, + reason: "not implemented", + } + .fail(), + }, + &ConcreteDataType::Duration(_) | &ConcreteDataType::Dictionary(_) => { + server_error::UnsupportedDataTypeSnafu { + data_type: origin, + reason: "not implemented", + } + .fail() } - .fail(), } } @@ -621,6 +960,7 @@ mod test { use common_time::interval::IntervalUnit; use common_time::timestamp::TimeUnit; + use common_time::Timestamp; use datatypes::schema::{ColumnSchema, Schema}; use datatypes::value::ListValue; use pgwire::api::results::{FieldFormat, FieldInfo}; @@ -816,6 +1156,34 @@ mod test { Type::INTERVAL, FieldFormat::Text, ), + FieldInfo::new( + "int_list".into(), + None, + None, + Type::INT8_ARRAY, + FieldFormat::Text, + ), + FieldInfo::new( + "float_list".into(), + None, + None, + Type::FLOAT8_ARRAY, + FieldFormat::Text, + ), + FieldInfo::new( + "string_list".into(), + None, + None, + Type::VARCHAR_ARRAY, + FieldFormat::Text, + ), + FieldInfo::new( + "timestamp_list".into(), + None, + None, + Type::TIMESTAMP_ARRAY, + FieldFormat::Text, + ), ]; let datatypes = vec![ @@ -846,6 +1214,10 @@ mod test { ConcreteDataType::datetime_datatype(), ConcreteDataType::timestamp_datatype(TimeUnit::Second), ConcreteDataType::interval_datatype(IntervalUnit::YearMonth), + ConcreteDataType::list_datatype(ConcreteDataType::int64_datatype()), + ConcreteDataType::list_datatype(ConcreteDataType::float64_datatype()), + ConcreteDataType::list_datatype(ConcreteDataType::string_datatype()), + ConcreteDataType::list_datatype(ConcreteDataType::timestamp_second_datatype()), ]; let values = vec![ Value::Null, @@ -875,6 +1247,22 @@ mod test { Value::DateTime(1000001i64.into()), Value::Timestamp(1000001i64.into()), Value::Interval(1000001i128.into()), + Value::List(ListValue::new( + vec![Value::Int64(1i64)], + ConcreteDataType::int64_datatype(), + )), + Value::List(ListValue::new( + vec![Value::Float64(1.0f64.into())], + ConcreteDataType::float64_datatype(), + )), + Value::List(ListValue::new( + vec![Value::String("tom".into())], + ConcreteDataType::string_datatype(), + )), + Value::List(ListValue::new( + vec![Value::Timestamp(Timestamp::new(1i64, TimeUnit::Second))], + ConcreteDataType::timestamp_second_datatype(), + )), ]; let query_context = QueryContextBuilder::default() .configuration_parameter(Default::default()) @@ -884,22 +1272,6 @@ mod test { for (value, datatype) in values.iter().zip(datatypes) { encode_value(&query_context, value, &mut builder, &datatype).unwrap(); } - - let err = encode_value( - &query_context, - &Value::List(ListValue::new(vec![], ConcreteDataType::int16_datatype())), - &mut builder, - &ConcreteDataType::list_datatype(ConcreteDataType::int16_datatype()), - ) - .unwrap_err(); - match err { - PgWireError::ApiError(e) => { - assert!(format!("{e}").contains("Internal error:")); - } - _ => { - unreachable!() - } - } } #[test] diff --git a/src/servers/src/postgres/types/datetime.rs b/src/servers/src/postgres/types/datetime.rs index fc324c047b..700f6dc2b5 100644 --- a/src/servers/src/postgres/types/datetime.rs +++ b/src/servers/src/postgres/types/datetime.rs @@ -19,10 +19,10 @@ use postgres_types::{IsNull, ToSql, Type}; use session::session_config::{PGDateOrder, PGDateTimeStyle}; #[derive(Debug)] -pub struct StylingDate<'a>(pub &'a NaiveDate, pub PGDateTimeStyle, pub PGDateOrder); +pub struct StylingDate(pub NaiveDate, pub PGDateTimeStyle, pub PGDateOrder); #[derive(Debug)] -pub struct StylingDateTime<'a>(pub &'a NaiveDateTime, pub PGDateTimeStyle, pub PGDateOrder); +pub struct StylingDateTime(pub NaiveDateTime, pub PGDateTimeStyle, pub PGDateOrder); fn date_format_string(style: PGDateTimeStyle, order: PGDateOrder) -> &'static str { match style { @@ -53,7 +53,7 @@ fn datetime_format_string(style: PGDateTimeStyle, order: PGDateOrder) -> &'stati }, } } -impl ToSqlText for StylingDate<'_> { +impl ToSqlText for StylingDate { fn to_sql_text( &self, ty: &Type, @@ -78,7 +78,7 @@ impl ToSqlText for StylingDate<'_> { } } -impl ToSqlText for StylingDateTime<'_> { +impl ToSqlText for StylingDateTime { fn to_sql_text( &self, ty: &Type, @@ -112,7 +112,7 @@ impl ToSqlText for StylingDateTime<'_> { macro_rules! delegate_to_sql { ($delegator:ident, $delegatee:ident) => { - impl ToSql for $delegator<'_> { + impl ToSql for $delegator { fn to_sql( &self, ty: &Type, @@ -148,7 +148,7 @@ mod tests { let naive_date = NaiveDate::from_ymd_opt(1997, 12, 17).unwrap(); { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::ISO, PGDateOrder::MDY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::ISO, PGDateOrder::MDY); let expected = "1997-12-17"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -157,7 +157,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::ISO, PGDateOrder::YMD); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::ISO, PGDateOrder::YMD); let expected = "1997-12-17"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -166,7 +166,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::ISO, PGDateOrder::DMY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::ISO, PGDateOrder::DMY); let expected = "1997-12-17"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -175,7 +175,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::German, PGDateOrder::MDY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::German, PGDateOrder::MDY); let expected = "17.12.1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -184,7 +184,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::German, PGDateOrder::YMD); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::German, PGDateOrder::YMD); let expected = "17.12.1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -193,7 +193,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::German, PGDateOrder::DMY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::German, PGDateOrder::DMY); let expected = "17.12.1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -202,8 +202,7 @@ mod tests { } { - let styling_date = - StylingDate(&naive_date, PGDateTimeStyle::Postgres, PGDateOrder::MDY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::Postgres, PGDateOrder::MDY); let expected = "12-17-1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -212,8 +211,7 @@ mod tests { } { - let styling_date = - StylingDate(&naive_date, PGDateTimeStyle::Postgres, PGDateOrder::YMD); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::Postgres, PGDateOrder::YMD); let expected = "12-17-1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -222,8 +220,7 @@ mod tests { } { - let styling_date = - StylingDate(&naive_date, PGDateTimeStyle::Postgres, PGDateOrder::DMY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::Postgres, PGDateOrder::DMY); let expected = "17-12-1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -232,7 +229,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::SQL, PGDateOrder::MDY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::SQL, PGDateOrder::MDY); let expected = "12/17/1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -241,7 +238,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::SQL, PGDateOrder::YMD); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::SQL, PGDateOrder::YMD); let expected = "12/17/1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -250,7 +247,7 @@ mod tests { } { - let styling_date = StylingDate(&naive_date, PGDateTimeStyle::SQL, PGDateOrder::DMY); + let styling_date = StylingDate(naive_date, PGDateTimeStyle::SQL, PGDateOrder::DMY); let expected = "17/12/1997"; let mut out = bytes::BytesMut::new(); let is_null = styling_date.to_sql_text(&Type::DATE, &mut out).unwrap(); @@ -266,7 +263,7 @@ mod tests { .unwrap(); { - let styling_datetime = StylingDateTime(&input, PGDateTimeStyle::ISO, PGDateOrder::MDY); + let styling_datetime = StylingDateTime(input, PGDateTimeStyle::ISO, PGDateOrder::MDY); let expected = "2021-09-01 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -277,7 +274,7 @@ mod tests { } { - let styling_datetime = StylingDateTime(&input, PGDateTimeStyle::ISO, PGDateOrder::YMD); + let styling_datetime = StylingDateTime(input, PGDateTimeStyle::ISO, PGDateOrder::YMD); let expected = "2021-09-01 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -288,7 +285,7 @@ mod tests { } { - let styling_datetime = StylingDateTime(&input, PGDateTimeStyle::ISO, PGDateOrder::DMY); + let styling_datetime = StylingDateTime(input, PGDateTimeStyle::ISO, PGDateOrder::DMY); let expected = "2021-09-01 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -300,7 +297,7 @@ mod tests { { let styling_datetime = - StylingDateTime(&input, PGDateTimeStyle::German, PGDateOrder::MDY); + StylingDateTime(input, PGDateTimeStyle::German, PGDateOrder::MDY); let expected = "01.09.2021 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -312,7 +309,7 @@ mod tests { { let styling_datetime = - StylingDateTime(&input, PGDateTimeStyle::German, PGDateOrder::YMD); + StylingDateTime(input, PGDateTimeStyle::German, PGDateOrder::YMD); let expected = "01.09.2021 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -324,7 +321,7 @@ mod tests { { let styling_datetime = - StylingDateTime(&input, PGDateTimeStyle::German, PGDateOrder::DMY); + StylingDateTime(input, PGDateTimeStyle::German, PGDateOrder::DMY); let expected = "01.09.2021 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -336,7 +333,7 @@ mod tests { { let styling_datetime = - StylingDateTime(&input, PGDateTimeStyle::Postgres, PGDateOrder::MDY); + StylingDateTime(input, PGDateTimeStyle::Postgres, PGDateOrder::MDY); let expected = "Wed Sep 01 12:34:56.789012 2021"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -348,7 +345,7 @@ mod tests { { let styling_datetime = - StylingDateTime(&input, PGDateTimeStyle::Postgres, PGDateOrder::YMD); + StylingDateTime(input, PGDateTimeStyle::Postgres, PGDateOrder::YMD); let expected = "Wed Sep 01 12:34:56.789012 2021"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -360,7 +357,7 @@ mod tests { { let styling_datetime = - StylingDateTime(&input, PGDateTimeStyle::Postgres, PGDateOrder::DMY); + StylingDateTime(input, PGDateTimeStyle::Postgres, PGDateOrder::DMY); let expected = "Wed 01 Sep 12:34:56.789012 2021"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -371,7 +368,7 @@ mod tests { } { - let styling_datetime = StylingDateTime(&input, PGDateTimeStyle::SQL, PGDateOrder::MDY); + let styling_datetime = StylingDateTime(input, PGDateTimeStyle::SQL, PGDateOrder::MDY); let expected = "09/01/2021 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -382,7 +379,7 @@ mod tests { } { - let styling_datetime = StylingDateTime(&input, PGDateTimeStyle::SQL, PGDateOrder::YMD); + let styling_datetime = StylingDateTime(input, PGDateTimeStyle::SQL, PGDateOrder::YMD); let expected = "09/01/2021 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime @@ -393,7 +390,7 @@ mod tests { } { - let styling_datetime = StylingDateTime(&input, PGDateTimeStyle::SQL, PGDateOrder::DMY); + let styling_datetime = StylingDateTime(input, PGDateTimeStyle::SQL, PGDateOrder::DMY); let expected = "01/09/2021 12:34:56.789012"; let mut out = bytes::BytesMut::new(); let is_null = styling_datetime diff --git a/tests-integration/tests/sql.rs b/tests-integration/tests/sql.rs index 1e87c54e5f..19acc37ea6 100644 --- a/tests-integration/tests/sql.rs +++ b/tests-integration/tests/sql.rs @@ -69,6 +69,7 @@ macro_rules! sql_tests { test_postgres_bytea, test_postgres_datestyle, test_postgres_parameter_inference, + test_postgres_array_types, test_mysql_prepare_stmt_insert_timestamp, ); )* @@ -1111,3 +1112,34 @@ pub async fn test_mysql_prepare_stmt_insert_timestamp(store_type: StorageType) { let _ = server.shutdown().await; guard.remove_all().await; } + +pub async fn test_postgres_array_types(store_type: StorageType) { + let (addr, mut guard, fe_pg_server) = setup_pg_server(store_type, "sql_inference").await; + + let (client, connection) = tokio_postgres::connect(&format!("postgres://{addr}/public"), NoTls) + .await + .unwrap(); + + let (tx, rx) = tokio::sync::oneshot::channel(); + tokio::spawn(async move { + connection.await.unwrap(); + tx.send(()).unwrap(); + }); + + let rows = client + .query( + "SELECT arrow_cast(1, 'List(Int8)'), arrow_cast('tom', 'List(Utf8)'), arrow_cast(3.14, 'List(Float32)'), arrow_cast('2023-01-02T12:53:02', 'List(Timestamp(Millisecond, None))')", + &[], + ) + .await + .unwrap(); + + assert_eq!(1, rows.len()); + + // Shutdown the client. + drop(client); + rx.await.unwrap(); + + let _ = fe_pg_server.shutdown().await; + guard.remove_all().await; +} diff --git a/tests/cases/standalone/common/function/geo.result b/tests/cases/standalone/common/function/geo.result index 3a63b5890b..8954447650 100644 --- a/tests/cases/standalone/common/function/geo.result +++ b/tests/cases/standalone/common/function/geo.result @@ -158,11 +158,11 @@ SELECT geohash(37.76938, -122.3889, 11); SELECT geohash(37.76938, -122.3889, 100); -Error: 3001(EngineExecuteQuery), Geohash error: Invalid length specified: 100. Accepted values are between 1 and 12, inclusive +Error: 3001(EngineExecuteQuery), Invalid geohash resolution 100, expect value: [1, 12] SELECT geohash(37.76938, -122.3889, -1); -Error: 3001(EngineExecuteQuery), Geohash error: Invalid length specified: 18446744073709551615. Accepted values are between 1 and 12, inclusive +Error: 3001(EngineExecuteQuery), Invalid geohash resolution -1, expect value: [1, 12] SELECT geohash(37.76938, -122.3889, 11::Int8); @@ -228,3 +228,11 @@ SELECT geohash(37.76938, -122.3889, 11::UInt64); | 9q8yygxneft | +------------------------------------------------------------------------------------+ +SELECT geohash_neighbours(37.76938, -122.3889, 11); + ++----------------------------------------------------------------------------------------------------------+ +| geohash_neighbours(Float64(37.76938),Float64(-122.3889),Int64(11)) | ++----------------------------------------------------------------------------------------------------------+ +| [9q8yygxnefv, 9q8yygxnefu, 9q8yygxnefs, 9q8yygxnefk, 9q8yygxnefm, 9q8yygxnefq, 9q8yygxnefw, 9q8yygxnefy] | ++----------------------------------------------------------------------------------------------------------+ + diff --git a/tests/cases/standalone/common/function/geo.sql b/tests/cases/standalone/common/function/geo.sql index 3a0e668acc..be2b3947bb 100644 --- a/tests/cases/standalone/common/function/geo.sql +++ b/tests/cases/standalone/common/function/geo.sql @@ -64,3 +64,5 @@ SELECT geohash(37.76938, -122.3889, 11::UInt16); SELECT geohash(37.76938, -122.3889, 11::UInt32); SELECT geohash(37.76938, -122.3889, 11::UInt64); + +SELECT geohash_neighbours(37.76938, -122.3889, 11); From 2feddca1cb550a3fd896d116a67c5821897ac7ab Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Mon, 23 Sep 2024 16:35:06 +0800 Subject: [PATCH 030/128] feat: include order by to commutativity rule set (#4753) * feat: include order by to commutativity rule set Signed-off-by: Ruihang Xia * tune sqlness replace interceptor Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- src/query/src/dist_plan/analyzer.rs | 101 ++++++++++++++---- src/query/src/dist_plan/commutativity.rs | 2 +- .../standalone/common/order/order_by.result | 99 +++++++++++++++++ .../standalone/common/order/order_by.sql | 41 +++++++ 4 files changed, 224 insertions(+), 19 deletions(-) diff --git a/src/query/src/dist_plan/analyzer.rs b/src/query/src/dist_plan/analyzer.rs index bbb3e5ddd9..c182eee1cd 100644 --- a/src/query/src/dist_plan/analyzer.rs +++ b/src/query/src/dist_plan/analyzer.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; use std::sync::Arc; use datafusion::datasource::DefaultTableSource; @@ -19,7 +20,8 @@ use datafusion::error::Result as DfResult; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_expr::expr::{Exists, InSubquery}; -use datafusion_expr::{col, Expr, LogicalPlan, LogicalPlanBuilder, Subquery}; +use datafusion_expr::utils::expr_to_columns; +use datafusion_expr::{col as col_fn, Expr, LogicalPlan, LogicalPlanBuilder, Subquery}; use datafusion_optimizer::analyzer::AnalyzerRule; use datafusion_optimizer::simplify_expressions::SimplifyExpressions; use datafusion_optimizer::{OptimizerContext, OptimizerRule}; @@ -104,7 +106,7 @@ impl DistPlannerAnalyzer { let project_exprs = output_schema .fields() .iter() - .map(|f| col(f.name())) + .map(|f| col_fn(f.name())) .collect::>(); rewrote_subquery = LogicalPlanBuilder::from(rewrote_subquery) .project(project_exprs)? @@ -137,6 +139,7 @@ struct PlanRewriter { status: RewriterStatus, /// Partition columns of the table in current pass partition_cols: Option>, + column_requirements: HashSet, } impl PlanRewriter { @@ -162,6 +165,7 @@ impl PlanRewriter { Commutativity::Commutative => {} Commutativity::PartialCommutative => { if let Some(plan) = partial_commutative_transformer(plan) { + self.update_column_requirements(&plan); self.stage.push(plan) } } @@ -169,6 +173,7 @@ impl PlanRewriter { if let Some(transformer) = transformer && let Some(plan) = transformer(plan) { + self.update_column_requirements(&plan); self.stage.push(plan) } } @@ -176,6 +181,7 @@ impl PlanRewriter { if let Some(transformer) = transformer && let Some(plan) = transformer(plan) { + self.update_column_requirements(&plan); self.stage.push(plan) } } @@ -189,6 +195,18 @@ impl PlanRewriter { false } + fn update_column_requirements(&mut self, plan: &LogicalPlan) { + let mut container = HashSet::new(); + for expr in plan.expressions() { + // this method won't fail + let _ = expr_to_columns(&expr, &mut container); + } + + for col in container { + self.column_requirements.insert(col.flat_name()); + } + } + fn is_expanded(&self) -> bool { self.status == RewriterStatus::Expanded } @@ -238,6 +256,67 @@ impl PlanRewriter { self.level -= 1; self.stack.pop(); } + + fn expand(&mut self, mut on_node: LogicalPlan) -> DfResult { + let mut rewriter = EnforceDistRequirementRewriter { + column_requirements: std::mem::take(&mut self.column_requirements), + }; + on_node = on_node.rewrite(&mut rewriter)?.data; + + // add merge scan as the new root + let mut node = MergeScanLogicalPlan::new(on_node, false).into_logical_plan(); + // expand stages + for new_stage in self.stage.drain(..) { + node = new_stage.with_new_exprs(new_stage.expressions(), vec![node.clone()])? + } + self.set_expanded(); + + Ok(node) + } +} + +/// Implementation of the [`TreeNodeRewriter`] trait which is responsible for rewriting +/// logical plans to enforce various requirement for distributed query. +/// +/// Requirements enforced by this rewriter: +/// - Enforce column requirements for `LogicalPlan::Projection` nodes. Makes sure the +/// required columns are available in the sub plan. +struct EnforceDistRequirementRewriter { + column_requirements: HashSet, +} + +impl TreeNodeRewriter for EnforceDistRequirementRewriter { + type Node = LogicalPlan; + + fn f_down(&mut self, node: Self::Node) -> DfResult> { + if let LogicalPlan::Projection(ref projection) = node { + let mut column_requirements = std::mem::take(&mut self.column_requirements); + if column_requirements.is_empty() { + return Ok(Transformed::no(node)); + } + + for expr in &projection.expr { + column_requirements.remove(&expr.name_for_alias()?); + } + if column_requirements.is_empty() { + return Ok(Transformed::no(node)); + } + + let mut new_exprs = projection.expr.clone(); + for col in &column_requirements { + new_exprs.push(col_fn(col)); + } + let new_node = + node.with_new_exprs(new_exprs, node.inputs().into_iter().cloned().collect())?; + return Ok(Transformed::yes(new_node)); + } + + Ok(Transformed::no(node)) + } + + fn f_up(&mut self, node: Self::Node) -> DfResult> { + Ok(Transformed::no(node)) + } } impl TreeNodeRewriter for PlanRewriter { @@ -274,14 +353,7 @@ impl TreeNodeRewriter for PlanRewriter { self.maybe_set_partitions(&node); let Some(parent) = self.get_parent() else { - // add merge scan as the new root - let mut node = MergeScanLogicalPlan::new(node, false).into_logical_plan(); - // expand stages - for new_stage in self.stage.drain(..) { - node = new_stage.with_new_exprs(node.expressions(), vec![node.clone()])? - } - self.set_expanded(); - + let node = self.expand(node)?; self.pop_stack(); return Ok(Transformed::yes(node)); }; @@ -289,14 +361,7 @@ impl TreeNodeRewriter for PlanRewriter { // TODO(ruihang): avoid this clone if self.should_expand(&parent.clone()) { // TODO(ruihang): does this work for nodes with multiple children?; - // replace the current node with expanded one - let mut node = MergeScanLogicalPlan::new(node, false).into_logical_plan(); - // expand stages - for new_stage in self.stage.drain(..) { - node = new_stage.with_new_exprs(node.expressions(), vec![node.clone()])? - } - self.set_expanded(); - + let node = self.expand(node)?; self.pop_stack(); return Ok(Transformed::yes(node)); } diff --git a/src/query/src/dist_plan/commutativity.rs b/src/query/src/dist_plan/commutativity.rs index 6da9d4bf92..c6b3ca2c62 100644 --- a/src/query/src/dist_plan/commutativity.rs +++ b/src/query/src/dist_plan/commutativity.rs @@ -69,7 +69,7 @@ impl Categorizer { // sort plan needs to consider column priority // We can implement a merge-sort on partial ordered data - Commutativity::Unimplemented + Commutativity::PartialCommutative } LogicalPlan::Join(_) => Commutativity::NonCommutative, LogicalPlan::CrossJoin(_) => Commutativity::NonCommutative, diff --git a/tests/cases/standalone/common/order/order_by.result b/tests/cases/standalone/common/order/order_by.result index 2e8014c6a2..f6385537c3 100644 --- a/tests/cases/standalone/common/order/order_by.result +++ b/tests/cases/standalone/common/order/order_by.result @@ -223,3 +223,102 @@ DROP TABLE test; Affected Rows: 0 +-- ORDER BY for partition table +CREATE TABLE IF NOT EXISTS `t` ( + `tag` STRING NULL, + `ts` TIMESTAMP(3) NOT NULL, + `num` BIGINT NULL, + TIME INDEX (`ts`), + PRIMARY KEY (`tag`) +) +PARTITION ON COLUMNS (`tag`) ( + tag <= 'z', + tag > 'z' +); + +Affected Rows: 0 + +INSERT INTO t (tag, ts, num) VALUES + ('abc', 0, 1), + ('abc', 3000, 2), + ('abc', 6000, 3), + ('abc', 9000, 4), + ('abc', 12000, 5), + ('zzz', 3000, 6), + ('zzz', 6000, 7), + ('zzz', 9000, 8), + ('zzz', 0, 9), + ('zzz', 3000, 10); + +Affected Rows: 10 + +select * from t where num > 3 order by ts desc limit 2; + ++-----+---------------------+-----+ +| tag | ts | num | ++-----+---------------------+-----+ +| abc | 1970-01-01T00:00:12 | 5 | +| abc | 1970-01-01T00:00:09 | 4 | ++-----+---------------------+-----+ + +select tag from t where num > 6 order by ts desc limit 2; + ++-----+---------------------+ +| tag | ts | ++-----+---------------------+ +| zzz | 1970-01-01T00:00:09 | +| zzz | 1970-01-01T00:00:06 | ++-----+---------------------+ + +select tag from t where num > 6 order by ts; + ++-----+---------------------+ +| tag | ts | ++-----+---------------------+ +| zzz | 1970-01-01T00:00:00 | +| zzz | 1970-01-01T00:00:03 | +| zzz | 1970-01-01T00:00:06 | +| zzz | 1970-01-01T00:00:09 | ++-----+---------------------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +explain analyze select tag from t where num > 6 order by ts desc limit 2; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_GlobalLimitExec: skip=0, fetch=2 REDACTED +|_|_|_SortPreservingMergeExec: [ts@1 DESC] REDACTED +|_|_|_SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[true] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_GlobalLimitExec: skip=0, fetch=2 REDACTED +|_|_|_SortPreservingMergeExec: [ts@1 DESC] REDACTED +|_|_|_SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[true] REDACTED +|_|_|_ProjectionExec: expr=[tag@0 as tag, ts@1 as ts] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: num@2 > 6 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED +|_|_|_| +| 1_| 1_|_GlobalLimitExec: skip=0, fetch=2 REDACTED +|_|_|_SortPreservingMergeExec: [ts@1 DESC] REDACTED +|_|_|_SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[true] REDACTED +|_|_|_ProjectionExec: expr=[tag@0 as tag, ts@1 as ts] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: num@2 > 6 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED +|_|_|_| +|_|_| Total rows: 2_| ++-+-+-+ + +drop table t; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/order/order_by.sql b/tests/cases/standalone/common/order/order_by.sql index 38ec572978..6e140e5c04 100644 --- a/tests/cases/standalone/common/order/order_by.sql +++ b/tests/cases/standalone/common/order/order_by.sql @@ -56,3 +56,44 @@ SELECT a-10 AS k FROM test UNION SELECT a-10 AS l FROM test ORDER BY a-10; SELECT a-10 AS k FROM test UNION SELECT a-11 AS l FROM test ORDER BY a-11; DROP TABLE test; + +-- ORDER BY for partition table +CREATE TABLE IF NOT EXISTS `t` ( + `tag` STRING NULL, + `ts` TIMESTAMP(3) NOT NULL, + `num` BIGINT NULL, + TIME INDEX (`ts`), + PRIMARY KEY (`tag`) +) +PARTITION ON COLUMNS (`tag`) ( + tag <= 'z', + tag > 'z' +); + +INSERT INTO t (tag, ts, num) VALUES + ('abc', 0, 1), + ('abc', 3000, 2), + ('abc', 6000, 3), + ('abc', 9000, 4), + ('abc', 12000, 5), + ('zzz', 3000, 6), + ('zzz', 6000, 7), + ('zzz', 9000, 8), + ('zzz', 0, 9), + ('zzz', 3000, 10); + +select * from t where num > 3 order by ts desc limit 2; + +select tag from t where num > 6 order by ts desc limit 2; + +select tag from t where num > 6 order by ts; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +explain analyze select tag from t where num > 6 order by ts desc limit 2; + +drop table t; From 5c64f0ce0950e653ac2238c61b37b32a489810c7 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Mon, 23 Sep 2024 18:55:20 +0800 Subject: [PATCH 031/128] refactor!: simplify NativeType trait and remove percentile UDAF (#4758) * refactor!: simplify NativeType trait and remove percentile UDAF Signed-off-by: Ruihang Xia * remove NativeType Signed-off-by: Ruihang Xia * recover a mis-deleted case Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- src/common/function/src/scalars/aggregate.rs | 3 - .../src/scalars/aggregate/percentile.rs | 436 ------------------ src/datatypes/src/types.rs | 2 +- src/datatypes/src/types/primitive_type.rs | 28 +- src/query/src/tests.rs | 1 - src/query/src/tests/percentile_test.rs | 97 ---- .../src/python/rspython/builtins/test.rs | 2 +- .../python/rspython/builtins/testcases.ron | 19 - 8 files changed, 5 insertions(+), 583 deletions(-) delete mode 100644 src/common/function/src/scalars/aggregate/percentile.rs delete mode 100644 src/query/src/tests/percentile_test.rs diff --git a/src/common/function/src/scalars/aggregate.rs b/src/common/function/src/scalars/aggregate.rs index 7ed4530983..c6875a680c 100644 --- a/src/common/function/src/scalars/aggregate.rs +++ b/src/common/function/src/scalars/aggregate.rs @@ -16,7 +16,6 @@ mod argmax; mod argmin; mod diff; mod mean; -mod percentile; mod polyval; mod scipy_stats_norm_cdf; mod scipy_stats_norm_pdf; @@ -28,7 +27,6 @@ pub use argmin::ArgminAccumulatorCreator; use common_query::logical_plan::AggregateFunctionCreatorRef; pub use diff::DiffAccumulatorCreator; pub use mean::MeanAccumulatorCreator; -pub use percentile::PercentileAccumulatorCreator; pub use polyval::PolyvalAccumulatorCreator; pub use scipy_stats_norm_cdf::ScipyStatsNormCdfAccumulatorCreator; pub use scipy_stats_norm_pdf::ScipyStatsNormPdfAccumulatorCreator; @@ -91,7 +89,6 @@ impl AggregateFunctions { register_aggr_func!("polyval", 2, PolyvalAccumulatorCreator); register_aggr_func!("argmax", 1, ArgmaxAccumulatorCreator); register_aggr_func!("argmin", 1, ArgminAccumulatorCreator); - register_aggr_func!("percentile", 2, PercentileAccumulatorCreator); register_aggr_func!("scipystatsnormcdf", 2, ScipyStatsNormCdfAccumulatorCreator); register_aggr_func!("scipystatsnormpdf", 2, ScipyStatsNormPdfAccumulatorCreator); } diff --git a/src/common/function/src/scalars/aggregate/percentile.rs b/src/common/function/src/scalars/aggregate/percentile.rs deleted file mode 100644 index 2d3e5482fe..0000000000 --- a/src/common/function/src/scalars/aggregate/percentile.rs +++ /dev/null @@ -1,436 +0,0 @@ -// 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::cmp::Reverse; -use std::collections::BinaryHeap; -use std::sync::Arc; - -use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; -use common_query::error::{ - self, BadAccumulatorImplSnafu, CreateAccumulatorSnafu, DowncastVectorSnafu, - FromScalarValueSnafu, InvalidInputColSnafu, Result, -}; -use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; -use common_query::prelude::*; -use datatypes::prelude::*; -use datatypes::types::OrdPrimitive; -use datatypes::value::{ListValue, OrderedFloat}; -use datatypes::vectors::{ConstantVector, Float64Vector, Helper, ListVector}; -use datatypes::with_match_primitive_type_id; -use num::NumCast; -use snafu::{ensure, OptionExt, ResultExt}; - -// https://numpy.org/doc/stable/reference/generated/numpy.percentile.html?highlight=percentile#numpy.percentile -// if the p is 50,then the Percentile become median -// we use two heap great and not_greater -// the not_greater push the value that smaller than P-value -// the greater push the value that bigger than P-value -// just like the percentile in numpy: -// Given a vector V of length N, the q-th percentile of V is the value q/100 of the way from the minimum to the maximum in a sorted copy of V. -// The values and distances of the two nearest neighbors as well as the method parameter will determine the percentile -// if the normalized ranking does not match the location of q exactly. -// This function is the same as the median if q=50, the same as the minimum if q=0 and the same as the maximum if q=100. -// This optional method parameter specifies the method to use when the desired quantile lies between two data points i < j. -// If g is the fractional part of the index surrounded by i and alpha and beta are correction constants modifying i and j. -// i+g = (q-alpha)/(n-alpha-beta+1) -// Below, 'q' is the quantile value, 'n' is the sample size and alpha and beta are constants. The following formula gives an interpolation "i + g" of where the quantile would be in the sorted sample. -// With 'i' being the floor and 'g' the fractional part of the result. -// the default method is linear where -// alpha = 1 -// beta = 1 -#[derive(Debug, Default)] -pub struct Percentile -where - T: WrapperType, -{ - greater: BinaryHeap>>, - not_greater: BinaryHeap>, - n: u64, - p: Option, -} - -impl Percentile -where - T: WrapperType, -{ - fn push(&mut self, value: T) { - let value = OrdPrimitive::(value); - - self.n += 1; - if self.not_greater.is_empty() { - self.not_greater.push(value); - return; - } - // to keep the not_greater length == floor+1 - // so to ensure the peek of the not_greater is array[floor] - // and the peek of the greater is array[floor+1] - let p = self.p.unwrap_or(0.0_f64); - let floor = (((self.n - 1) as f64) * p / (100_f64)).floor(); - if value <= *self.not_greater.peek().unwrap() { - self.not_greater.push(value); - if self.not_greater.len() > (floor + 1.0) as usize { - self.greater.push(Reverse(self.not_greater.pop().unwrap())); - } - } else { - self.greater.push(Reverse(value)); - if self.not_greater.len() < (floor + 1.0) as usize { - self.not_greater.push(self.greater.pop().unwrap().0); - } - } - } -} - -impl Accumulator for Percentile -where - T: WrapperType, -{ - fn state(&self) -> Result> { - let nums = self - .greater - .iter() - .map(|x| &x.0) - .chain(self.not_greater.iter()) - .map(|&n| n.into()) - .collect::>(); - Ok(vec![ - Value::List(ListValue::new(nums, T::LogicalType::build_data_type())), - self.p.into(), - ]) - } - - fn update_batch(&mut self, values: &[VectorRef]) -> Result<()> { - if values.is_empty() { - return Ok(()); - } - ensure!(values.len() == 2, InvalidInputStateSnafu); - ensure!(values[0].len() == values[1].len(), InvalidInputStateSnafu); - - if values[0].len() == 0 { - return Ok(()); - } - - // This is a unary accumulator, so only one column is provided. - let column = &values[0]; - let mut len = 1; - let column: &::VectorType = if column.is_const() { - len = column.len(); - let column: &ConstantVector = unsafe { Helper::static_cast(column) }; - unsafe { Helper::static_cast(column.inner()) } - } else { - unsafe { Helper::static_cast(column) } - }; - - let x = &values[1]; - let x = Helper::check_get_scalar::(x).context(error::InvalidInputTypeSnafu { - err_msg: "expecting \"POLYVAL\" function's second argument to be float64", - })?; - // `get(0)` is safe because we have checked `values[1].len() == values[0].len() != 0` - let first = x.get(0); - ensure!(!first.is_null(), InvalidInputColSnafu); - - for i in 1..x.len() { - ensure!(first == x.get(i), InvalidInputColSnafu); - } - - let first = match first { - Value::Float64(OrderedFloat(v)) => v, - // unreachable because we have checked `first` is not null and is i64 above - _ => unreachable!(), - }; - if let Some(p) = self.p { - ensure!(p == first, InvalidInputColSnafu); - } else { - self.p = Some(first); - }; - - (0..len).for_each(|_| { - for v in column.iter_data().flatten() { - self.push(v); - } - }); - Ok(()) - } - - fn merge_batch(&mut self, states: &[VectorRef]) -> Result<()> { - if states.is_empty() { - return Ok(()); - } - - ensure!( - states.len() == 2, - BadAccumulatorImplSnafu { - err_msg: "expect 2 states in `merge_batch`" - } - ); - - let p = &states[1]; - let p = p - .as_any() - .downcast_ref::() - .with_context(|| DowncastVectorSnafu { - err_msg: format!( - "expect float64vector, got vector type {}", - p.vector_type_name() - ), - })?; - let p = p.get(0); - if p.is_null() { - return Ok(()); - } - let p = match p { - Value::Float64(OrderedFloat(p)) => p, - _ => unreachable!(), - }; - self.p = Some(p); - - let values = &states[0]; - let values = values - .as_any() - .downcast_ref::() - .with_context(|| DowncastVectorSnafu { - err_msg: format!( - "expect ListVector, got vector type {}", - values.vector_type_name() - ), - })?; - for value in values.values_iter() { - if let Some(value) = value.context(FromScalarValueSnafu)? { - let column: &::VectorType = unsafe { Helper::static_cast(&value) }; - for v in column.iter_data().flatten() { - self.push(v); - } - } - } - Ok(()) - } - - fn evaluate(&self) -> Result { - if self.not_greater.is_empty() { - assert!( - self.greater.is_empty(), - "not expected in two-heap percentile algorithm, there must be a bug when implementing it" - ); - } - let not_greater = self.not_greater.peek(); - if not_greater.is_none() { - return Ok(Value::Null); - } - let not_greater = (*self.not_greater.peek().unwrap()).as_primitive(); - let percentile = if self.greater.is_empty() { - NumCast::from(not_greater).unwrap() - } else { - let greater = self.greater.peek().unwrap(); - let p = if let Some(p) = self.p { - p - } else { - return Ok(Value::Null); - }; - let fract = (((self.n - 1) as f64) * p / 100_f64).fract(); - let not_greater_v: f64 = NumCast::from(not_greater).unwrap(); - let greater_v: f64 = NumCast::from(greater.0.as_primitive()).unwrap(); - not_greater_v * (1.0 - fract) + greater_v * fract - }; - Ok(Value::from(percentile)) - } -} - -#[as_aggr_func_creator] -#[derive(Debug, Default, AggrFuncTypeStore)] -pub struct PercentileAccumulatorCreator {} - -impl AggregateFunctionCreator for PercentileAccumulatorCreator { - fn creator(&self) -> AccumulatorCreatorFunction { - let creator: AccumulatorCreatorFunction = Arc::new(move |types: &[ConcreteDataType]| { - let input_type = &types[0]; - with_match_primitive_type_id!( - input_type.logical_type_id(), - |$S| { - Ok(Box::new(Percentile::<<$S as LogicalPrimitiveType>::Wrapper>::default())) - }, - { - let err_msg = format!( - "\"PERCENTILE\" aggregate function not support data type {:?}", - input_type.logical_type_id(), - ); - CreateAccumulatorSnafu { err_msg }.fail()? - } - ) - }); - creator - } - - fn output_type(&self) -> Result { - let input_types = self.input_types()?; - ensure!(input_types.len() == 2, InvalidInputStateSnafu); - // unwrap is safe because we have checked input_types len must equals 1 - Ok(ConcreteDataType::float64_datatype()) - } - - fn state_types(&self) -> Result> { - let input_types = self.input_types()?; - ensure!(input_types.len() == 2, InvalidInputStateSnafu); - Ok(vec![ - ConcreteDataType::list_datatype(input_types.into_iter().next().unwrap()), - ConcreteDataType::float64_datatype(), - ]) - } -} - -#[cfg(test)] -mod test { - use datatypes::vectors::{Float64Vector, Int32Vector}; - - use super::*; - #[test] - fn test_update_batch() { - // test update empty batch, expect not updating anything - let mut percentile = Percentile::::default(); - percentile.update_batch(&[]).unwrap(); - assert!(percentile.not_greater.is_empty()); - assert!(percentile.greater.is_empty()); - assert_eq!(Value::Null, percentile.evaluate().unwrap()); - - // test update one not-null value - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(42)])), - Arc::new(Float64Vector::from(vec![Some(100.0_f64)])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(42.0_f64), percentile.evaluate().unwrap()); - - // test update one null value - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Option::::None])), - Arc::new(Float64Vector::from(vec![Some(100.0_f64)])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::Null, percentile.evaluate().unwrap()); - - // test update no null-value batch - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(-1i32), Some(1), Some(2)])), - Arc::new(Float64Vector::from(vec![ - Some(100.0_f64), - Some(100.0_f64), - Some(100.0_f64), - ])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(2_f64), percentile.evaluate().unwrap()); - - // test update null-value batch - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(-2i32), None, Some(3), Some(4)])), - Arc::new(Float64Vector::from(vec![ - Some(100.0_f64), - Some(100.0_f64), - Some(100.0_f64), - Some(100.0_f64), - ])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(4_f64), percentile.evaluate().unwrap()); - - // test update with constant vector - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(ConstantVector::new( - Arc::new(Int32Vector::from_vec(vec![4])), - 2, - )), - Arc::new(Float64Vector::from(vec![Some(100.0_f64), Some(100.0_f64)])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(4_f64), percentile.evaluate().unwrap()); - - // test left border - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(-1i32), Some(1), Some(2)])), - Arc::new(Float64Vector::from(vec![ - Some(0.0_f64), - Some(0.0_f64), - Some(0.0_f64), - ])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(-1.0_f64), percentile.evaluate().unwrap()); - - // test medium - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(-1i32), Some(1), Some(2)])), - Arc::new(Float64Vector::from(vec![ - Some(50.0_f64), - Some(50.0_f64), - Some(50.0_f64), - ])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(1.0_f64), percentile.evaluate().unwrap()); - - // test right border - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(-1i32), Some(1), Some(2)])), - Arc::new(Float64Vector::from(vec![ - Some(100.0_f64), - Some(100.0_f64), - Some(100.0_f64), - ])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(2.0_f64), percentile.evaluate().unwrap()); - - // the following is the result of numpy.percentile - // numpy.percentile - // a = np.array([[10,7,4]]) - // np.percentile(a,40) - // >> 6.400000000000 - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(10i32), Some(7), Some(4)])), - Arc::new(Float64Vector::from(vec![ - Some(40.0_f64), - Some(40.0_f64), - Some(40.0_f64), - ])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!(Value::from(6.400000000_f64), percentile.evaluate().unwrap()); - - // the following is the result of numpy.percentile - // a = np.array([[10,7,4]]) - // np.percentile(a,95) - // >> 9.7000000000000011 - let mut percentile = Percentile::::default(); - let v: Vec = vec![ - Arc::new(Int32Vector::from(vec![Some(10i32), Some(7), Some(4)])), - Arc::new(Float64Vector::from(vec![ - Some(95.0_f64), - Some(95.0_f64), - Some(95.0_f64), - ])), - ]; - percentile.update_batch(&v).unwrap(); - assert_eq!( - Value::from(9.700_000_000_000_001_f64), - percentile.evaluate().unwrap() - ); - } -} diff --git a/src/datatypes/src/types.rs b/src/datatypes/src/types.rs index 0bedd2965c..a0e6d501a6 100644 --- a/src/datatypes/src/types.rs +++ b/src/datatypes/src/types.rs @@ -48,7 +48,7 @@ pub use list_type::ListType; pub use null_type::NullType; pub use primitive_type::{ Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, LogicalPrimitiveType, - NativeType, OrdPrimitive, UInt16Type, UInt32Type, UInt64Type, UInt8Type, WrapperType, + OrdPrimitive, UInt16Type, UInt32Type, UInt64Type, UInt8Type, WrapperType, }; pub use string_type::StringType; pub use time_type::{ diff --git a/src/datatypes/src/types/primitive_type.rs b/src/datatypes/src/types/primitive_type.rs index 800be3d11b..cae71976fd 100644 --- a/src/datatypes/src/types/primitive_type.rs +++ b/src/datatypes/src/types/primitive_type.rs @@ -18,7 +18,6 @@ use std::fmt; use arrow::datatypes::{ArrowNativeType, ArrowPrimitiveType, DataType as ArrowDataType}; use common_time::interval::IntervalUnit; use common_time::{Date, DateTime}; -use num::NumCast; use serde::{Deserialize, Serialize}; use snafu::OptionExt; @@ -31,27 +30,6 @@ use crate::types::{DateTimeType, DateType}; use crate::value::{Value, ValueRef}; use crate::vectors::{MutableVector, PrimitiveVector, PrimitiveVectorBuilder, Vector}; -/// Data types that can be used as arrow's native type. -pub trait NativeType: ArrowNativeType + NumCast {} - -macro_rules! impl_native_type { - ($Type: ident) => { - impl NativeType for $Type {} - }; -} - -impl_native_type!(u8); -impl_native_type!(u16); -impl_native_type!(u32); -impl_native_type!(u64); -impl_native_type!(i8); -impl_native_type!(i16); -impl_native_type!(i32); -impl_native_type!(i64); -impl_native_type!(i128); -impl_native_type!(f32); -impl_native_type!(f64); - /// Represents the wrapper type that wraps a native type using the `newtype pattern`, /// such as [Date](`common_time::Date`) is a wrapper type for the underlying native /// type `i32`. @@ -70,7 +48,7 @@ pub trait WrapperType: /// Logical primitive type that this wrapper type belongs to. type LogicalType: LogicalPrimitiveType; /// The underlying native type. - type Native: NativeType; + type Native: ArrowNativeType; /// Convert native type into this wrapper type. fn from_native(value: Self::Native) -> Self; @@ -84,7 +62,7 @@ pub trait LogicalPrimitiveType: 'static + Sized { /// Arrow primitive type of this logical type. type ArrowPrimitive: ArrowPrimitiveType; /// Native (physical) type of this logical type. - type Native: NativeType; + type Native: ArrowNativeType; /// Wrapper type that the vector returns. type Wrapper: WrapperType + for<'a> Scalar, RefType<'a> = Self::Wrapper> @@ -107,7 +85,7 @@ pub trait LogicalPrimitiveType: 'static + Sized { /// A new type for [WrapperType], complement the `Ord` feature for it. Wrapping non ordered /// primitive types like `f32` and `f64` in `OrdPrimitive` can make them be used in places that -/// require `Ord`. For example, in `Median` or `Percentile` UDAFs. +/// require `Ord`. For example, in `Median` UDAFs. #[derive(Debug, Clone, Copy, PartialEq)] pub struct OrdPrimitive(pub T); diff --git a/src/query/src/tests.rs b/src/query/src/tests.rs index 3ecd69a455..1fee632073 100644 --- a/src/query/src/tests.rs +++ b/src/query/src/tests.rs @@ -25,7 +25,6 @@ mod argmax_test; mod argmin_test; mod mean_test; mod my_sum_udaf_example; -mod percentile_test; mod polyval_test; mod query_engine_test; mod scipy_stats_norm_cdf_test; diff --git a/src/query/src/tests/percentile_test.rs b/src/query/src/tests/percentile_test.rs deleted file mode 100644 index 1c01401b89..0000000000 --- a/src/query/src/tests/percentile_test.rs +++ /dev/null @@ -1,97 +0,0 @@ -// 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::sync::Arc; - -use common_recordbatch::RecordBatch; -use datatypes::for_all_primitive_types; -use datatypes::prelude::*; -use datatypes::schema::{ColumnSchema, Schema}; -use datatypes::vectors::Int32Vector; -use function::{create_query_engine, get_numbers_from_table}; -use num_traits::AsPrimitive; -use table::test_util::MemTable; - -use super::new_query_engine_with_table; -use crate::error::Result; -use crate::tests::{exec_selection, function}; -use crate::QueryEngine; - -#[tokio::test] -async fn test_percentile_aggregator() -> Result<()> { - common_telemetry::init_default_ut_logging(); - let engine = create_query_engine(); - - macro_rules! test_percentile { - ([], $( { $T:ty } ),*) => { - $( - let column_name = format!("{}_number", std::any::type_name::<$T>()); - test_percentile_success::<$T>(&column_name, "numbers", engine.clone()).await?; - )* - } - } - for_all_primitive_types! { test_percentile } - Ok(()) -} - -#[tokio::test] -async fn test_percentile_correctness() -> Result<()> { - let engine = create_correctness_engine(); - let sql = String::from("select PERCENTILE(corr_number,88.0) as percentile from corr_numbers"); - let record_batch = exec_selection(engine, &sql).await; - let column = record_batch[0].column(0); - let value = column.get(0); - assert_eq!(value, Value::from(9.280_000_000_000_001_f64)); - Ok(()) -} - -async fn test_percentile_success( - column_name: &str, - table_name: &str, - engine: Arc, -) -> Result<()> -where - T: WrapperType + AsPrimitive, -{ - let sql = format!("select PERCENTILE({column_name},50.0) as percentile from {table_name}"); - let result = exec_selection(engine.clone(), &sql).await; - let value = function::get_value_from_batches("percentile", result); - - let numbers = get_numbers_from_table::(column_name, table_name, engine.clone()).await; - let expected_value = numbers.iter().map(|&n| n.as_()).collect::>(); - - let expected_value: inc_stats::Percentiles = expected_value.iter().cloned().collect(); - let expected_value = expected_value.percentile(0.5).unwrap(); - assert_eq!(value, expected_value.into()); - Ok(()) -} - -fn create_correctness_engine() -> Arc { - // create engine - - let mut column_schemas = vec![]; - let mut columns = vec![]; - - let column_schema = ColumnSchema::new("corr_number", ConcreteDataType::int32_datatype(), true); - column_schemas.push(column_schema); - - let numbers = [3_i32, 6_i32, 8_i32, 10_i32]; - - let column: VectorRef = Arc::new(Int32Vector::from_slice(numbers)); - columns.push(column); - - let schema = Arc::new(Schema::new(column_schemas)); - let number_table = MemTable::table("corr_numbers", RecordBatch::new(schema, columns).unwrap()); - new_query_engine_with_table(number_table) -} diff --git a/src/script/src/python/rspython/builtins/test.rs b/src/script/src/python/rspython/builtins/test.rs index a108663ad7..3908ab9eef 100644 --- a/src/script/src/python/rspython/builtins/test.rs +++ b/src/script/src/python/rspython/builtins/test.rs @@ -345,7 +345,7 @@ fn run_builtin_fn_testcases() { match case.expect{ Ok(v) => { error!("\nError:\n{err_res}"); - panic!("Expect Ok: {v:?}, found Error"); + panic!("Expect Ok: {v:?}, found Error in case {}", case.script); }, Err(err) => { if !err_res.contains(&err){ diff --git a/src/script/src/python/rspython/builtins/testcases.ron b/src/script/src/python/rspython/builtins/testcases.ron index 0b053538b4..0022d0e874 100644 --- a/src/script/src/python/rspython/builtins/testcases.ron +++ b/src/script/src/python/rspython/builtins/testcases.ron @@ -989,25 +989,6 @@ argmin(p)"#, }, script: r#" from greptime import * -percentile(x, p)"#, - expect: Ok(( - ty: Float64, - value: Float(-0.97) - )) - ), - TestCase( - input: { - "x": Var( - ty: Float64, - value: FloatVec([-1.0, 2.0, 3.0]) - ), - "p": Var( - ty: Float64, - value: FloatVec([0.5, 0.5, 0.5]) - ) - }, - script: r#" -from greptime import * scipy_stats_norm_cdf(x, p)"#, expect: Ok(( ty: Float64, From 54f6e13d13a7035f3b3a0deeac6fc294a0f66507 Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Tue, 24 Sep 2024 11:11:12 +0800 Subject: [PATCH 032/128] fix: Release CI & make rustls use `ring` (#4750) * feat: new dev-build * fix: copy * chore: rm dbg run * fix: binstall install script * chore: typos * fix: update useable image * fix: properly uninstall gcc-9 * chore: another try * chore: print current cc version * chore: another try to release * fix: use gcc-10 by `update-alternatives` * chore: update dev-build image * remove gcc-9 again and install make/cmake * use new image * alias gcc-10/g++-10 to every variant * again..... * fix.... * again release .... * chore: remove auto remove * feat: rustls default to ring * chore: update Cargo.lock * chore: update Cargo.lock * Apply suggestions from code review --------- Co-authored-by: Ruihang Xia --- Cargo.lock | 1850 ++++++++---------- Cargo.toml | 9 + Makefile | 2 +- docker/dev-builder/binstall/pull_binstall.sh | 50 + docker/dev-builder/centos/Dockerfile | 4 +- docker/dev-builder/ubuntu/Dockerfile | 11 +- 6 files changed, 924 insertions(+), 1002 deletions(-) create mode 100755 docker/dev-builder/binstall/pull_binstall.sh diff --git a/Cargo.lock b/Cargo.lock index 013b0a205a..991829a758 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10,18 +10,18 @@ checksum = "fe438c63458706e03479442743baae6c88256498e6431708f6dfc520a26515d3" [[package]] name = "addr2line" -version = "0.22.0" +version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e4503c46a5c0c7844e948c9a4d6acd9f50cccb4de1c48eb9e291ea17470c678" +checksum = "f5fb1d8e4442bd405fdfd1dacb42792696b0cf9cb15882e5d097b742a676d375" dependencies = [ "gimli", ] [[package]] -name = "adler" -version = "1.0.2" +name = "adler2" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" [[package]] name = "adler32" @@ -147,9 +147,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.14" +version = "0.6.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "418c75fa768af9c03be99d17643f93f79bbba589895012a80e3452a19ddda15b" +checksum = "64e15c1ab1f89faffbf04a634d5e1962e9074f2741eef6d97f3c4e322426d526" dependencies = [ "anstyle", "anstyle-parse", @@ -162,33 +162,33 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.7" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "038dfcf04a5feb68e9c60b21c9625a54c2c0616e79b72b0fd87075a056ae1d1b" +checksum = "1bec1de6f59aedf83baf9ff929c98f2ad654b97c9510f4e70cf6f661d49fd5b1" [[package]] name = "anstyle-parse" -version = "0.2.4" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c03a11a9034d92058ceb6ee011ce58af4a9bf61491aa7e1e59ecd24bd40d22d4" +checksum = "eb47de1e80c2b463c735db5b217a0ddc39d612e7ac9e2e96a5aed1f57616c1cb" dependencies = [ "utf8parse", ] [[package]] name = "anstyle-query" -version = "1.1.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad186efb764318d35165f1758e7dcef3b10628e26d41a44bc5550652e6804391" +checksum = "6d36fc52c7f6c869915e99412912f22093507da8d9e942ceaf66fe4b7c14422a" dependencies = [ "windows-sys 0.52.0", ] [[package]] name = "anstyle-wincon" -version = "3.0.3" +version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61a38449feb7068f52bb06c12759005cf459ee52bb4adc1d5a7c4322d716fb19" +checksum = "5bf74e1b6e971609db8ca7a9ce79fd5768ab6ae46441c572e46cf596f59e57f8" dependencies = [ "anstyle", "windows-sys 0.52.0", @@ -196,9 +196,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.86" +version = "1.0.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3d1d046238990b9cf5bcde22a3fb3584ee5cf65fb2765f454ed428c7a0063da" +checksum = "86fdf8605db99b54d3cd748a44c6d04df638eb5dafb219b135d0149bd0db01f6" [[package]] name = "anymap" @@ -282,15 +282,15 @@ checksum = "3d62b7694a562cdf5a74227903507c56ab2cc8bdd1f781ed5cb4cf9c9f810bfc" [[package]] name = "arrayref" -version = "0.3.7" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b4930d2cb77ce62f89ee5d5289b4ac049559b1c45539271f5ed4fdc7db34545" +checksum = "76a2e8124351fda1ef8aaaa3bbd7ebbcb486bbcd4225aca0aa0d84bb2db8fecb" [[package]] name = "arrayvec" -version = "0.7.4" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" @@ -443,7 +443,7 @@ dependencies = [ "arrow-schema", "flatbuffers", "lz4_flex 0.11.3", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -459,7 +459,7 @@ dependencies = [ "arrow-schema", "chrono", "half 2.4.1", - "indexmap 2.2.6", + "indexmap 2.5.0", "lexical-core", "num", "serde", @@ -502,7 +502,7 @@ version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "02d9483aaabe910c4781153ae1b6ae0393f72d9ef757d38d09d450070cf2e528" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "serde", ] @@ -607,9 +607,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.11" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd066d0b4ef8ecb03a55319dc13aa6910616d0f44008a045bb1835af830abff5" +checksum = "fec134f64e2bc57411226dfc4e52dec859ddfc7e711fc5e07b612584f000e4aa" dependencies = [ "brotli 6.0.0", "bzip2", @@ -620,8 +620,8 @@ dependencies = [ "pin-project-lite", "tokio", "xz2", - "zstd 0.13.1", - "zstd-safe 7.1.0", + "zstd 0.13.2", + "zstd-safe 7.2.1", ] [[package]] @@ -654,7 +654,7 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -676,7 +676,7 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -687,13 +687,13 @@ checksum = "8b75356056920673b02621b35afd0f7dda9306d03c79a30f5c56c44cf256e3de" [[package]] name = "async-trait" -version = "0.1.80" +version = "0.1.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -740,9 +740,12 @@ dependencies = [ [[package]] name = "atomic" -version = "0.5.3" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c59bdb34bc650a32731b31bd8f0829cc15d24a708ee31559e0bb34f2bc320cba" +checksum = "8d818003e740b63afc82337e3160717f4f63078720a810b7b903e70a5d1d2994" +dependencies = [ + "bytemuck", +] [[package]] name = "atomic-waker" @@ -788,7 +791,7 @@ checksum = "3c87f3f15e7794432337fc718554eaa4dc8f04c9677a950ffe366f20a162ae42" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -812,25 +815,13 @@ dependencies = [ "cc", ] -[[package]] -name = "aws-lc-rs" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f95446d919226d587817a7d21379e6eb099b97b45110a7f272a444ca5c54070" -dependencies = [ - "aws-lc-sys", - "mirai-annotations", - "paste", - "zeroize", -] - [[package]] name = "aws-lc-sys" -version = "0.21.0" +version = "0.21.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5055edc4a9a1b2a917a818258cdfb86a535947feebd9981adc99667a062c6f85" +checksum = "b3ddc4a5b231dd6958b140ff3151b6412b3f4321fab354f399eec8f14b06df62" dependencies = [ - "bindgen", + "bindgen 0.69.4", "cc", "cmake", "dunce", @@ -853,7 +844,7 @@ dependencies = [ "headers 0.3.9", "http 0.2.12", "http-body 0.4.6", - "hyper 0.14.29", + "hyper 0.14.30", "itoa", "matchit", "memchr", @@ -866,7 +857,7 @@ dependencies = [ "serde_json", "serde_path_to_error", "serde_urlencoded", - "sync_wrapper", + "sync_wrapper 0.1.2", "tokio", "tower", "tower-layer", @@ -899,7 +890,7 @@ dependencies = [ "heck 0.4.1", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -927,26 +918,28 @@ dependencies = [ [[package]] name = "backon" -version = "1.0.2" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2caf634d05fe0642d0fb1ab43497fa627088ecd93f84b2d0f2a5d7b91f7730db" +checksum = "e4fa97bb310c33c811334143cf64c5bb2b7b3c06e453db6b095d7061eff8f113" dependencies = [ "fastrand", + "gloo-timers", + "tokio", ] [[package]] name = "backtrace" -version = "0.3.73" +version = "0.3.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cc23269a4f8976d0a4d2e7109211a419fe30e8d88d677cd60b6bc79c5732e0a" +checksum = "8d82cb332cdfaed17ae235a638438ac4d4839913cc2af585c3c6746e8f8bee1a" dependencies = [ "addr2line", - "cc", "cfg-if", "libc", "miniz_oxide", "object", "rustc-demangle", + "windows-targets 0.52.6", ] [[package]] @@ -1011,7 +1004,7 @@ version = "0.69.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a00dc851838a2120612785d195287475a3ac45514741da670b735818822129a0" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "cexpr", "clang-sys", "itertools 0.12.1", @@ -1024,10 +1017,28 @@ dependencies = [ "regex", "rustc-hash 1.1.0", "shlex", - "syn 2.0.66", + "syn 2.0.77", "which", ] +[[package]] +name = "bindgen" +version = "0.70.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f49d8fed880d473ea71efb9bf597651e77201bdd4893efe54c9e5d65ae04ce6f" +dependencies = [ + "bitflags 2.6.0", + "cexpr", + "clang-sys", + "itertools 0.13.0", + "proc-macro2", + "quote", + "regex", + "rustc-hash 1.1.0", + "shlex", + "syn 2.0.77", +] + [[package]] name = "bit-set" version = "0.5.3" @@ -1051,9 +1062,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1" +checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de" [[package]] name = "bitpacking" @@ -1087,9 +1098,9 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.1" +version = "1.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30cca6d3674597c30ddf2c587bf8d9d65c9a84d2326d941cc79c9842dfe0ef52" +checksum = "d82033247fd8e890df8f740e407ad4d038debb9eb1f40533fffb32e7d17dc6f7" dependencies = [ "arrayref", "arrayvec", @@ -1146,10 +1157,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c3ef8005764f53cd4dca619f5bf64cafd4664dada50ece25e4d81de54c80cc0b" dependencies = [ "once_cell", - "proc-macro-crate 3.1.0", + "proc-macro-crate 3.2.0", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", "syn_derive", ] @@ -1266,9 +1277,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.16.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78834c15cb5d5efe3452d58b1e8ba890dd62d21907f867f383358198e56ebca5" +checksum = "94bbb0ad554ad961ddc5da507a12a29b14e4ae5bda06b19f575a3e6079d2e2ae" [[package]] name = "byteorder" @@ -1278,9 +1289,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.7.1" +version = "1.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8318a53db07bb3f8dca91a600466bdb3f2eaadeedfdbcf02e1accbad9271ba50" +checksum = "428d9aa8fbc0670b7b8d6030a7fadd0f86151cae55e4dbbece15f3780a3dfaf3" dependencies = [ "serde", ] @@ -1424,9 +1435,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.20" +version = "1.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45bcde016d64c21da4be18b655631e5ab6d3107607e71a73a9f53eb48aae23fb" +checksum = "07b1695e2c7e8fc85310cde85aeaab7e3097f593c91d209d3f9df76c928100f0" dependencies = [ "jobserver", "libc", @@ -1493,7 +1504,7 @@ version = "0.13.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6026d8cd82ada8bbcfe337805dd1eb6afdc9e80fa4d57e977b3a36315e0c5525" dependencies = [ - "indexmap 2.2.6", + "indexmap 2.5.0", "lazy_static", "num-traits", "regex", @@ -1513,7 +1524,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-targets 0.52.5", + "windows-targets 0.52.6", ] [[package]] @@ -1611,7 +1622,7 @@ checksum = "0b023947811758c97c59bf9d1c188fd619ad4718dcaa767947df1cadb14f39f4" dependencies = [ "glob", "libc", - "libloading 0.8.3", + "libloading 0.8.5", ] [[package]] @@ -1643,9 +1654,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.7" +version = "4.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5db83dced34638ad474f39f250d7fea9598bdd239eaced1bdf45d597da0f433f" +checksum = "b0956a43b323ac1afaffc053ed5c4b7c1f1800bacd1683c353aabbb752515dd3" dependencies = [ "clap_builder", "clap_derive", @@ -1653,26 +1664,26 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.7" +version = "4.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7e204572485eb3fbf28f871612191521df159bc3e15a9f5064c66dba3a8c05f" +checksum = "4d72166dd41634086d5803a47eb71ae740e61d84709c36f3c34110173db3961b" dependencies = [ "anstream", "anstyle", - "clap_lex 0.7.1", + "clap_lex 0.7.2", "strsim 0.11.1", ] [[package]] name = "clap_derive" -version = "4.5.5" +version = "4.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c780290ccf4fb26629baa7a1081e68ced113f1d3ec302fa5948f1c381ebf06c6" +checksum = "4ac6a0c7b1a9e9a5186361f67dfa1b88213572f427fb9ab038efb2bd8c582dab" dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -1686,9 +1697,9 @@ dependencies = [ [[package]] name = "clap_lex" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b82cf0babdbd58558212896d1a4272303a57bdb245c2bf1147185fb45640e70" +checksum = "1462739cb27611015575c0c11df5df7601141071f07518d56fcc1be504cbec97" [[package]] name = "client" @@ -1743,9 +1754,9 @@ dependencies = [ [[package]] name = "cmake" -version = "0.1.50" +version = "0.1.51" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a31c789563b815f77f4250caee12365734369f942439b7defd71e18a48197130" +checksum = "fb1e43aa7fd152b1f968787f7dbcdeb306d1867ff373c69955211876c053f91a" dependencies = [ "cc", ] @@ -1760,7 +1771,7 @@ dependencies = [ "cache", "catalog", "chrono", - "clap 4.5.7", + "clap 4.5.18", "client", "common-base", "common-catalog", @@ -1815,16 +1826,16 @@ dependencies = [ "tempfile", "tikv-jemallocator", "tokio", - "toml 0.8.14", + "toml 0.8.19", "tonic 0.11.0", "tracing-appender", ] [[package]] name = "colorchoice" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b6a852b24ab71dffc585bcb46eaf7959d175cb865a7152e35b348d1b2960422" +checksum = "d3fd119d74b830634cea2a0f58bbd0d54540518a14397557951e79340abc28c0" [[package]] name = "combine" @@ -1842,7 +1853,7 @@ version = "7.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" dependencies = [ - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "unicode-width", ] @@ -1867,7 +1878,7 @@ dependencies = [ "paste", "serde", "snafu 0.8.4", - "toml 0.8.14", + "toml 0.8.19", "zeroize", ] @@ -1902,7 +1913,7 @@ dependencies = [ "sysinfo", "temp-env", "tempfile", - "toml 0.8.14", + "toml 0.8.19", ] [[package]] @@ -2027,7 +2038,7 @@ dependencies = [ "common-telemetry", "common-test-util", "common-version", - "hyper 0.14.29", + "hyper 0.14.30", "reqwest", "serde", "tempfile", @@ -2090,7 +2101,7 @@ dependencies = [ "quote", "snafu 0.8.4", "static_assertions", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -2139,7 +2150,7 @@ dependencies = [ "futures-util", "hex", "humantime-serde", - "hyper 0.14.29", + "hyper 0.14.30", "itertools 0.10.5", "lazy_static", "moka", @@ -2173,7 +2184,7 @@ version = "0.9.3" dependencies = [ "async-stream", "async-trait", - "backon 1.0.2", + "backon 1.2.0", "common-base", "common-error", "common-macro", @@ -2347,15 +2358,15 @@ dependencies = [ "humantime-serde", "num_cpus", "rskafka", - "rustls 0.23.10", - "rustls-native-certs", - "rustls-pemfile 2.1.2", + "rustls 0.23.13", + "rustls-native-certs 0.7.3", + "rustls-pemfile 2.1.3", "serde", "serde_json", "serde_with", "snafu 0.8.4", "tokio", - "toml 0.8.14", + "toml 0.8.19", ] [[package]] @@ -2475,18 +2486,18 @@ checksum = "373e9fafaa20882876db20562275ff58d50e0caa2590077fe7ce7bef90211d0d" [[package]] name = "const_format" -version = "0.2.32" +version = "0.2.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3a214c7af3d04997541b18d432afaff4c455e79e2029079647e72fc2bd27673" +checksum = "50c655d81ff1114fb0dcdea9225ea9f0cc712a6f8d189378e82bdf62a473a64b" dependencies = [ "const_format_proc_macros", ] [[package]] name = "const_format_proc_macros" -version = "0.2.32" +version = "0.2.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7f6ff08fd20f4f299298a28e2dfa8a8ba1036e6cd2460ac1de7b425d76f2500" +checksum = "eff1a44b93f47b1bac19a27932f5c591e43d1ba357ee4f61526c8a25603f0eb1" dependencies = [ "proc-macro2", "quote", @@ -2495,9 +2506,9 @@ dependencies = [ [[package]] name = "constant_time_eq" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" +checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" [[package]] name = "core-foundation" @@ -2511,9 +2522,9 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.6" +version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" +checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" [[package]] name = "core2" @@ -2526,18 +2537,18 @@ dependencies = [ [[package]] name = "cpp_demangle" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8227005286ec39567949b33df9896bcadfa6051bccca2488129f108ca23119" +checksum = "96e58d342ad113c2b878f16d5d034c03be492ae460cdbc02b7f0f2284d310c7d" dependencies = [ "cfg-if", ] [[package]] name = "cpufeatures" -version = "0.2.12" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" +checksum = "608697df725056feaccfa42cffdaeeec3fccc4ffc38358ecd19b243e716a78e0" dependencies = [ "libc", ] @@ -2612,7 +2623,7 @@ dependencies = [ "anes", "cast", "ciborium", - "clap 4.5.7", + "clap 4.5.18", "criterion-plot", "is-terminal", "itertools 0.10.5", @@ -2764,12 +2775,12 @@ dependencies = [ [[package]] name = "darling" -version = "0.20.9" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83b2eb4d90d12bdda5ed17de686c2acb4c57914f8f921b8da7e112b5a36f3fe1" +checksum = "6f63b86c8a8826a49b8c21f08a2d07338eec8d900540f8630dc76284be802989" dependencies = [ - "darling_core 0.20.9", - "darling_macro 0.20.9", + "darling_core 0.20.10", + "darling_macro 0.20.10", ] [[package]] @@ -2788,16 +2799,16 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.20.9" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "622687fe0bac72a04e5599029151f5796111b90f1baaa9b544d807a5e31cd120" +checksum = "95133861a8032aaea082871032f5815eb9e98cef03fa916ab4500513994df9e5" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim 0.11.1", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -2813,13 +2824,13 @@ dependencies = [ [[package]] name = "darling_macro" -version = "0.20.9" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "733cabb43482b1a1b53eee8583c2b9e8684d592215ea83efd305dd31bc2f0178" +checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ - "darling_core 0.20.9", + "darling_core 0.20.10", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -2851,7 +2862,7 @@ dependencies = [ "arrow-array", "arrow-ipc", "arrow-schema", - "async-compression 0.4.11", + "async-compression 0.4.12", "async-trait", "bytes", "bzip2", @@ -2874,7 +2885,7 @@ dependencies = [ "glob", "half 2.4.1", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.5.0", "itertools 0.12.1", "log", "num_cpus", @@ -2890,7 +2901,7 @@ dependencies = [ "url", "uuid", "xz2", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -2955,7 +2966,7 @@ dependencies = [ "paste", "serde_json", "sqlparser 0.45.0 (registry+https://github.com/rust-lang/crates.io-index)", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", ] @@ -3033,7 +3044,7 @@ dependencies = [ "datafusion-expr", "datafusion-physical-expr", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.5.0", "itertools 0.12.1", "log", "regex-syntax 0.8.4", @@ -3061,7 +3072,7 @@ dependencies = [ "half 2.4.1", "hashbrown 0.14.5", "hex", - "indexmap 2.2.6", + "indexmap 2.5.0", "itertools 0.12.1", "log", "paste", @@ -3103,7 +3114,7 @@ dependencies = [ "futures", "half 2.4.1", "hashbrown 0.14.5", - "indexmap 2.2.6", + "indexmap 2.5.0", "itertools 0.12.1", "log", "once_cell", @@ -3126,7 +3137,7 @@ dependencies = [ "log", "regex", "sqlparser 0.45.0 (registry+https://github.com/rust-lang/crates.io-index)", - "strum 0.26.2", + "strum 0.26.3", ] [[package]] @@ -3198,7 +3209,7 @@ dependencies = [ "substrait 0.9.3", "table", "tokio", - "toml 0.8.14", + "toml 0.8.19", "tonic 0.11.0", ] @@ -3309,7 +3320,7 @@ checksum = "2cdc8d50f426189eef89dac62fabfa0abb27d5cc008f25bf4156a0203325becc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -3320,7 +3331,7 @@ checksum = "67e77553c4162a157adbf834ebae5b415acbecbeafc7a74b0e886657506a7611" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -3343,11 +3354,11 @@ dependencies = [ [[package]] name = "derive_builder" -version = "0.20.0" +version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0350b5cb0331628a5916d6c5c0b72e97393b8b6b03b47a9284f4e7f5a405ffd7" +checksum = "cd33f37ee6a119146a1781d3356a7c26028f83d779b2e04ecd45fdc75c76877b" dependencies = [ - "derive_builder_macro 0.20.0", + "derive_builder_macro 0.20.1", ] [[package]] @@ -3376,14 +3387,14 @@ dependencies = [ [[package]] name = "derive_builder_core" -version = "0.20.0" +version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d48cda787f839151732d396ac69e3473923d54312c070ee21e9effcaa8ca0b1d" +checksum = "7431fa049613920234f22c47fdc33e6cf3ee83067091ea4277a3f8c4587aae38" dependencies = [ - "darling 0.20.9", + "darling 0.20.10", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -3408,12 +3419,12 @@ dependencies = [ [[package]] name = "derive_builder_macro" -version = "0.20.0" +version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "206868b8242f27cecce124c19fd88157fbd0dd334df2587f36417bafbc85097b" +checksum = "4abae7035bf79b9877b779505d8cf3749285b80c43941eda66604841889451dc" dependencies = [ - "derive_builder_core 0.20.0", - "syn 2.0.66", + "derive_builder_core 0.20.1", + "syn 2.0.77", ] [[package]] @@ -3433,7 +3444,7 @@ checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", "unicode-xid", ] @@ -3517,17 +3528,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "displaydoc" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "487585f4d0c6655fe74905e2504d8ad6908e4db67f744eb140876906c2f3175d" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - [[package]] name = "dlv-list" version = "0.3.0" @@ -3602,7 +3602,7 @@ dependencies = [ "serde", "thiserror", "time", - "winnow 0.6.13", + "winnow 0.6.18", ] [[package]] @@ -3613,9 +3613,9 @@ checksum = "0d6ef0072f8a535281e4876be788938b528e9a1d43900b82c2569af7da799125" [[package]] name = "either" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dca9240753cf90908d7e4aac30f630662b02aebaa1b58a3cadabdb23385b58b" +checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" [[package]] name = "ena" @@ -3649,14 +3649,14 @@ checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" [[package]] name = "enum-as-inner" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ffccbb6966c05b32ef8fbac435df276c4ae4d3dc55a8cd0eb9745e6c12f546a" +checksum = "a1e6a265c649f3f5979b601d26f1d05ada116434c87741c9493cb56218f76cbc" dependencies = [ - "heck 0.4.1", + "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -3668,7 +3668,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -3793,9 +3793,9 @@ checksum = "59668941c55e5c186b8b58c391629af56774ec768f73c08bbcd56f09348eb00b" [[package]] name = "fastrand" -version = "2.1.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" +checksum = "e8c02a5121d4ea3eb16a80748c74f5549a5665e4c21333c6098f283870fbdea6" [[package]] name = "fd-lock" @@ -3840,14 +3840,14 @@ dependencies = [ [[package]] name = "filetime" -version = "0.2.23" +version = "0.2.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ee447700ac8aa0b2f2bd7bc4462ad686ba06baa6727ac149a2d6277f0d240fd" +checksum = "35c0522e981e68cbfa8c3f978441a5f34b30b96e146b33cd3359176b50fe8586" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.4.1", - "windows-sys 0.52.0", + "libredox", + "windows-sys 0.59.0", ] [[package]] @@ -3870,9 +3870,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flagset" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdeb3aa5e95cf9aabc17f060cfa0ced7b83f042390760ca53bf09df9968acaa1" +checksum = "b3ea1ec5f8307826a5b71094dd91fc04d4ae75d5709b20ad351c7fb4815c86ec" [[package]] name = "flatbuffers" @@ -3886,9 +3886,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.30" +version = "1.0.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" +checksum = "324a1be68054ef05ad64b861cc9eaf1d623d2d8cb25b4bf2cb9cdd902b4bf253" dependencies = [ "crc32fast", "libz-sys", @@ -4072,7 +4072,7 @@ dependencies = [ "strfmt", "table", "tokio", - "toml 0.8.14", + "toml 0.8.19", "tonic 0.11.0", "tower", "uuid", @@ -4080,54 +4080,58 @@ dependencies = [ [[package]] name = "frunk" -version = "0.4.2" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11a351b59e12f97b4176ee78497dff72e4276fb1ceb13e19056aca7fa0206287" +checksum = "874b6a17738fc273ec753618bac60ddaeac48cb1d7684c3e7bd472e57a28b817" dependencies = [ "frunk_core", "frunk_derives", "frunk_proc_macros", + "serde", ] [[package]] name = "frunk_core" -version = "0.4.2" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af2469fab0bd07e64ccf0ad57a1438f63160c69b2e57f04a439653d68eb558d6" +checksum = "3529a07095650187788833d585c219761114005d5976185760cf794d265b6a5c" +dependencies = [ + "serde", +] [[package]] name = "frunk_derives" -version = "0.4.2" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fa992f1656e1707946bbba340ad244f0814009ef8c0118eb7b658395f19a2e" +checksum = "e99b8b3c28ae0e84b604c75f721c21dc77afb3706076af5e8216d15fd1deaae3" dependencies = [ "frunk_proc_macro_helpers", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "frunk_proc_macro_helpers" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35b54add839292b743aeda6ebedbd8b11e93404f902c56223e51b9ec18a13d2c" +checksum = "05a956ef36c377977e512e227dcad20f68c2786ac7a54dacece3746046fea5ce" dependencies = [ "frunk_core", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "frunk_proc_macros" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71b85a1d4a9a6b300b41c05e8e13ef2feca03e0334127f29eca9506a7fe13a93" +checksum = "67e86c2c9183662713fea27ea527aad20fb15fee635a71081ff91bf93df4dc51" dependencies = [ "frunk_core", "frunk_proc_macro_helpers", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -4257,7 +4261,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -4370,9 +4374,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.29.0" +version = "0.31.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40ecd4077b5ae9fd2e9e169b102c6c330d0605168eb0e8bf79952b256dbefffd" +checksum = "32085ea23f3234fc7846555e85283ba4de91e21016dc0455a16286d87a292d64" [[package]] name = "git2" @@ -4380,7 +4384,7 @@ version = "0.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b903b73e45dc0c6c596f2d37eccece7c1c8bb6e4407b001096387c63d0d93724" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "libc", "libgit2-sys", "log", @@ -4393,6 +4397,18 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" +[[package]] +name = "gloo-timers" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbb143cf96099802033e0d4f4963b19fd2e0b728bcf076cd9cf7f6634f092994" +dependencies = [ + "futures-channel", + "futures-core", + "js-sys", + "wasm-bindgen", +] + [[package]] name = "greptime-proto" version = "0.1.0" @@ -4419,7 +4435,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.2.6", + "indexmap 2.5.0", "slab", "tokio", "tokio-util", @@ -4581,6 +4597,12 @@ version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" +[[package]] +name = "hermit-abi" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbf6a919d6cf397374f7dfeeea91d974c7c0a7221d0d0f4f20d859d329e53fcc" + [[package]] name = "hex" version = "0.4.3" @@ -4681,9 +4703,9 @@ dependencies = [ [[package]] name = "http-body" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cac85db508abc24a2e48553ba12a996e87244a0395ce011e62b37158745d643" +checksum = "1efedce1fb8e6913f23e0c92de8e62cd5b772a67e7b3946df930a62566c93184" dependencies = [ "bytes", "http 1.1.0", @@ -4698,7 +4720,7 @@ dependencies = [ "bytes", "futures-util", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "pin-project-lite", ] @@ -4710,9 +4732,9 @@ checksum = "add0ab9360ddbd88cfeb3bd9574a1d85cfdfa14db10b3e21d3700dbc4328758f" [[package]] name = "httparse" -version = "1.9.3" +version = "1.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0e7a4dd27b9476dc40cb050d3632d3bba3a70ddbff012285f7f8559a1e7e545" +checksum = "0fcc0b4a115bf80b728eb8ea024ad5bd707b615bfed49e0665b6e0f86fd082d9" [[package]] name = "httpdate" @@ -4732,7 +4754,7 @@ dependencies = [ "os_info", "serde", "serde_derive", - "toml 0.8.14", + "toml 0.8.19", "uuid", ] @@ -4793,7 +4815,7 @@ dependencies = [ "proc-macro-crate 1.3.1", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -4808,7 +4830,7 @@ dependencies = [ "rust-sitter", "rust-sitter-tool", "slotmap", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -4817,7 +4839,7 @@ version = "0.6.2" source = "git+https://github.com/GreptimeTeam/hydroflow.git?branch=main#b072ee026f97f8537165e1fb247101e0ab2fb320" dependencies = [ "auto_impl", - "clap 4.5.7", + "clap 4.5.18", "data-encoding", "itertools 0.10.5", "prettyplease", @@ -4827,7 +4849,7 @@ dependencies = [ "serde", "serde_json", "slotmap", - "syn 2.0.66", + "syn 2.0.77", "webbrowser", ] @@ -4841,14 +4863,14 @@ dependencies = [ "proc-macro-crate 1.3.1", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "hyper" -version = "0.14.29" +version = "0.14.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f361cde2f109281a220d4307746cdfd5ee3f410da58a70377762396775634b33" +checksum = "a152ddd61dfaec7273fe8419ab357f33aee0d914c5f4efbf0d96fa749eea5ec9" dependencies = [ "bytes", "futures-channel", @@ -4870,15 +4892,15 @@ dependencies = [ [[package]] name = "hyper" -version = "1.3.1" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe575dd17d0862a9a33781c8c4696a55c320909004a67a00fb286ba8b1bc496d" +checksum = "50dfd22e0e76d0f662d429a5f80fcaf3855009297eab6a0a9f8543834744ba05" dependencies = [ "bytes", "futures-channel", "futures-util", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "httparse", "itoa", "pin-project-lite", @@ -4897,11 +4919,11 @@ dependencies = [ "futures-util", "headers 0.4.0", "http 1.1.0", - "hyper 1.3.1", - "hyper-rustls 0.27.2", + "hyper 1.4.1", + "hyper-rustls", "hyper-util", "pin-project-lite", - "rustls-native-certs", + "rustls-native-certs 0.7.3", "tokio", "tokio-rustls 0.26.0", "tower-service", @@ -4909,38 +4931,21 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.26.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0bea761b46ae2b24eb4aef630d8d1c398157b6fc29e6350ecf090a0b70c952c" +version = "0.27.3" +source = "git+https://github.com/GreptimeTeam/hyper-rustls#a951e03fb914f1830e244400472814d38775118d" dependencies = [ "futures-util", "http 1.1.0", - "hyper 1.3.1", - "hyper-util", - "rustls 0.22.4", - "rustls-pki-types", - "tokio", - "tokio-rustls 0.25.0", - "tower-service", -] - -[[package]] -name = "hyper-rustls" -version = "0.27.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ee4be2c948921a1a5320b629c4193916ed787a7f7f293fd3f7f5a6c9de74155" -dependencies = [ - "futures-util", - "http 1.1.0", - "hyper 1.3.1", + "hyper 1.4.1", "hyper-util", "log", - "rustls 0.23.10", - "rustls-native-certs", + "rustls 0.23.13", + "rustls-native-certs 0.8.0", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", "tower-service", + "webpki-roots 0.26.6", ] [[package]] @@ -4949,7 +4954,7 @@ version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bbb958482e8c7be4bc3cf272a766a2b0bf1a6755e7a6ae777f017a31d11b13b1" dependencies = [ - "hyper 0.14.29", + "hyper 0.14.30", "pin-project-lite", "tokio", "tokio-io-timeout", @@ -4961,7 +4966,7 @@ version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3203a961e5c83b6f5498933e78b6b263e208c197b63e9c6c53cc82ffd3f63793" dependencies = [ - "hyper 1.3.1", + "hyper 1.4.1", "hyper-util", "pin-project-lite", "tokio", @@ -4970,16 +4975,16 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.5" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b875924a60b96e5d7b9ae7b066540b1dd1cbd90d1828f54c92e02a283351c56" +checksum = "da62f120a8a37763efb0cf8fdf264b884c7b8b9ac8660b900c8661030c00e6ba" dependencies = [ "bytes", "futures-channel", "futures-util", "http 1.1.0", - "http-body 1.0.0", - "hyper 1.3.1", + "http-body 1.0.1", + "hyper 1.4.1", "pin-project-lite", "socket2 0.5.7", "tokio", @@ -4990,9 +4995,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.60" +version = "0.1.61" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7ffbb5a1b541ea2561f8c41c087286cc091e21e556a4f09a8f6cbf17b69b141" +checksum = "235e081f3925a06703c2d0117ea8b91f042756fd6e7a6e5d901e8ca1a996b220" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -5011,124 +5016,6 @@ dependencies = [ "cc", ] -[[package]] -name = "icu_collections" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db2fa452206ebee18c4b5c2274dbf1de17008e874b4dc4f0aea9d01ca79e4526" -dependencies = [ - "displaydoc", - "yoke", - "zerofrom", - "zerovec", -] - -[[package]] -name = "icu_locid" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13acbb8371917fc971be86fc8057c41a64b521c184808a698c02acc242dbf637" -dependencies = [ - "displaydoc", - "litemap", - "tinystr", - "writeable", - "zerovec", -] - -[[package]] -name = "icu_locid_transform" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01d11ac35de8e40fdeda00d9e1e9d92525f3f9d887cdd7aa81d727596788b54e" -dependencies = [ - "displaydoc", - "icu_locid", - "icu_locid_transform_data", - "icu_provider", - "tinystr", - "zerovec", -] - -[[package]] -name = "icu_locid_transform_data" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdc8ff3388f852bede6b579ad4e978ab004f139284d7b28715f773507b946f6e" - -[[package]] -name = "icu_normalizer" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19ce3e0da2ec68599d193c93d088142efd7f9c5d6fc9b803774855747dc6a84f" -dependencies = [ - "displaydoc", - "icu_collections", - "icu_normalizer_data", - "icu_properties", - "icu_provider", - "smallvec", - "utf16_iter", - "utf8_iter", - "write16", - "zerovec", -] - -[[package]] -name = "icu_normalizer_data" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8cafbf7aa791e9b22bec55a167906f9e1215fd475cd22adfcf660e03e989516" - -[[package]] -name = "icu_properties" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f8ac670d7422d7f76b32e17a5db556510825b29ec9154f235977c9caba61036" -dependencies = [ - "displaydoc", - "icu_collections", - "icu_locid_transform", - "icu_properties_data", - "icu_provider", - "tinystr", - "zerovec", -] - -[[package]] -name = "icu_properties_data" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67a8effbc3dd3e4ba1afa8ad918d5684b8868b3b26500753effea8d2eed19569" - -[[package]] -name = "icu_provider" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ed421c8a8ef78d3e2dbc98a973be2f3770cb42b606e3ab18d6237c4dfde68d9" -dependencies = [ - "displaydoc", - "icu_locid", - "icu_provider_macros", - "stable_deref_trait", - "tinystr", - "writeable", - "yoke", - "zerofrom", - "zerovec", -] - -[[package]] -name = "icu_provider_macros" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", -] - [[package]] name = "ident_case" version = "1.0.1" @@ -5137,14 +5024,12 @@ checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" [[package]] name = "idna" -version = "1.0.0" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4716a3a0933a1d01c2f72450e89596eb51dd34ef3c211ccd875acdf1f8fe47ed" +checksum = "634d9b1461af396cad843f47fdba5597a4f9e6ddd4bfb6ff5d85028c25cb12f6" dependencies = [ - "icu_normalizer", - "icu_properties", - "smallvec", - "utf8_iter", + "unicode-bidi", + "unicode-normalization", ] [[package]] @@ -5155,18 +5040,18 @@ checksum = "cb56e1aa765b4b4f3aadfab769793b7087bb03a4ea4920644a6d238e2df5b9ed" [[package]] name = "include_dir" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18762faeff7122e89e0857b02f7ce6fcc0d101d5e9ad2ad7846cc01d61b7f19e" +checksum = "923d117408f1e49d914f1a379a309cffe4f18c05cf4e3d12e613a15fc81bd0dd" dependencies = [ "include_dir_macros", ] [[package]] name = "include_dir_macros" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b139284b5cf57ecfa712bcc66950bb635b31aff41c188e8a4cfc758eca374a3f" +checksum = "7cab85a7ed0bd5f0e76d93846e0147172bed2e2d3f859bcc33a8d9699cad1a75" dependencies = [ "proc-macro2", "quote", @@ -5218,9 +5103,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.2.6" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" +checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" dependencies = [ "equivalent", "hashbrown 0.14.5", @@ -5235,12 +5120,12 @@ checksum = "b248f5224d1d606005e02c97f5aa4e88eeb230488bcc03bc9ca4d7991399f2b5" [[package]] name = "inferno" -version = "0.11.19" +version = "0.11.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "321f0f839cd44a4686e9504b0a62b4d69a50b62072144c71c68f5873c167b8d9" +checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" dependencies = [ "ahash 0.8.11", - "indexmap 2.2.6", + "indexmap 2.5.0", "is-terminal", "itoa", "log", @@ -5312,9 +5197,9 @@ checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "integer-encoding" -version = "4.0.0" +version = "4.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "924df4f0e24e2e7f9cdd90babb0b96f93b20f3ecfa949ea9e6613756b8c8e1bf" +checksum = "0d762194228a2f1c11063e46e32e5acb96e66e906382b9eb5441f2e0504bbd5a" [[package]] name = "inventory" @@ -5324,15 +5209,15 @@ checksum = "f958d3d68f4167080a18141e10381e7634563984a537f2a49a30fd8e53ac5767" [[package]] name = "ipnet" -version = "2.9.0" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" +checksum = "187674a687eed5fe42285b40c6291f9a01517d415fad1c3cbc6a9f778af7fcd4" [[package]] name = "iri-string" -version = "0.7.2" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f5f6c2df22c009ac44f6f1499308e7a3ac7ba42cd2378475cc691510e1eef1b" +checksum = "9c25163201be6ded9e686703e85532f8f852ea1f92ba625cb3c51f7fe6d07a4a" dependencies = [ "memchr", "serde", @@ -5353,11 +5238,11 @@ dependencies = [ [[package]] name = "is-terminal" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" +checksum = "261f68e344040fbd0edea105bef17c66edf46f984ddb1115b775ce31be948f4b" dependencies = [ - "hermit-abi 0.3.9", + "hermit-abi 0.4.0", "libc", "windows-sys 0.52.0", ] @@ -5370,9 +5255,9 @@ checksum = "06d198e9919d9822d5f7083ba8530e04de87841eaf21ead9af8f2304efd57c89" [[package]] name = "is_terminal_polyfill" -version = "1.70.0" +version = "1.70.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8478577c03552c21db0e2724ffb8986a5ce7af88107e6be5d2ee6e158c12800" +checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" [[package]] name = "iso8601" @@ -5432,7 +5317,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c1e2b0210dc78b49337af9e49d7ae41a39dceac6e5985613f1cf7763e2f76a25" dependencies = [ "cedarwood", - "derive_builder 0.20.0", + "derive_builder 0.20.1", "fxhash", "lazy_static", "phf", @@ -5464,18 +5349,18 @@ checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" [[package]] name = "jobserver" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2b099aaa34a9751c5bf0878add70444e1ed2dd73f347be99003d4577277de6e" +checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" dependencies = [ "libc", ] [[package]] name = "js-sys" -version = "0.3.69" +version = "0.3.70" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" +checksum = "1868808506b929d7b0cfa8f75951347aa71bb21144b7791bae35d9bccfcfe37a" dependencies = [ "wasm-bindgen", ] @@ -5512,7 +5397,7 @@ dependencies = [ "fast-float", "itoa", "nom", - "ordered-float 4.2.0", + "ordered-float 4.2.2", "rand", "ryu", "serde_json", @@ -5587,7 +5472,7 @@ version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ee7893dab2e44ae5f9d0173f26ff4aa327c10b01b06a72b52dd9405b628640d" dependencies = [ - "indexmap 2.2.6", + "indexmap 2.5.0", ] [[package]] @@ -5612,9 +5497,9 @@ dependencies = [ [[package]] name = "kube" -version = "0.92.0" +version = "0.92.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12dc4487eda98835dcaa7ac92a14165446db29dbd67a743c79fe9f41bf38ee72" +checksum = "231c5a5392d9e2a9b0d923199760d3f1dd73b95288f2871d16c7c90ba4954506" dependencies = [ "k8s-openapi", "kube-client", @@ -5625,9 +5510,9 @@ dependencies = [ [[package]] name = "kube-client" -version = "0.92.0" +version = "0.92.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "408f35eab36927d3b883e4ad54c3080ea8c49f899ac84a7856e7182e4ee3b392" +checksum = "8f4bf54135062ff60e2a0dfb3e7a9c8e931fc4a535b4d6bd561e0a1371321c61" dependencies = [ "base64 0.22.1", "bytes", @@ -5636,19 +5521,19 @@ dependencies = [ "futures", "home", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "http-body-util", - "hyper 1.3.1", + "hyper 1.4.1", "hyper-http-proxy", - "hyper-rustls 0.27.2", + "hyper-rustls", "hyper-timeout 0.5.1", "hyper-util", "jsonpath-rust", "k8s-openapi", "kube-core", "pem", - "rustls 0.23.10", - "rustls-pemfile 2.1.2", + "rustls 0.23.13", + "rustls-pemfile 2.1.3", "secrecy", "serde", "serde_json", @@ -5663,9 +5548,9 @@ dependencies = [ [[package]] name = "kube-core" -version = "0.92.0" +version = "0.92.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f776624097c1e09e72eb1e9e0c2bb5d17d97c27a6a87734390a9fba246a8f67f" +checksum = "40fb9bd8141cbc0fe6b0d9112d371679b4cb607b45c31dd68d92e40864a12975" dependencies = [ "chrono", "form_urlencoded", @@ -5680,22 +5565,22 @@ dependencies = [ [[package]] name = "kube-derive" -version = "0.92.0" +version = "0.92.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ae07adfd7d21b7fa582789206391243f98e155b46c806eb494839569853bcfd" +checksum = "08fc86f70076921fdf2f433bbd2a796dc08ac537dc1db1f062cfa63ed4fa15fb" dependencies = [ - "darling 0.20.9", + "darling 0.20.10", "proc-macro2", "quote", "serde_json", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "kube-runtime" -version = "0.92.0" +version = "0.92.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12e5933f2d429f3a05d4cb67f935b25c94a133b0baeb558ab3917c270a11f6ef" +checksum = "b7eb2fb986f81770eb55ec7f857e197019b31b38768d2410f6c1046ffac34225" dependencies = [ "ahash 0.8.11", "async-broadcast", @@ -5780,16 +5665,16 @@ dependencies = [ "proc-macro2", "quote", "regex", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "lazy_static" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" dependencies = [ - "spin 0.5.2", + "spin 0.9.8", ] [[package]] @@ -5870,9 +5755,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.155" +version = "0.2.158" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97b3888a4aecf77e811145cadf6eef5901f4782c53886191b2f693f24761847c" +checksum = "d8adc4bb1803a324070e64a98ae98f38934d91957a99cfb3a43dcbc01bc56439" [[package]] name = "libfuzzer-sys" @@ -5909,12 +5794,12 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.3" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c2a198fb6b0eada2a8df47933734e6d35d350665a33a3593d7164fa52c75c19" +checksum = "4979f22fdb869068da03c9f7528f8297c6fd2606bc3a4affe42e6a823fdb8da4" dependencies = [ "cfg-if", - "windows-targets 0.52.5", + "windows-targets 0.52.6", ] [[package]] @@ -5929,8 +5814,9 @@ version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "libc", + "redox_syscall 0.5.4", ] [[package]] @@ -5946,9 +5832,9 @@ dependencies = [ [[package]] name = "libz-sys" -version = "1.1.18" +version = "1.1.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c15da26e5af7e25c90b37a2d75cdbf940cf4a55316de9d84c679c9b8bfabf82e" +checksum = "d2d16453e800a8cf6dd2fc3eb4bc99b786a9b90c663b8559a5b1a041bf89e472" dependencies = [ "cc", "libc", @@ -5968,12 +5854,6 @@ version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" -[[package]] -name = "litemap" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "643cb0b8d4fcc284004d5fd0d67ccf61dfffadb7f75e1e71bc420f4688a3a704" - [[package]] name = "lock_api" version = "0.4.12" @@ -5986,9 +5866,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.21" +version = "0.4.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" [[package]] name = "log-store" @@ -6058,7 +5938,7 @@ dependencies = [ "cactus", "cfgrammar", "filetime", - "indexmap 2.2.6", + "indexmap 2.5.0", "lazy_static", "lrtable", "num-traits", @@ -6086,28 +5966,27 @@ dependencies = [ [[package]] name = "lru" -version = "0.12.3" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3262e75e648fce39813cb56ac41f3c3e3f65217ebf3844d818d1f9398cfb0dc" +checksum = "37ee39891760e7d94734f6f63fedc29a2e4a152f836120753a72503f09fcf904" dependencies = [ "hashbrown 0.14.5", ] [[package]] name = "lz4" -version = "1.25.0" +version = "1.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d6eab492fe7f8651add23237ea56dbf11b3c4ff762ab83d40a47f11433421f91" +checksum = "a231296ca742e418c43660cb68e082486ff2538e8db432bc818580f3965025ed" dependencies = [ - "libc", "lz4-sys", ] [[package]] name = "lz4-sys" -version = "1.9.5" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9764018d143cc854c9f17f0b907de70f14393b1f502da6375dce70f00514eb3" +checksum = "fcb44a01837a858d47e5a630d2ccf304c8efcc4b83b8f9f75b7a9ee4fcc6e57d" dependencies = [ "cc", "libc", @@ -6206,9 +6085,9 @@ checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" [[package]] name = "matrixmultiply" -version = "0.3.8" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7574c1cf36da4798ab73da5b215bbf444f50718207754cb522201d78d1cd0ff2" +checksum = "9380b911e3e96d10c1f415da0876389aaf1b56759054eeb0de7df940c456ba1a" dependencies = [ "autocfg", "rawpointer", @@ -6268,9 +6147,9 @@ dependencies = [ [[package]] name = "memmap2" -version = "0.9.4" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe751422e4a8caa417e13c3ea66452215d7d63e19e604f4980461212f3ae1322" +checksum = "fd3f7eed9d3848f8b98834af67102b720745c4ec028fcd0aa0239277e7de374f" dependencies = [ "libc", ] @@ -6335,7 +6214,7 @@ dependencies = [ "api", "async-trait", "chrono", - "clap 4.5.7", + "clap 4.5.18", "client", "common-base", "common-catalog", @@ -6379,7 +6258,7 @@ dependencies = [ "tokio", "tokio-postgres", "tokio-stream", - "toml 0.8.14", + "toml 0.8.19", "tonic 0.11.0", "tower", "tracing", @@ -6443,9 +6322,9 @@ checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" [[package]] name = "mime_guess" -version = "2.0.4" +version = "2.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4192263c238a5f0d0c6bfd21f336a313a4ce1c450542449ca191bb657b4642ef" +checksum = "f7c44f8e672c00fe5308fa235f821cb4198414e1c77935c1ab6948d3fd78550e" dependencies = [ "mime", "unicase", @@ -6468,11 +6347,11 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.7.3" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87dfd01fe195c66b572b37921ad8803d010623c0aca821bea2302239d155cdae" +checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" dependencies = [ - "adler", + "adler2", ] [[package]] @@ -6498,10 +6377,16 @@ dependencies = [ ] [[package]] -name = "mirai-annotations" -version = "1.12.0" +name = "mio" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9be0862c1b3f26a88803c4a49de6889c10e608b3ee9344e6ef5b45fb37ad3d1" +checksum = "80e04d1dcff3aae0704555fe5fee3bcfaf3d1fdf8a7e521d5b9d2b42acb52cec" +dependencies = [ + "hermit-abi 0.3.9", + "libc", + "wasi", + "windows-sys 0.52.0", +] [[package]] name = "mito2" @@ -6569,7 +6454,7 @@ dependencies = [ "tokio", "tokio-stream", "tokio-util", - "toml 0.8.14", + "toml 0.8.19", "uuid", ] @@ -6602,9 +6487,9 @@ dependencies = [ [[package]] name = "moka" -version = "0.12.7" +version = "0.12.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e0d88686dc561d743b40de8269b26eaf0dc58781bde087b0984646602021d08" +checksum = "32cf62eb4dd975d2dde76432fb1075c49e3ee2331cf36f1f8fd4b66550d32b6f" dependencies = [ "async-lock", "async-trait", @@ -6675,14 +6560,14 @@ version = "0.30.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "56b0d8a0db9bf6d2213e11f2c701cb91387b0614361625ab7b9743b41aa4938f" dependencies = [ - "darling 0.20.9", + "darling 0.20.10", "heck 0.4.1", "num-bigint", "proc-macro-crate 1.3.1", "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", "termcolor", "thiserror", ] @@ -6693,14 +6578,14 @@ version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "afe0450cc9344afff34915f8328600ab5ae19260802a334d0f72d2d5bdda3bfe" dependencies = [ - "darling 0.20.9", + "darling 0.20.10", "heck 0.4.1", "num-bigint", - "proc-macro-crate 3.1.0", + "proc-macro-crate 3.2.0", "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", "termcolor", "thiserror", ] @@ -6720,7 +6605,7 @@ dependencies = [ "keyed_priority_queue", "lazy_static", "lru", - "mio", + "mio 0.8.11", "mysql_common 0.31.0", "once_cell", "pem", @@ -6750,8 +6635,8 @@ checksum = "06f19e4cfa0ab5a76b627cec2d81331c49b034988eaf302c3bafeada684eadef" dependencies = [ "base64 0.21.7", "bigdecimal", - "bindgen", - "bitflags 2.5.0", + "bindgen 0.70.1", + "bitflags 2.6.0", "bitvec", "btoi", "byteorder", @@ -6789,8 +6674,8 @@ checksum = "478b0ff3f7d67b79da2b96f56f334431aef65e15ba4b29dd74a4236e29582bdc" dependencies = [ "base64 0.21.7", "bigdecimal", - "bindgen", - "bitflags 2.5.0", + "bindgen 0.70.1", + "bitflags 2.6.0", "bitvec", "btoi", "byteorder", @@ -6818,7 +6703,7 @@ dependencies = [ "thiserror", "time", "uuid", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -6904,7 +6789,7 @@ version = "0.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab2156c4fce2f8df6c499cc1c763e4394b7482525bf2a9701c9d79d215f519e4" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "cfg-if", "cfg_aliases 0.1.1", "libc", @@ -6933,7 +6818,7 @@ version = "6.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6205bd8bb1e454ad2e27422015fb5e4f2bcc7e08fa8f27058670d208324a4d2d" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "crossbeam-channel", "filetime", "fsevent-sys", @@ -6941,7 +6826,7 @@ dependencies = [ "kqueue", "libc", "log", - "mio", + "mio 0.8.11", "walkdir", "windows-sys 0.48.0", ] @@ -6981,9 +6866,9 @@ dependencies = [ [[package]] name = "num-bigint" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c165a9ab64cf766f73521c0dd2cfdff64f488b8f0b3e621face3462d3db536d7" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ "num-integer", "num-traits", @@ -7029,7 +6914,7 @@ checksum = "ed3955f1a9c7c0c15e092f9c887db08b1fc683305fdf6eb6684f22555355e202" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -7134,9 +7019,9 @@ dependencies = [ [[package]] name = "object" -version = "0.36.0" +version = "0.36.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "576dfe1fc8f9df304abb159d767a29d0476f7750fbf8aa7ad07816004a207434" +checksum = "084f1a5821ac4c651660a94a7153d27ac9d8a53736203f58b31945ded098070a" dependencies = [ "memchr", ] @@ -7203,15 +7088,15 @@ checksum = "e296cf87e61c9cfc1a61c3c63a0f7f286ed4554e0e22be84e8a38e1d264a2a29" [[package]] name = "oorandom" -version = "11.1.3" +version = "11.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" +checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9" [[package]] name = "opendal" -version = "0.49.1" +version = "0.49.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba615070686c8781ce97376fdafca29d7c42f47b31d2230d7c8c1642ec823950" +checksum = "9b04d09b9822c2f75a1d2fc513a2c1279c70e91e7407936fffdf6a6976ec530a" dependencies = [ "anyhow", "async-trait", @@ -7228,7 +7113,7 @@ dependencies = [ "md-5", "once_cell", "percent-encoding", - "quick-xml 0.36.1", + "quick-xml 0.36.2", "reqsign", "reqwest", "serde", @@ -7280,7 +7165,7 @@ checksum = "1e32339a5dc40459130b3bd269e9892439f55b33e772d2a9d402a789baaf4e8a" dependencies = [ "futures-core", "futures-sink", - "indexmap 2.2.6", + "indexmap 2.5.0", "js-sys", "once_cell", "pin-project-lite", @@ -7371,7 +7256,7 @@ dependencies = [ "glob", "once_cell", "opentelemetry 0.21.0", - "ordered-float 4.2.0", + "ordered-float 4.2.2", "percent-encoding", "rand", "thiserror", @@ -7393,7 +7278,7 @@ dependencies = [ "glob", "once_cell", "opentelemetry 0.22.0", - "ordered-float 4.2.0", + "ordered-float 4.2.2", "percent-encoding", "rand", "thiserror", @@ -7505,9 +7390,9 @@ dependencies = [ [[package]] name = "ordered-float" -version = "4.2.0" +version = "4.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a76df7075c7d4d01fdcb46c912dd17fba5b60c78ea480b475f2b6ab6f666584e" +checksum = "4a91171844676f8c7990ce64959210cd2eaef32c2612c50f9fae9f8aaa6065a6" dependencies = [ "num-traits", ] @@ -7595,9 +7480,9 @@ dependencies = [ [[package]] name = "parking" -version = "2.2.0" +version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae" +checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" @@ -7644,10 +7529,10 @@ dependencies = [ "cfg-if", "libc", "petgraph", - "redox_syscall 0.5.2", + "redox_syscall 0.5.4", "smallvec", "thread-id", - "windows-targets 0.52.5", + "windows-targets 0.52.6", ] [[package]] @@ -7682,7 +7567,7 @@ dependencies = [ "thrift", "tokio", "twox-hash", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -7824,9 +7709,9 @@ checksum = "df202b0b0f5b8e389955afd5f27b007b00fb948162953f1db9c70d2c7e3157d7" [[package]] name = "pest" -version = "2.7.10" +version = "2.7.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "560131c633294438da9f7c4b08189194b20946c8274c6b9e38881a7874dc8ee8" +checksum = "fdbef9d1d47087a895abd220ed25eb4ad973a5e26f6a4367b038c25e28dfc2d9" dependencies = [ "memchr", "thiserror", @@ -7835,9 +7720,9 @@ dependencies = [ [[package]] name = "pest_derive" -version = "2.7.10" +version = "2.7.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26293c9193fbca7b1a3bf9b79dc1e388e927e6cacaa78b4a3ab705a1d3d41459" +checksum = "4d3a6e3394ec80feb3b6393c725571754c6188490265c61aaf260810d6b95aa0" dependencies = [ "pest", "pest_generator", @@ -7845,22 +7730,22 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.7.10" +version = "2.7.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ec22af7d3fb470a85dd2ca96b7c577a1eb4ef6f1683a9fe9a8c16e136c04687" +checksum = "94429506bde1ca69d1b5601962c73f4172ab4726571a59ea95931218cb0e930e" dependencies = [ "pest", "pest_meta", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "pest_meta" -version = "2.7.10" +version = "2.7.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7a240022f37c361ec1878d646fc5b7d7c4d28d5946e1a80ad5a7a4f4ca0bdcd" +checksum = "ac8a071862e93690b6e34e9a5fb8e33ff3734473ac0245b27232222c4906a33f" dependencies = [ "once_cell", "pest", @@ -7874,7 +7759,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset", - "indexmap 2.2.6", + "indexmap 2.5.0", ] [[package]] @@ -7965,7 +7850,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -8038,9 +7923,9 @@ dependencies = [ [[package]] name = "piper" -version = "0.2.3" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae1d5c74c9876f070d3e8fd503d748c7d974c3e48da8f41350fa5222ef9b4391" +checksum = "96c8c490f422ef9a4efd2cb5b42b76c8613d7e7dfc1caf667b8a3350a5acc066" dependencies = [ "atomic-waker", "fastrand", @@ -8115,9 +8000,9 @@ checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" [[package]] name = "plotters" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a15b6eccb8484002195a3e44fe65a4ce8e93a625797a063735536fd59cb01cf3" +checksum = "5aeb6f403d7a4911efb1e33402027fc44f29b5bf6def3effcc22d7bb75f2b747" dependencies = [ "num-traits", "plotters-backend", @@ -8128,15 +8013,15 @@ dependencies = [ [[package]] name = "plotters-backend" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "414cec62c6634ae900ea1c56128dfe87cf63e7caece0852ec76aba307cebadb7" +checksum = "df42e13c12958a16b3f7f4386b9ab1f3e7933914ecea48da7139435263a4172a" [[package]] name = "plotters-svg" -version = "0.3.6" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81b30686a7d9c3e010b84284bdd26a29f2138574f52f5eb6f794fc0ad924e705" +checksum = "51bae2ac328883f7acdfea3d66a7c35751187f870bc81f94563733a154d7a670" dependencies = [ "plotters-backend", ] @@ -8166,9 +8051,9 @@ dependencies = [ [[package]] name = "portable-atomic" -version = "1.6.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7170ef9988bc169ba16dd36a7fa041e5c4cbeb6a35b76d4c03daded371eae7c0" +checksum = "d30538d42559de6b034bc76fd6dd4c38961b1ee5c6c56e3808c50128fdbc22ce" [[package]] name = "postgres-protocol" @@ -8190,9 +8075,9 @@ dependencies = [ [[package]] name = "postgres-types" -version = "0.2.7" +version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02048d9e032fb3cc3413bbf7b83a15d84a5d419778e2628751896d856498eee9" +checksum = "f66ea23a2d0e5734297357705193335e0a957696f34bed2f2faefacb2fec336f" dependencies = [ "array-init", "bytes", @@ -8238,9 +8123,12 @@ dependencies = [ [[package]] name = "ppv-lite86" -version = "0.2.17" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +dependencies = [ + "zerocopy", +] [[package]] name = "precomputed-hash" @@ -8264,15 +8152,15 @@ dependencies = [ [[package]] name = "predicates-core" -version = "1.0.6" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b794032607612e7abeb4db69adb4e33590fa6cf1149e95fd7cb00e634b92f174" +checksum = "ae8177bee8e75d6846599c6b9ff679ed51e882816914eec639944d7c9aa11931" [[package]] name = "predicates-tree" -version = "1.0.9" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "368ba315fb8c5052ab692e68a0eefec6ec57b23a36959c14496f0b0df2c0cecf" +checksum = "41b740d195ed3166cd147c8047ec98db0e22ec019eb8eeb76d343b795304fb13" dependencies = [ "predicates-core", "termtree", @@ -8280,9 +8168,9 @@ dependencies = [ [[package]] name = "pretty_assertions" -version = "1.4.0" +version = "1.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af7cee1a6c8a5b9208b3cb1061f10c0cb689087b3d8ce85fb9d2dd7a29b6ba66" +checksum = "3ae130e2f271fbc2ac3a40fb1d07180839cdbbe443c7a27e1e3c13c5cac0116d" dependencies = [ "diff", "yansi", @@ -8300,12 +8188,12 @@ dependencies = [ [[package]] name = "prettyplease" -version = "0.2.20" +version = "0.2.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f12335488a2f3b0a83b14edad48dca9879ce89b2edd10e80237e4e852dd645e" +checksum = "479cf940fbbb3426c32c5d5176f62ad57549a0bb84773423ba8be9d089f5faba" dependencies = [ "proc-macro2", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -8320,11 +8208,11 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "3.1.0" +version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d37c51ca738a55da99dc0c4a34860fd675453b8b36209178c2249bb13651284" +checksum = "8ecf48c7ca261d60b74ab1a7b20da18bede46776b2e55535cb958eb595c5fa7b" dependencies = [ - "toml_edit 0.21.1", + "toml_edit 0.22.21", ] [[package]] @@ -8353,9 +8241,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.85" +version = "1.0.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22244ce15aa966053a896d1accb3a6e68469b97c7f33f284b99f0d576879fc23" +checksum = "5e719e8df665df0d1c8fbfd238015744736151d4445ec0836b8e628aae103b77" dependencies = [ "unicode-ident", ] @@ -8366,7 +8254,7 @@ version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "731e0d9356b0c25f16f33b5be79b1c57b562f141ebfcdb0ad8ac2c13a24293b4" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "hex", "lazy_static", "procfs-core", @@ -8379,7 +8267,7 @@ version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d3554923a69f4ce04c4a754260c338f505ce22642d3830e049a399fc2059a29" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "hex", ] @@ -8473,12 +8361,12 @@ dependencies = [ [[package]] name = "prost" -version = "0.13.1" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13db3d3fde688c61e2446b4d843bc27a7e8af269a69440c0308021dc92333cc" +checksum = "7b0487d90e047de87f984913713b85c601c05609aad5b0df4b4573fbf69aa13f" dependencies = [ "bytes", - "prost-derive 0.13.1", + "prost-derive 0.13.3", ] [[package]] @@ -8498,15 +8386,15 @@ dependencies = [ "prost 0.12.6", "prost-types 0.12.6", "regex", - "syn 2.0.66", + "syn 2.0.77", "tempfile", ] [[package]] name = "prost-build" -version = "0.13.1" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5bb182580f71dd070f88d01ce3de9f4da5021db7115d2e1c3605a754153b77c1" +checksum = "0c1318b19085f08681016926435853bbf7858f9c082d0999b80550ff5d9abe15" dependencies = [ "bytes", "heck 0.5.0", @@ -8515,8 +8403,8 @@ dependencies = [ "multimap", "once_cell", "petgraph", - "prost 0.13.1", - "prost-types 0.13.1", + "prost 0.13.3", + "prost-types 0.13.3", "regex", "tempfile", ] @@ -8544,20 +8432,20 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "prost-derive" -version = "0.13.1" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18bec9b0adc4eba778b33684b7ba3e7137789434769ee3ce3930463ef904cfca" +checksum = "e9552f850d5f0964a4e4d0bf306459ac29323ddfbae05e35a7c0d35cb0803cc5" dependencies = [ "anyhow", "itertools 0.13.0", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -8580,11 +8468,11 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.13.1" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cee5168b05f49d4b0ca581206eb14a7b22fafd963efe729ac48eb03266e25cc2" +checksum = "4759aa0d3a6232fb8dbdb97b61de2c20047c68aca932c7ed76da9d788508d670" dependencies = [ - "prost 0.13.1", + "prost 0.13.3", ] [[package]] @@ -8651,12 +8539,12 @@ dependencies = [ name = "puffin" version = "0.9.3" dependencies = [ - "async-compression 0.4.11", + "async-compression 0.4.12", "async-trait", "async-walkdir", "auto_impl", "base64 0.21.7", - "bitflags 2.5.0", + "bitflags 2.6.0", "common-error", "common-macro", "common-runtime", @@ -8738,7 +8626,7 @@ dependencies = [ "proc-macro2", "pyo3-macros-backend", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -8751,7 +8639,7 @@ dependencies = [ "proc-macro2", "pyo3-build-config", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -8861,19 +8749,67 @@ dependencies = [ [[package]] name = "quick-xml" -version = "0.36.1" +version = "0.36.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96a05e2e8efddfa51a84ca47cec303fac86c8541b686d37cac5efc0e094417bc" +checksum = "f7649a7b4df05aed9ea7ec6f628c67c9953a43869b8bc50929569b2999d443fe" dependencies = [ "memchr", "serde", ] [[package]] -name = "quote" -version = "1.0.36" +name = "quinn" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" +checksum = "8c7c5fdde3cdae7203427dc4f0a68fe0ed09833edc525a03456b153b79828684" +dependencies = [ + "bytes", + "pin-project-lite", + "quinn-proto", + "quinn-udp", + "rustc-hash 2.0.0", + "rustls 0.23.13", + "socket2 0.5.7", + "thiserror", + "tokio", + "tracing", +] + +[[package]] +name = "quinn-proto" +version = "0.11.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fadfaed2cd7f389d0161bb73eeb07b7b78f8691047a6f3e73caaeae55310a4a6" +dependencies = [ + "bytes", + "rand", + "ring 0.17.8", + "rustc-hash 2.0.0", + "rustls 0.23.13", + "slab", + "thiserror", + "tinyvec", + "tracing", +] + +[[package]] +name = "quinn-udp" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fe68c2e9e1a1234e218683dbdf9f9dfcb094113c5ac2b938dfcb9bab4c4140b" +dependencies = [ + "libc", + "once_cell", + "socket2 0.5.7", + "tracing", + "windows-sys 0.59.0", +] + +[[package]] +name = "quote" +version = "1.0.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" dependencies = [ "proc-macro2", ] @@ -8971,11 +8907,11 @@ dependencies = [ [[package]] name = "raw-cpuid" -version = "11.0.2" +version = "11.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e29830cbb1290e404f24c73af91c5d8d631ce7e128691e9477556b540cd01ecd" +checksum = "cb9ee317cfe3fbd54b36a511efc1edd42e216903c9cd575e686dd68a2ba90d8d" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", ] [[package]] @@ -9021,27 +8957,18 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.4.1" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +checksum = "0884ad60e090bf1345b93da0a5de8923c93884cd03f40dfcfddd3b4bee661853" dependencies = [ - "bitflags 1.3.2", -] - -[[package]] -name = "redox_syscall" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c82cf8cff14456045f55ec4241383baeff27af886adb72ffb2162f99911de0fd" -dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", ] [[package]] name = "redox_users" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd283d9651eeda4b2a83a43c1c91b266c40fd76ecd39a50a8c630ae69dc72891" +checksum = "ba009ff324d1fc1b900bd1fdb31564febe58a8ccc8a6fdbb93b543d33b13ca43" dependencies = [ "getrandom", "libredox", @@ -9065,14 +8992,14 @@ checksum = "bcc303e793d3734489387d205e9b186fac9c6cfacedd98cbb2e8a5943595f3e6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "regex" -version = "1.10.5" +version = "1.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b91213439dad192326a0d7c6ee3955910425f441d7038e0d6933b0aec5c4517f" +checksum = "4219d74c6b67a3654a9fbebc4b419e22126d13d2f3c4a07ee0cb61ff79a79619" dependencies = [ "aho-corasick", "memchr", @@ -9167,7 +9094,7 @@ dependencies = [ "rand", "reqwest", "rsa 0.9.6", - "rust-ini 0.21.0", + "rust-ini 0.21.1", "serde", "serde_json", "sha1", @@ -9176,19 +9103,19 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.4" +version = "0.12.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "566cafdd92868e0939d3fb961bd0dc25fcfaaed179291093b3d43e6b3150ea10" +checksum = "f8f4955649ef5c38cc7f9e8aa41761d48fb9677197daea9984dc54f56aad5e63" dependencies = [ "base64 0.22.1", "bytes", "futures-core", "futures-util", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "http-body-util", - "hyper 1.3.1", - "hyper-rustls 0.26.0", + "hyper 1.4.1", + "hyper-rustls", "hyper-util", "ipnet", "js-sys", @@ -9198,16 +9125,17 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.22.4", - "rustls-native-certs", - "rustls-pemfile 2.1.2", + "quinn", + "rustls 0.23.13", + "rustls-native-certs 0.7.3", + "rustls-pemfile 2.1.3", "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", - "sync_wrapper", + "sync_wrapper 1.0.1", "tokio", - "tokio-rustls 0.25.0", + "tokio-rustls 0.26.0", "tokio-util", "tower-service", "url", @@ -9215,8 +9143,8 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams", "web-sys", - "webpki-roots 0.26.2", - "winreg 0.52.0", + "webpki-roots 0.26.6", + "windows-registry", ] [[package]] @@ -9256,9 +9184,9 @@ dependencies = [ [[package]] name = "rgb" -version = "0.8.37" +version = "0.8.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05aaa8004b64fd573fc9d002f4e632d51ad4f026c2b5ba95fcb6c2f32c2c47d8" +checksum = "57397d16646700483b67d2dd6511d79318f9d057fdbd21a4066aeac8b41d310a" dependencies = [ "bytemuck", ] @@ -9295,9 +9223,9 @@ dependencies = [ [[package]] name = "rkyv" -version = "0.7.44" +version = "0.7.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cba464629b3394fc4dbc6f940ff8f5b4ff5c7aef40f29166fd4ad12acbc99c0" +checksum = "9008cd6385b9e161d8229e1f6549dd23c3d022f132a2ea37ac3a10ac4935779b" dependencies = [ "bitvec", "bytecheck", @@ -9313,9 +9241,9 @@ dependencies = [ [[package]] name = "rkyv_derive" -version = "0.7.44" +version = "0.7.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7dddfff8de25e6f62b9d64e6e432bf1c6736c57d20323e15ee10435fbda7c65" +checksum = "503d1d27590a2b0a3a4ca4c94755aa2875657196ecbf401a42eff41d7de532c0" dependencies = [ "proc-macro2", "quote", @@ -9402,18 +9330,18 @@ dependencies = [ "crc32c", "flate2", "futures", - "integer-encoding 4.0.0", + "integer-encoding 4.0.2", "lz4", "parking_lot 0.12.3", "rand", "rsasl", - "rustls 0.23.10", + "rustls 0.23.13", "snap", "thiserror", "tokio", "tokio-rustls 0.26.0", "tracing", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -9436,13 +9364,13 @@ checksum = "4165dfae59a39dd41d8dec720d3cbfbc71f69744efb480a3920f5d4e0cc6798d" dependencies = [ "cfg-if", "glob", - "proc-macro-crate 3.1.0", + "proc-macro-crate 3.2.0", "proc-macro2", "quote", "regex", "relative-path", "rustc_version", - "syn 2.0.66", + "syn 2.0.77", "unicode-ident", ] @@ -9454,7 +9382,7 @@ checksum = "b3a8fb4672e840a587a66fc577a5491375df51ddb88f2a2c2a792598c326fe14" dependencies = [ "quote", "rand", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -9477,7 +9405,7 @@ dependencies = [ "proc-macro2", "quote", "rust-embed-utils", - "syn 2.0.66", + "syn 2.0.77", "walkdir", ] @@ -9503,9 +9431,9 @@ dependencies = [ [[package]] name = "rust-ini" -version = "0.21.0" +version = "0.21.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d625ed57d8f49af6cfa514c42e1a71fadcff60eb0b1c517ff82fe41aa025b41" +checksum = "4e310ef0e1b6eeb79169a1171daf9abcb87a2e17c03bee2c4bb100b55c75409f" dependencies = [ "cfg-if", "ordered-multimap 0.7.3", @@ -9573,9 +9501,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.35.0" +version = "1.36.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1790d1c4c0ca81211399e0e0af16333276f375209e71a37b67698a373db5b47a" +checksum = "b082d80e3e3cc52b2ed634388d436fe1f4de6af5786cc2de9ba9737527bdf555" dependencies = [ "arrayvec", "borsh", @@ -9608,20 +9536,20 @@ checksum = "583034fd73374156e66797ed8e5b0d5690409c9226b22d87cb7f19821c05d152" [[package]] name = "rustc_version" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" dependencies = [ "semver", ] [[package]] name = "rustix" -version = "0.38.34" +version = "0.38.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" +checksum = "8acb788b847c24f28525660c4d7758620a7210875711f79e7f663cc152726811" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "errno", "libc", "linux-raw-sys", @@ -9661,35 +9589,46 @@ dependencies = [ "log", "ring 0.17.8", "rustls-pki-types", - "rustls-webpki 0.102.4", + "rustls-webpki 0.102.8", "subtle", "zeroize", ] [[package]] name = "rustls" -version = "0.23.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05cff451f60db80f490f3c182b77c35260baace73209e9cdbbe526bfe3a4d402" +version = "0.23.13" +source = "git+https://github.com/GreptimeTeam/rustls#e0917109155435821c4416d699babe749b597a03" dependencies = [ - "aws-lc-rs", "log", "once_cell", "ring 0.17.8", "rustls-pki-types", - "rustls-webpki 0.102.4", + "rustls-webpki 0.102.8", "subtle", "zeroize", ] [[package]] name = "rustls-native-certs" -version = "0.7.1" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a88d6d420651b496bdd98684116959239430022a115c1240e6c3993be0b15fba" +checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" dependencies = [ "openssl-probe", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", + "rustls-pki-types", + "schannel", + "security-framework", +] + +[[package]] +name = "rustls-native-certs" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcaf18a4f2be7326cd874a5fa579fae794320a0f388d365dca7e480e55f83f8a" +dependencies = [ + "openssl-probe", + "rustls-pemfile 2.1.3", "rustls-pki-types", "schannel", "security-framework", @@ -9706,9 +9645,9 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "2.1.2" +version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29993a25686778eb88d4189742cd713c9bce943bc54251a33509dc63cbacf73d" +checksum = "196fe16b00e106300d3e45ecfcb764fa292a535d7326a29a5875c579c7417425" dependencies = [ "base64 0.22.1", "rustls-pki-types", @@ -9716,9 +9655,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "976295e77ce332211c0d24d92c0e83e50f5c5f046d11082cea19f3df13a3562d" +checksum = "fc0a2ce646f8655401bb81e7927b812614bd5d91dbc968696be50603510fcaf0" [[package]] name = "rustls-webpki" @@ -9732,11 +9671,10 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.102.4" +version = "0.102.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff448f7e92e913c4b7d4c6d8e4540a1724b319b4152b8aef6d4cf8339712b33e" +checksum = "64ca1bc8749bd4cf37b5ce386cc146580777b4e8572c7b97baf22c83f444bee9" dependencies = [ - "aws-lc-rs", "ring 0.17.8", "rustls-pki-types", "untrusted 0.9.0", @@ -10022,7 +9960,7 @@ dependencies = [ "widestring", "winapi", "windows 0.39.0", - "winreg 0.10.1", + "winreg", ] [[package]] @@ -10165,11 +10103,11 @@ checksum = "ece8e78b2f38ec51c51f5d475df0a7187ba5111b2a28bdc761ee05b075d40a71" [[package]] name = "schannel" -version = "0.1.23" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbc91545643bcf3a0bbb6569265615222618bdf33ce4ffbbd13c4bbd4c093534" +checksum = "e9aaafd5a2b6e3d657ff009d82fbd630b6bd54dd4eb06f21693925cdf80f9b8b" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -10194,7 +10132,7 @@ dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -10295,7 +10233,7 @@ dependencies = [ "heck 0.4.1", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -10310,11 +10248,11 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.11.0" +version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c627723fd09706bacdb5cf41499e95098555af3c3c29d014dc3c458ef6be11c0" +checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "core-foundation", "core-foundation-sys", "libc", @@ -10323,9 +10261,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.11.0" +version = "2.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "317936bbbd05227752583946b9e66d7ce3b489f84e11a94a510b4437fef407d7" +checksum = "ea4a292869320c0272d7bc55a5a6aafaff59b4f63404a003887b679a2e05b4b6" dependencies = [ "core-foundation-sys", "libc", @@ -10348,9 +10286,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.210" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "c8e3592472072e6e22e0a54d5904d9febf8508f65fb8552499a1abc7d1078c3a" dependencies = [ "serde_derive", ] @@ -10367,13 +10305,13 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.210" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "243902eda00fad750862fc144cea25caca5e20d615af0a81bee94ca738f1df1f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -10384,16 +10322,16 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "serde_json" -version = "1.0.122" +version = "1.0.128" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "784b6203951c57ff748476b126ccb5e8e2959a5c19e5c617ab1956be3dbc68da" +checksum = "6ff5456707a1de34e7e37f2a6fd3d3f808c318259cbd01ab6377795054b483d8" dependencies = [ - "indexmap 2.2.6", + "indexmap 2.5.0", "itoa", "memchr", "ryu", @@ -10418,28 +10356,28 @@ checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "serde_spanned" -version = "0.6.6" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79e674e01f999af37c49f70a6ede167a8a60b2503e56c5599532a65baa5969a0" +checksum = "eb5b1b31579f3811bf615c144393417496f152e12ac8b7663bf664f4a815306d" dependencies = [ "serde", ] [[package]] name = "serde_tokenstream" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8790a7c3fe883e443eaa2af6f705952bc5d6e8671a220b9335c8cae92c037e74" +checksum = "64060d864397305347a78851c51588fd283767e7e7589829e8121d65512340f1" dependencies = [ "proc-macro2", "quote", "serde", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -10456,15 +10394,15 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.8.1" +version = "3.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ad483d2ab0149d5a5ebcd9972a3852711e0153d863bf5a5d0391d28883c4a20" +checksum = "69cecfa94848272156ea67b2b1a53f20fc7bc638c4a46d2f8abde08f05f4b857" dependencies = [ "base64 0.22.1", "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.2.6", + "indexmap 2.5.0", "serde", "serde_derive", "serde_json", @@ -10474,14 +10412,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.8.1" +version = "3.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65569b702f41443e8bc8bbb1c5779bd0450bbe723b56198980e80ec45780bce2" +checksum = "a8fee4991ef4f274617a51ad4af30519438dacb2f56ac773b08a1922ff743350" dependencies = [ - "darling 0.20.9", + "darling 0.20.10", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -10490,7 +10428,7 @@ version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 2.2.6", + "indexmap 2.5.0", "itoa", "ryu", "serde", @@ -10547,7 +10485,7 @@ dependencies = [ "http 0.2.12", "http-body 0.4.6", "humantime-serde", - "hyper 0.14.29", + "hyper 0.14.30", "influxdb_line_protocol", "itertools 0.10.5", "jsonb", @@ -10575,8 +10513,8 @@ dependencies = [ "regex", "reqwest", "rust-embed", - "rustls 0.23.10", - "rustls-pemfile 2.1.2", + "rustls 0.23.13", + "rustls-pemfile 2.1.3", "rustls-pki-types", "schemars", "script", @@ -10603,7 +10541,7 @@ dependencies = [ "tower-http 0.4.4", "urlencoding", "uuid", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -10669,9 +10607,9 @@ dependencies = [ [[package]] name = "shadow-rs" -version = "0.31.0" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66caf2de9b7e61293c00006cd2807d6c4e4b31018c5ea21d008f44f4852b93c3" +checksum = "02c282402d25101f9c893e9cd7e4cae535fe7db18b81291de973026c219ddf1e" dependencies = [ "const_format", "git2", @@ -10729,9 +10667,9 @@ dependencies = [ [[package]] name = "simdutf8" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" +checksum = "e3a9fe34e3e7a50316060351f37187a3f546bce95496156754b601a5fa71b76e" [[package]] name = "similar" @@ -10854,7 +10792,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -10962,9 +10900,9 @@ dependencies = [ [[package]] name = "sqlformat" -version = "0.2.4" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f895e3734318cc55f1fe66258926c9b910c124d47520339efecbb6c59cec7c1f" +checksum = "7bba3a93db0cc4f7bdece8bb09e77e2e785c20bfebf79eb8340ed80708048790" dependencies = [ "nom", "unicode_categories", @@ -10993,7 +10931,7 @@ name = "sqlness-runner" version = "0.9.3" dependencies = [ "async-trait", - "clap 4.5.7", + "clap 4.5.18", "client", "common-error", "common-query", @@ -11048,7 +10986,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -11058,7 +10996,7 @@ source = "git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09 dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -11165,7 +11103,7 @@ version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1120e6a8cbd4d85d5532d2e8a245aef2128e1853981f8b6d9943264184843102" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "num_enum", "optional", ] @@ -11318,9 +11256,9 @@ dependencies = [ [[package]] name = "strum" -version = "0.26.2" +version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" dependencies = [ "strum_macros 0.26.4", ] @@ -11348,7 +11286,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -11361,7 +11299,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -11414,7 +11352,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "syn 2.0.66", + "syn 2.0.77", "typify", "walkdir", ] @@ -11427,42 +11365,42 @@ checksum = "9ec889155c56a34200d2c5aee147b8d29545fa7cce7f68b38d927f5d24ced8ef" dependencies = [ "heck 0.5.0", "prettyplease", - "prost 0.13.1", - "prost-build 0.13.1", - "prost-types 0.13.1", + "prost 0.13.3", + "prost-build 0.13.3", + "prost-types 0.13.3", "schemars", "semver", "serde", "serde_json", "serde_yaml", - "syn 2.0.66", + "syn 2.0.77", "typify", "walkdir", ] [[package]] name = "subtle" -version = "2.5.0" +version = "2.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbolic-common" -version = "12.9.2" +version = "12.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71297dc3e250f7dbdf8adb99e235da783d690f5819fdeb4cce39d9cfb0aca9f1" +checksum = "9fdf97c441f18a4f92425b896a4ec7a27e03631a0b1047ec4e34e9916a9a167e" dependencies = [ "debugid", - "memmap2 0.9.4", + "memmap2 0.9.5", "stable_deref_trait", "uuid", ] [[package]] name = "symbolic-demangle" -version = "12.9.2" +version = "12.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "424fa2c9bf2c862891b9cfd354a752751a6730fd838a4691e7f6c2c7957b9daf" +checksum = "bc8ece6b129e97e53d1fbb3f61d33a6a9e5369b11d01228c068094d6d134eaea" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -11482,9 +11420,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.66" +version = "2.0.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c42f3f41a2de00b01c0aaad383c5a45241efc8b2d1eda5661812fda5f3cdcff5" +checksum = "9f35bcdf61fd8e7be6caf75f429fdca8beb3ed76584befb503b1569faee373ed" dependencies = [ "proc-macro2", "quote", @@ -11519,7 +11457,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -11529,21 +11467,19 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" [[package]] -name = "synstructure" -version = "0.13.1" +name = "sync_wrapper" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", + "futures-core", ] [[package]] name = "sysinfo" -version = "0.30.12" +version = "0.30.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "732ffa00f53e6b2af46208fba5718d9662a421049204e156328b66791ffa15ae" +checksum = "0a5b4ddaee55fb2bea2bf0e5000747e5f5c0de765e5a5ff87f4cd106439f4bb3" dependencies = [ "cfg-if", "core-foundation-sys", @@ -11644,7 +11580,7 @@ dependencies = [ "lru", "lz4_flex 0.11.3", "measure_time", - "memmap2 0.9.4", + "memmap2 0.9.5", "num_cpus", "once_cell", "oneshot", @@ -11668,7 +11604,7 @@ dependencies = [ "time", "uuid", "winapi", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -11749,7 +11685,7 @@ dependencies = [ "tantivy-bitpacker", "tantivy-common", "tantivy-fst", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -11780,9 +11716,9 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "target-lexicon" -version = "0.12.14" +version = "0.12.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1fc403891a21bcfb7c37834ba66a547a8f402146eba7265b5a6d88059c9ff2f" +checksum = "61c41af27dd6d1e27b1b16b489db798443478cef1f06a660c96db617ba5de3b1" [[package]] name = "temp-env" @@ -11795,14 +11731,15 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.10.1" +version = "3.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" +checksum = "04cbcdd0c794ebb0d4cf35e88edd2f7d2c4c3e9a5a6dab322839b321c6a87a64" dependencies = [ "cfg-if", "fastrand", + "once_cell", "rustix", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -11895,7 +11832,7 @@ dependencies = [ "cache", "catalog", "chrono", - "clap 4.5.7", + "clap 4.5.18", "client", "cmd", "common-base", @@ -11957,7 +11894,7 @@ dependencies = [ "tower", "url", "uuid", - "zstd 0.13.1", + "zstd 0.13.2", ] [[package]] @@ -11983,29 +11920,29 @@ checksum = "23d434d3f8967a09480fb04132ebe0a3e088c173e6d0ee7897abbdf4eab0f8b9" [[package]] name = "thiserror" -version = "1.0.63" +version = "1.0.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0342370b38b6a11b6cc11d6a805569958d54cfa061a29969c3b5ce2ea405724" +checksum = "d50af8abc119fb8bb6dbabcfa89656f46f84aa0ac7688088608076ad2b459a84" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.63" +version = "1.0.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" +checksum = "08904e7672f5eb876eaaf87e0ce17857500934f4981c4a0ab2b4aa98baac7fc3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] name = "thread-id" -version = "4.2.1" +version = "4.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0ec81c46e9eb50deaa257be2f148adf052d1fb7701cfd55ccfab2525280b70b" +checksum = "cfe8f25bbdd100db7e1d34acf7fd2dc59c4bf8f7483f505eaa7d4f12f76cc0ea" dependencies = [ "libc", "winapi", @@ -12123,16 +12060,6 @@ dependencies = [ "log", ] -[[package]] -name = "tinystr" -version = "0.7.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9117f5d4db391c1cf6927e7bea3db74b9a1c1add8f7eda9ffd5364f40f57b82f" -dependencies = [ - "displaydoc", - "zerovec", -] - [[package]] name = "tinytemplate" version = "1.2.1" @@ -12145,9 +12072,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.6.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" dependencies = [ "tinyvec_macros", ] @@ -12160,22 +12087,21 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.38.0" +version = "1.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba4f4a02a7a80d6f274636f0aa95c7e383b912d41fe721a31f29e29698585a4a" +checksum = "e2b070231665d27ad9ec9b8df639893f46727666c6767db40317fbe920a5d998" dependencies = [ "backtrace", "bytes", "libc", - "mio", - "num_cpus", + "mio 1.0.2", "parking_lot 0.12.3", "pin-project-lite", "signal-hook-registry", "socket2 0.5.7", "tokio-macros", "tracing", - "windows-sys 0.48.0", + "windows-sys 0.52.0", ] [[package]] @@ -12190,13 +12116,13 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "2.3.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" +checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -12225,9 +12151,9 @@ dependencies = [ [[package]] name = "tokio-postgres" -version = "0.7.11" +version = "0.7.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03adcf0147e203b6032c0b2d30be1415ba03bc348901f3ff1cc0df6a733e60c3" +checksum = "3b5d3742945bc7d7f210693b0c58ae542c6fd47b17adbbda0885f3dcb34a6bdb" dependencies = [ "async-trait", "byteorder", @@ -12256,7 +12182,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04fb792ccd6bbcd4bba408eb8a292f70fc4a3589e5d793626f45190e6454b6ab" dependencies = [ "ring 0.17.8", - "rustls 0.23.10", + "rustls 0.23.13", "tokio", "tokio-postgres", "tokio-rustls 0.26.0", @@ -12298,19 +12224,18 @@ dependencies = [ [[package]] name = "tokio-rustls" version = "0.26.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" +source = "git+https://github.com/GreptimeTeam/tokio-rustls#4604ca6badfd1d10424718e5570cc481ab787fc8" dependencies = [ - "rustls 0.23.10", + "rustls 0.23.13", "rustls-pki-types", "tokio", ] [[package]] name = "tokio-stream" -version = "0.1.15" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" +checksum = "4f4e6ce100d0eb49a2734f8c0812bcd324cf357d21810932c5df6b96ef2b86f1" dependencies = [ "futures-core", "pin-project-lite", @@ -12333,9 +12258,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.11" +version = "0.7.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cf6b47b3771c49ac75ad09a6162f53ad4b8088b76ac60e8ec1455b31a189fe1" +checksum = "61e7c3654c13bcd040d4a03abee2c75b1d14a37b423cf5a813ceae1cc903ec6a" dependencies = [ "bytes", "futures-core", @@ -12357,21 +12282,21 @@ dependencies = [ [[package]] name = "toml" -version = "0.8.14" +version = "0.8.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f49eb2ab21d2f26bd6db7bf383edc527a7ebaee412d17af4d40fdccd442f335" +checksum = "a1ed1f98e3fdc28d6d910e6737ae6ab1a93bf1985935a1193e68f93eeb68d24e" dependencies = [ "serde", "serde_spanned", "toml_datetime", - "toml_edit 0.22.14", + "toml_edit 0.22.21", ] [[package]] name = "toml_datetime" -version = "0.6.6" +version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4badfd56924ae69bcc9039335b2e017639ce3f9b001c393c1b2d1ef846ce2cbf" +checksum = "0dd7358ecb8fc2f8d014bf86f6f638ce72ba252a2c3a2572f2a795f1d23efb41" dependencies = [ "serde", ] @@ -12382,33 +12307,22 @@ version = "0.19.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" dependencies = [ - "indexmap 2.2.6", + "indexmap 2.5.0", "toml_datetime", "winnow 0.5.40", ] [[package]] name = "toml_edit" -version = "0.21.1" +version = "0.22.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a8534fd7f78b5405e860340ad6575217ce99f38d4d5c8f2442cb5ecb50090e1" +checksum = "3b072cee73c449a636ffd6f32bd8de3a9f7119139aff882f44943ce2986dc5cf" dependencies = [ - "indexmap 2.2.6", - "toml_datetime", - "winnow 0.5.40", -] - -[[package]] -name = "toml_edit" -version = "0.22.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f21c7aaf97f1bd9ca9d4f9e73b0a6c74bd5afef56f2bc931943a6e1c37e04e38" -dependencies = [ - "indexmap 2.2.6", + "indexmap 2.5.0", "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.13", + "winnow 0.6.18", ] [[package]] @@ -12426,7 +12340,7 @@ dependencies = [ "h2", "http 0.2.12", "http-body 0.4.6", - "hyper 0.14.29", + "hyper 0.14.30", "hyper-timeout 0.4.1", "percent-encoding", "pin-project", @@ -12454,12 +12368,12 @@ dependencies = [ "h2", "http 0.2.12", "http-body 0.4.6", - "hyper 0.14.29", + "hyper 0.14.30", "hyper-timeout 0.4.1", "percent-encoding", "pin-project", "prost 0.12.6", - "rustls-pemfile 2.1.2", + "rustls-pemfile 2.1.3", "rustls-pki-types", "tokio", "tokio-rustls 0.25.0", @@ -12481,7 +12395,7 @@ dependencies = [ "proc-macro2", "prost-build 0.12.6", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -12524,9 +12438,9 @@ version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "61c5bb1d698276a2443e5ecfabc1008bf15a36c12e6a7176e7bf089ea9131140" dependencies = [ - "async-compression 0.4.11", + "async-compression 0.4.12", "base64 0.21.7", - "bitflags 2.5.0", + "bitflags 2.6.0", "bytes", "futures-core", "futures-util", @@ -12555,10 +12469,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e9cd434a998747dd2c4276bc96ee2e0c7a2eadf3cae88e52be55a05fa9053f5" dependencies = [ "base64 0.21.7", - "bitflags 2.5.0", + "bitflags 2.6.0", "bytes", "http 1.1.0", - "http-body 1.0.0", + "http-body 1.0.1", "http-body-util", "mime", "pin-project-lite", @@ -12569,15 +12483,15 @@ dependencies = [ [[package]] name = "tower-layer" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c20c8dbed6283a09604c3e69b4b7eeb54e298b8a600d4d5ecb5ad39de609f1d0" +checksum = "121c2a6cda46980bb0fcd1647ffaf6cd3fc79a013de288782836f6df9c48780e" [[package]] name = "tower-service" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" +checksum = "8df9b6e13f2d32c91b9bd719c00d1958837bc7dec474d94952798cc8e69eeec3" [[package]] name = "tracing" @@ -12611,7 +12525,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -12814,9 +12728,9 @@ checksum = "343e926fc669bc8cde4fa3129ab681c63671bae288b1f1081ceee6d9d37904fc" [[package]] name = "triomphe" -version = "0.1.12" +version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b2cb4fbb9995eeb36ac86fadf24031ccd58f99d6b4b2d7b911db70bddb80d90" +checksum = "859eb650cfee7434994602c3a68b25d77ad9e68c8a6cd491616ef86661382eb3" [[package]] name = "try-lock" @@ -12837,9 +12751,9 @@ dependencies = [ [[package]] name = "typeid" -version = "1.0.0" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "059d83cc991e7a42fc37bd50941885db0888e34209f8cfd9aab07ddec03bc9cf" +checksum = "0e13db2e0ccd5e14a544e8a246ba2312cd25223f616442d7f2cb0e3db614236e" [[package]] name = "typenum" @@ -12849,9 +12763,9 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "typetag" -version = "0.2.16" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "661d18414ec032a49ece2d56eee03636e43c4e8d577047ab334c0ba892e29aaf" +checksum = "52ba3b6e86ffe0054b2c44f2d86407388b933b16cb0a70eea3929420db1d9bbe" dependencies = [ "erased-serde", "inventory", @@ -12862,13 +12776,13 @@ dependencies = [ [[package]] name = "typetag-impl" -version = "0.2.16" +version = "0.2.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac73887f47b9312552aa90ef477927ff014d63d1920ca8037c6c1951eab64bb1" +checksum = "70b20a22c42c8f1cd23ce5e34f165d4d37038f5b663ad20fb6adbdf029172483" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -12896,7 +12810,7 @@ dependencies = [ "semver", "serde", "serde_json", - "syn 2.0.66", + "syn 2.0.77", "thiserror", "unicode-ident", ] @@ -12914,7 +12828,7 @@ dependencies = [ "serde", "serde_json", "serde_tokenstream", - "syn 2.0.66", + "syn 2.0.77", "typify-impl", ] @@ -12940,9 +12854,9 @@ dependencies = [ [[package]] name = "tzdb_data" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1889fdffac09d65c1d95c42d5202e9b21ad8c758f426e9fe09088817ea998d6" +checksum = "654c1ec546942ce0594e8d220e6b8e3899e0a0a8fe70ddd54d32a376dfefe3f8" dependencies = [ "tz-rs", ] @@ -13107,42 +13021,42 @@ checksum = "623f59e6af2a98bdafeb93fa277ac8e1e40440973001ca15cf4ae1541cd16d56" [[package]] name = "unicode-ident" -version = "1.0.12" +version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" +checksum = "e91b56cd4cadaeb79bbf1a5645f6b4f8dc5bde8834ad5894a8db35fda9efa1fe" [[package]] name = "unicode-normalization" -version = "0.1.23" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" +checksum = "5033c97c4262335cded6d6fc3e5c18ab755e1a3dc96376350f3d8e9f009ad956" dependencies = [ "tinyvec", ] [[package]] name = "unicode-properties" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4259d9d4425d9f0661581b804cb85fe66a4c631cadd8f490d1c13a35d5d9291" +checksum = "52ea75f83c0137a9b98608359a5f1af8144876eb67bcb1ce837368e906a9f524" [[package]] name = "unicode-segmentation" -version = "1.11.0" +version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" [[package]] name = "unicode-width" -version = "0.1.13" +version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" +checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" [[package]] name = "unicode-xid" -version = "0.2.4" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f962df74c8c05a667b5ee8bcf162993134c104e96440b663c8daa176dc772d8c" +checksum = "ebc1c04c71510c7f702b52b7c350734c9ff1295c464a03335b00bb84fc54f853" [[package]] name = "unicode_categories" @@ -13184,9 +13098,9 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "url" -version = "2.5.1" +version = "2.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7c25da092f0a868cdf09e8674cd3b7ef3a7d92a24253e663a2fb85e2496de56" +checksum = "22784dbdf76fdde8af1aeda5622b546b422b6fc585325248a2bf9f5e41e94d6c" dependencies = [ "form_urlencoded", "idna", @@ -13199,12 +13113,6 @@ version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "daf8dba3b7eb870caf1ddeed7bc9d2a049f3cfdfae7cb521b087cc33ae4c49da" -[[package]] -name = "utf16_iter" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8232dd3cdaed5356e0f716d285e4b40b932ac434100fe9b7e0e8e935b9e6246" - [[package]] name = "utf8-ranges" version = "1.0.5" @@ -13217,12 +13125,6 @@ version = "0.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "86bd8d4e895da8537e5315b8254664e6b769c4ff3db18321b297a1e7004392e3" -[[package]] -name = "utf8_iter" -version = "1.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" - [[package]] name = "utf8parse" version = "0.2.2" @@ -13231,9 +13133,9 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.8.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a183cf7feeba97b4dd1c0d46788634f6221d87fa961b305bed08c851829efcc0" +checksum = "81dfa00651efa65069b0b6b651f4aaa31ba9e3c3ce0137aaad053604ee7e0314" dependencies = [ "atomic", "getrandom", @@ -13244,13 +13146,13 @@ dependencies = [ [[package]] name = "uuid-macro-internal" -version = "1.8.0" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9881bea7cbe687e36c9ab3b778c36cd0487402e270304e8b1296d5085303c1a2" +checksum = "ee1cd046f83ea2c4e920d6ee9f7c3537ef928d75dce5d84a87c2c5d6b3999a3a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -13281,9 +13183,9 @@ checksum = "f1bddf1187be692e79c5ffeab891132dfb0f236ed36a43c7ed39f1165ee20191" [[package]] name = "vergen" -version = "8.3.1" +version = "8.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e27d6bdd219887a9eadd19e1c34f32e47fa332301184935c6d9bca26f3cca525" +checksum = "2990d9ea5967266ea0ccf413a4aa5c42a93dbcfda9cb49a97de6931726b12566" dependencies = [ "anyhow", "rustversion", @@ -13292,9 +13194,9 @@ dependencies = [ [[package]] name = "version_check" -version = "0.9.4" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" [[package]] name = "vob" @@ -13346,34 +13248,35 @@ checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" [[package]] name = "wasm-bindgen" -version = "0.2.92" +version = "0.2.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" +checksum = "a82edfc16a6c469f5f44dc7b571814045d60404b55a0ee849f9bcfa2e63dd9b5" dependencies = [ "cfg-if", + "once_cell", "wasm-bindgen-macro", ] [[package]] name = "wasm-bindgen-backend" -version = "0.2.92" +version = "0.2.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" +checksum = "9de396da306523044d3302746f1208fa71d7532227f15e347e2d93e4145dd77b" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.42" +version = "0.4.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76bc14366121efc8dbb487ab05bcc9d346b3b5ec0eaa76e46594cabbe51762c0" +checksum = "61e9300f63a621e96ed275155c108eb6f843b6a26d053f122ab69724559dc8ed" dependencies = [ "cfg-if", "js-sys", @@ -13383,9 +13286,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.92" +version = "0.2.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" +checksum = "585c4c91a46b072c92e908d99cb1dcdf95c5218eeb6f3bf1efa991ee7a68cccf" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -13393,22 +13296,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.92" +version = "0.2.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" +checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", + "syn 2.0.77", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.92" +version = "0.2.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" +checksum = "c62a0a307cb4a311d3a07867860911ca130c3494e8c2719593806c08bc5d0484" [[package]] name = "wasm-streams" @@ -13425,9 +13328,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.69" +version = "0.3.70" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" +checksum = "26fdeaafd9bd129f65e7c031593c24d62186301e0c72c8978fa1678be7d532c0" dependencies = [ "js-sys", "wasm-bindgen", @@ -13497,9 +13400,9 @@ checksum = "5f20c57d8d7db6d3b86154206ae5d8fba62dd39573114de97c2cb0578251f8e1" [[package]] name = "webpki-roots" -version = "0.26.2" +version = "0.26.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c452ad30530b54a4d8e71952716a212b08efd0f3562baa66c29a618b07da7c3" +checksum = "841c67bff177718f1d4dfefde8d8f0e78f9b6589319ba88312f567fc5841a958" dependencies = [ "rustls-pki-types", ] @@ -13518,20 +13421,20 @@ dependencies = [ [[package]] name = "whoami" -version = "1.5.1" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a44ab49fad634e88f55bf8f9bb3abd2f27d7204172a112c7c9987e01c1c94ea9" +checksum = "372d5b87f58ec45c384ba03563b03544dc5fadc3983e434b286913f5b4a9bb6d" dependencies = [ - "redox_syscall 0.4.1", + "redox_syscall 0.5.4", "wasite", "web-sys", ] [[package]] name = "wide" -version = "0.7.24" +version = "0.7.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a040b111774ab63a19ef46bbc149398ab372b4ccdcfd719e9814dbd7dfd76c8" +checksum = "b828f995bf1e9622031f8009f8481a85406ce1f4d4588ff746d872043e855690" dependencies = [ "bytemuck", "safe_arch", @@ -13561,11 +13464,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.8" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d4cc384e1e73b93bafa6fb4f1df8c41695c8a91cf9c4c64358067d15a7b6c6b" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -13594,7 +13497,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e48a53791691ab099e5e2ad123536d0fff50652600abaf43bbf952894110d0be" dependencies = [ "windows-core", - "windows-targets 0.52.5", + "windows-targets 0.52.6", ] [[package]] @@ -13603,7 +13506,37 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.5", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-registry" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e400001bb720a623c1c69032f8e3e4cf09984deec740f007dd2b03ec864804b0" +dependencies = [ + "windows-result", + "windows-strings", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-result" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-strings" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" +dependencies = [ + "windows-result", + "windows-targets 0.52.6", ] [[package]] @@ -13630,7 +13563,16 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.5", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", ] [[package]] @@ -13665,18 +13607,18 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" dependencies = [ - "windows_aarch64_gnullvm 0.52.5", - "windows_aarch64_msvc 0.52.5", - "windows_i686_gnu 0.52.5", + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", "windows_i686_gnullvm", - "windows_i686_msvc 0.52.5", - "windows_x86_64_gnu 0.52.5", - "windows_x86_64_gnullvm 0.52.5", - "windows_x86_64_msvc 0.52.5", + "windows_i686_msvc 0.52.6", + "windows_x86_64_gnu 0.52.6", + "windows_x86_64_gnullvm 0.52.6", + "windows_x86_64_msvc 0.52.6", ] [[package]] @@ -13693,9 +13635,9 @@ checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" [[package]] name = "windows_aarch64_msvc" @@ -13717,9 +13659,9 @@ checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_aarch64_msvc" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" [[package]] name = "windows_i686_gnu" @@ -13741,15 +13683,15 @@ checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_gnu" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" [[package]] name = "windows_i686_gnullvm" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" [[package]] name = "windows_i686_msvc" @@ -13771,9 +13713,9 @@ checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_i686_msvc" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" [[package]] name = "windows_x86_64_gnu" @@ -13795,9 +13737,9 @@ checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnu" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" [[package]] name = "windows_x86_64_gnullvm" @@ -13813,9 +13755,9 @@ checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_gnullvm" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" [[package]] name = "windows_x86_64_msvc" @@ -13837,9 +13779,9 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "windows_x86_64_msvc" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" @@ -13852,9 +13794,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.13" +version = "0.6.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59b5e5f6c299a3c7890b876a2a587f3115162487e704907d9b6cd29473052ba1" +checksum = "68a9bda4691f099d435ad181000724da8e5899daa10713c2d432552b9ccd3a6f" dependencies = [ "memchr", ] @@ -13868,28 +13810,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "winreg" -version = "0.52.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a277a57398d4bfa075df44f501a17cfdf8542d224f0d36095a2adc7aee4ef0a5" -dependencies = [ - "cfg-if", - "windows-sys 0.48.0", -] - -[[package]] -name = "write16" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" - -[[package]] -name = "writeable" -version = "0.5.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" - [[package]] name = "wyz" version = "0.5.1" @@ -13920,9 +13840,9 @@ dependencies = [ [[package]] name = "xml-rs" -version = "0.8.20" +version = "0.8.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "791978798f0597cfc70478424c2b4fdc2b7a8024aaff78497ef00f24ef674193" +checksum = "af4e2e2f7cba5a093896c1e150fbfe177d1883e7448200efb81d40b9d339ef26" [[package]] name = "xz2" @@ -13944,73 +13864,29 @@ dependencies = [ [[package]] name = "yansi" -version = "0.5.1" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" - -[[package]] -name = "yoke" -version = "0.7.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c5b1314b079b0930c31e3af543d8ee1757b1951ae1e1565ec704403a7240ca5" -dependencies = [ - "serde", - "stable_deref_trait", - "yoke-derive", - "zerofrom", -] - -[[package]] -name = "yoke-derive" -version = "0.7.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28cc31741b18cb6f1d5ff12f5b7523e3d6eb0852bbbad19d73905511d9849b95" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", - "synstructure", -] +checksum = "cfe53a6657fd280eaa890a3bc59152892ffa3e30101319d168b781ed6529b049" [[package]] name = "zerocopy" -version = "0.7.34" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae87e3fcd617500e5d106f0380cf7b77f3c6092aae37191433159dda23cfb087" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" dependencies = [ + "byteorder", "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.34" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15e934569e47891f7d9411f1a451d947a60e000ab3bd24fbb970f000387d1b3b" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", -] - -[[package]] -name = "zerofrom" -version = "0.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91ec111ce797d0e0784a1116d0ddcdbea84322cd79e5d5ad173daeba4f93ab55" -dependencies = [ - "zerofrom-derive", -] - -[[package]] -name = "zerofrom-derive" -version = "0.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ea7b4a3637ea8669cedf0f1fd5c286a17f3de97b8dd5a70a6c167a1730e63a5" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", - "synstructure", + "syn 2.0.77", ] [[package]] @@ -14030,29 +13906,7 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.66", -] - -[[package]] -name = "zerovec" -version = "0.10.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa2b893d79df23bfb12d5461018d408ea19dfafe76c2c7ef6d4eba614f8ff079" -dependencies = [ - "yoke", - "zerofrom", - "zerovec-derive", -] - -[[package]] -name = "zerovec-derive" -version = "0.10.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.66", + "syn 2.0.77", ] [[package]] @@ -14075,11 +13929,11 @@ dependencies = [ [[package]] name = "zstd" -version = "0.13.1" +version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d789b1514203a1120ad2429eae43a7bd32b90976a7bb8a05f7ec02fa88cc23a" +checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" dependencies = [ - "zstd-safe 7.1.0", + "zstd-safe 7.2.1", ] [[package]] @@ -14104,18 +13958,18 @@ dependencies = [ [[package]] name = "zstd-safe" -version = "7.1.0" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd99b45c6bc03a018c8b8a86025678c87e55526064e38f9df301989dce7ec0a" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" dependencies = [ "zstd-sys", ] [[package]] name = "zstd-sys" -version = "2.0.10+zstd.1.5.6" +version = "2.0.13+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c253a4914af5bafc8fa8c86ee400827e83cf6ec01195ec1f1ed8441bf00d65aa" +checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" dependencies = [ "cc", "pkg-config", diff --git a/Cargo.toml b/Cargo.toml index f0b93ac653..44b2cda122 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -245,6 +245,15 @@ store-api = { path = "src/store-api" } substrait = { path = "src/common/substrait" } table = { path = "src/table" } +[patch.crates-io] +# change all rustls dependencies to use our fork to default to `ring` to make it "just work" +hyper-rustls = { git = "https://github.com/GreptimeTeam/hyper-rustls" } +rustls = { git = "https://github.com/GreptimeTeam/rustls" } +tokio-rustls = { git = "https://github.com/GreptimeTeam/tokio-rustls" } +# This is commented, since we are not using aws-lc-sys, if we need to use it, we need to uncomment this line or use a release after this commit, or it wouldn't compile with gcc < 8.1 +# see https://github.com/aws/aws-lc-rs/pull/526 +# aws-lc-sys = { git ="https://github.com/aws/aws-lc-rs", rev = "556558441e3494af4b156ae95ebc07ebc2fd38aa" } + [workspace.dependencies.meter-macros] git = "https://github.com/GreptimeTeam/greptime-meter.git" rev = "80eb97c24c88af4dd9a86f8bbaf50e741d4eb8cd" diff --git a/Makefile b/Makefile index da9e8da5c0..c63acb67b4 100644 --- a/Makefile +++ b/Makefile @@ -8,7 +8,7 @@ CARGO_BUILD_OPTS := --locked IMAGE_REGISTRY ?= docker.io IMAGE_NAMESPACE ?= greptime IMAGE_TAG ?= latest -DEV_BUILDER_IMAGE_TAG ?= 2024-06-06-1acda74c-20240919113454 +DEV_BUILDER_IMAGE_TAG ?= 2024-06-06-5674c14f-20240920110415 BUILDX_MULTI_PLATFORM_BUILD ?= false BUILDX_BUILDER_NAME ?= gtbuilder BASE_IMAGE ?= ubuntu diff --git a/docker/dev-builder/binstall/pull_binstall.sh b/docker/dev-builder/binstall/pull_binstall.sh new file mode 100755 index 0000000000..a14ee6c7ea --- /dev/null +++ b/docker/dev-builder/binstall/pull_binstall.sh @@ -0,0 +1,50 @@ +#!/bin/bash + +set -euxo pipefail + +cd "$(mktemp -d)" +# Fix version to v1.6.6, this is different than the latest version in original install script in +# https://raw.githubusercontent.com/cargo-bins/cargo-binstall/main/install-from-binstall-release.sh +base_url="https://github.com/cargo-bins/cargo-binstall/releases/download/v1.6.6/cargo-binstall-" + +os="$(uname -s)" +if [ "$os" == "Darwin" ]; then + url="${base_url}universal-apple-darwin.zip" + curl -LO --proto '=https' --tlsv1.2 -sSf "$url" + unzip cargo-binstall-universal-apple-darwin.zip +elif [ "$os" == "Linux" ]; then + machine="$(uname -m)" + if [ "$machine" == "armv7l" ]; then + machine="armv7" + fi + target="${machine}-unknown-linux-musl" + if [ "$machine" == "armv7" ]; then + target="${target}eabihf" + fi + + url="${base_url}${target}.tgz" + curl -L --proto '=https' --tlsv1.2 -sSf "$url" | tar -xvzf - +elif [ "${OS-}" = "Windows_NT" ]; then + machine="$(uname -m)" + target="${machine}-pc-windows-msvc" + url="${base_url}${target}.zip" + curl -LO --proto '=https' --tlsv1.2 -sSf "$url" + unzip "cargo-binstall-${target}.zip" +else + echo "Unsupported OS ${os}" + exit 1 +fi + +./cargo-binstall -y --force cargo-binstall + +CARGO_HOME="${CARGO_HOME:-$HOME/.cargo}" + +if ! [[ ":$PATH:" == *":$CARGO_HOME/bin:"* ]]; then + if [ -n "${CI:-}" ] && [ -n "${GITHUB_PATH:-}" ]; then + echo "$CARGO_HOME/bin" >> "$GITHUB_PATH" + else + echo + printf "\033[0;31mYour path is missing %s, you might want to add it.\033[0m\n" "$CARGO_HOME/bin" + echo + fi +fi diff --git a/docker/dev-builder/centos/Dockerfile b/docker/dev-builder/centos/Dockerfile index af7e778e03..04327ab393 100644 --- a/docker/dev-builder/centos/Dockerfile +++ b/docker/dev-builder/centos/Dockerfile @@ -32,7 +32,9 @@ RUN rustup toolchain install ${RUST_TOOLCHAIN} # Install cargo-binstall with a specific version to adapt the current rust toolchain. # Note: if we use the latest version, we may encounter the following `use of unstable library feature 'io_error_downcast'` error. -RUN cargo install cargo-binstall --version 1.6.6 --locked +# compile from source take too long, so we use the precompiled binary instead +COPY $DOCKER_BUILD_ROOT/docker/dev-builder/binstall/pull_binstall.sh /usr/local/bin/pull_binstall.sh +RUN chmod +x /usr/local/bin/pull_binstall.sh && /usr/local/bin/pull_binstall.sh # Install nextest. RUN cargo binstall cargo-nextest --no-confirm diff --git a/docker/dev-builder/ubuntu/Dockerfile b/docker/dev-builder/ubuntu/Dockerfile index d70824e911..733bfdab62 100644 --- a/docker/dev-builder/ubuntu/Dockerfile +++ b/docker/dev-builder/ubuntu/Dockerfile @@ -26,7 +26,12 @@ RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y \ # https://github.com/GreptimeTeam/greptimedb/actions/runs/10935485852/job/30357457188#step:3:7106 # `aws-lc-sys` require gcc >= 10.3.0 to work, hence alias to use gcc-10 -RUN alias gcc="gcc-10" && alias cc="gcc-10" +RUN apt-get remove -y gcc-9 g++-9 cpp-9 && \ + apt-get install -y gcc-10 g++-10 cpp-10 make cmake && \ + ln -sf /usr/bin/gcc-10 /usr/bin/gcc && ln -sf /usr/bin/g++-10 /usr/bin/g++ && \ + ln -sf /usr/bin/gcc-10 /usr/bin/cc && \ + ln -sf /usr/bin/g++-10 /usr/bin/cpp && ln -sf /usr/bin/g++-10 /usr/bin/c++ && \ + cc --version && gcc --version && g++ --version && cpp --version && c++ --version # Remove Python 3.8 and install pip. RUN apt-get -y purge python3.8 && \ @@ -61,7 +66,9 @@ RUN rustup toolchain install ${RUST_TOOLCHAIN} # Install cargo-binstall with a specific version to adapt the current rust toolchain. # Note: if we use the latest version, we may encounter the following `use of unstable library feature 'io_error_downcast'` error. -RUN cargo install cargo-binstall --version 1.6.6 --locked +# compile from source take too long, so we use the precompiled binary instead +COPY $DOCKER_BUILD_ROOT/docker/dev-builder/binstall/pull_binstall.sh /usr/local/bin/pull_binstall.sh +RUN chmod +x /usr/local/bin/pull_binstall.sh && /usr/local/bin/pull_binstall.sh # Install nextest. RUN cargo binstall cargo-nextest --no-confirm From d1b252736d8d82c9e36fce155ab1fbb55f91ed5a Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Tue, 24 Sep 2024 11:48:16 +0800 Subject: [PATCH 033/128] refactor: unify the styling in `create_or_alter_tables_on_demand` (#4756) * refactor: refactor `create_or_alter_tables_on_demand` * chore: apply suggestions from CR --- src/operator/src/insert.rs | 176 +++++++++++++++++-------------------- 1 file changed, 82 insertions(+), 94 deletions(-) diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index c56e51c7a3..e0f0d39236 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -505,28 +505,25 @@ impl Inserter { let table_info = table.table_info(); table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); if let Some(alter_expr) = - self.get_alter_table_expr_on_demand(req, table, ctx)? + self.get_alter_table_expr_on_demand(req, &table, ctx)? { alter_tables.push(alter_expr); } } None => { - create_tables.push(req); + let create_expr = + self.get_create_table_expr_on_demand(req, &auto_create_table_type, ctx)?; + create_tables.push(create_expr); } } } match auto_create_table_type { - AutoCreateTableType::Logical(on_physical_table) => { + AutoCreateTableType::Logical(_) => { if !create_tables.is_empty() { // Creates logical tables in batch. let tables = self - .create_logical_tables( - create_tables, - ctx, - &on_physical_table, - statement_executor, - ) + .create_logical_tables(create_tables, ctx, statement_executor) .await?; for table in tables { @@ -544,14 +541,9 @@ impl Inserter { AutoCreateTableType::Physical | AutoCreateTableType::Log | AutoCreateTableType::LastNonNull => { - for req in create_tables { + for create_table in create_tables { let table = self - .create_non_logical_table( - req, - ctx, - statement_executor, - auto_create_table_type.clone(), - ) + .create_physical_table(create_table, ctx, statement_executor) .await?; let table_info = table.table_info(); table_name_to_ids.insert(table_info.name.clone(), table_info.table_id()); @@ -605,7 +597,8 @@ impl Inserter { options: None, }, ]; - let create_table_expr = &mut build_create_table_expr(&table_reference, &default_schema)?; + let create_table_expr = + &mut build_create_table_expr(&table_reference, &default_schema, default_engine())?; create_table_expr.engine = METRIC_ENGINE_NAME.to_string(); create_table_expr @@ -641,10 +634,61 @@ impl Inserter { .context(CatalogSnafu) } + fn get_create_table_expr_on_demand( + &self, + req: &RowInsertRequest, + create_type: &AutoCreateTableType, + ctx: &QueryContextRef, + ) -> Result { + let mut table_options = Vec::with_capacity(4); + if let Some(ttl) = ctx.extension(TTL_KEY) { + table_options.push((TTL_KEY, ttl)); + } + + let mut engine_name = default_engine(); + match create_type { + AutoCreateTableType::Logical(physical_table) => { + engine_name = METRIC_ENGINE_NAME; + table_options.push((LOGICAL_TABLE_METADATA_KEY, physical_table)); + } + AutoCreateTableType::Physical => { + if let Some(append_mode) = ctx.extension(APPEND_MODE_KEY) { + table_options.push((APPEND_MODE_KEY, append_mode)); + } + if let Some(merge_mode) = ctx.extension(MERGE_MODE_KEY) { + table_options.push((MERGE_MODE_KEY, merge_mode)); + } + } + // Set append_mode to true for log table. + // because log tables should keep rows with the same ts and tags. + AutoCreateTableType::Log => { + table_options.push((APPEND_MODE_KEY, "true")); + } + AutoCreateTableType::LastNonNull => { + table_options.push((MERGE_MODE_KEY, "last_non_null")); + } + } + + let schema = ctx.current_schema(); + let table_ref = TableReference::full(ctx.current_catalog(), &schema, &req.table_name); + // SAFETY: `req.rows` is guaranteed to be `Some` by `handle_row_inserts_with_create_type()`. + let request_schema = req.rows.as_ref().unwrap().schema.as_slice(); + let mut create_table_expr = + build_create_table_expr(&table_ref, request_schema, engine_name)?; + info!("Table `{table_ref}` does not exist, try creating table"); + for (k, v) in table_options { + create_table_expr + .table_options + .insert(k.to_string(), v.to_string()); + } + + Ok(create_table_expr) + } + fn get_alter_table_expr_on_demand( &self, req: &RowInsertRequest, - table: TableRef, + table: &TableRef, ctx: &QueryContextRef, ) -> Result> { let catalog_name = ctx.current_catalog(); @@ -667,76 +711,37 @@ impl Inserter { })) } - /// Creates a non-logical table by create type. - /// # Panics - /// Panics if `create_type` is `AutoCreateTableType::Logical`. - async fn create_non_logical_table( - &self, - req: &RowInsertRequest, - ctx: &QueryContextRef, - statement_executor: &StatementExecutor, - create_type: AutoCreateTableType, - ) -> Result { - let mut hint_options = vec![]; - - if let Some(ttl) = ctx.extension(TTL_KEY) { - hint_options.push((TTL_KEY, ttl)); - } - - match create_type { - AutoCreateTableType::Logical(_) => unreachable!(), - AutoCreateTableType::Physical => { - if let Some(append_mode) = ctx.extension(APPEND_MODE_KEY) { - hint_options.push((APPEND_MODE_KEY, append_mode)); - } - if let Some(merge_mode) = ctx.extension(MERGE_MODE_KEY) { - hint_options.push((MERGE_MODE_KEY, merge_mode)); - } - } - // Set append_mode to true for log table. - // because log tables should keep rows with the same ts and tags. - AutoCreateTableType::Log => { - hint_options.push((APPEND_MODE_KEY, "true")); - } - AutoCreateTableType::LastNonNull => { - hint_options.push((MERGE_MODE_KEY, "last_non_null")); - } - } - let options: &[(&str, &str)] = hint_options.as_slice(); - - self.create_table_with_options(req, ctx, statement_executor, options) - .await - } - /// Creates a table with options. - async fn create_table_with_options( + async fn create_physical_table( &self, - req: &RowInsertRequest, + mut create_table_expr: CreateTableExpr, ctx: &QueryContextRef, statement_executor: &StatementExecutor, - options: &[(&str, &str)], ) -> Result { - let schema = ctx.current_schema(); - let table_ref = TableReference::full(ctx.current_catalog(), &schema, &req.table_name); - // SAFETY: `req.rows` is guaranteed to be `Some` by `handle_row_inserts_with_create_type()`. - let request_schema = req.rows.as_ref().unwrap().schema.as_slice(); - let create_table_expr = &mut build_create_table_expr(&table_ref, request_schema)?; + { + let table_ref = TableReference::full( + &create_table_expr.catalog_name, + &create_table_expr.schema_name, + &create_table_expr.table_name, + ); - info!("Table `{table_ref}` does not exist, try creating table"); - for (k, v) in options { - create_table_expr - .table_options - .insert(k.to_string(), v.to_string()); + info!("Table `{table_ref}` does not exist, try creating table"); } let res = statement_executor - .create_table_inner(create_table_expr, None, ctx.clone()) + .create_table_inner(&mut create_table_expr, None, ctx.clone()) .await; + let table_ref = TableReference::full( + &create_table_expr.catalog_name, + &create_table_expr.schema_name, + &create_table_expr.table_name, + ); + match res { Ok(table) => { info!( "Successfully created table {} with options: {:?}", - table_ref, options + table_ref, create_table_expr.table_options, ); Ok(table) } @@ -749,30 +754,12 @@ impl Inserter { async fn create_logical_tables( &self, - create_tables: Vec<&RowInsertRequest>, + create_table_exprs: Vec, ctx: &QueryContextRef, - physical_table: &str, statement_executor: &StatementExecutor, ) -> Result> { let catalog_name = ctx.current_catalog(); let schema_name = ctx.current_schema(); - let create_table_exprs = create_tables - .iter() - .map(|req| { - let table_ref = TableReference::full(catalog_name, &schema_name, &req.table_name); - let request_schema = req.rows.as_ref().unwrap().schema.as_slice(); - let mut create_table_expr = build_create_table_expr(&table_ref, request_schema)?; - - create_table_expr.engine = METRIC_ENGINE_NAME.to_string(); - create_table_expr.table_options.insert( - LOGICAL_TABLE_METADATA_KEY.to_string(), - physical_table.to_string(), - ); - - Ok(create_table_expr) - }) - .collect::>>()?; - let res = statement_executor .create_logical_tables(catalog_name, &schema_name, &create_table_exprs, ctx.clone()) .await; @@ -827,6 +814,7 @@ fn validate_column_count_match(requests: &RowInsertRequests) -> Result<()> { fn build_create_table_expr( table: &TableReference, request_schema: &[ColumnSchema], + engine: &str, ) -> Result { - CreateExprFactory.create_table_expr_by_column_schemas(table, request_schema, default_engine()) + CreateExprFactory.create_table_expr_by_column_schemas(table, request_schema, engine) } From e3c0b5482f379f71433eb2fbd0d8ea63c38018f1 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Tue, 24 Sep 2024 16:45:55 +0800 Subject: [PATCH 034/128] feat: returning warning instead of error on unsupported `SET` statement (#4761) * feat: add capability to send warning to pgclient * fix: refactor query context to carry query scope data * feat: return a warning for unsupported postgres statement --- src/operator/src/statement.rs | 16 +++++-- src/servers/src/postgres/handler.rs | 42 ++++++++++++++++--- src/servers/src/postgres/types.rs | 2 +- src/servers/src/postgres/types/error.rs | 40 +++++++++--------- src/session/src/context.rs | 56 ++++++++++++++++++------- src/session/src/lib.rs | 2 +- 6 files changed, 112 insertions(+), 46 deletions(-) diff --git a/src/operator/src/statement.rs b/src/operator/src/statement.rs index 4dc43e0d92..7c76d0dcff 100644 --- a/src/operator/src/statement.rs +++ b/src/operator/src/statement.rs @@ -46,7 +46,7 @@ use datafusion_expr::LogicalPlan; use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef}; use query::parser::QueryStatement; use query::QueryEngineRef; -use session::context::QueryContextRef; +use session::context::{Channel, QueryContextRef}; use session::table_name::table_idents_to_full_name; use snafu::{ensure, OptionExt, ResultExt}; use sql::statements::copy::{CopyDatabase, CopyDatabaseArgument, CopyTable, CopyTableArgument}; @@ -338,10 +338,18 @@ impl StatementExecutor { "CLIENT_ENCODING" => validate_client_encoding(set_var)?, _ => { - return NotSupportedSnafu { - feat: format!("Unsupported set variable {}", var_name), + // for postgres, we give unknown SET statements a warning with + // success, this is prevent the SET call becoming a blocker + // of connection establishment + // + if query_ctx.channel() == Channel::Postgres { + query_ctx.set_warning(format!("Unsupported set variable {}", var_name)); + } else { + return NotSupportedSnafu { + feat: format!("Unsupported set variable {}", var_name), + } + .fail(); } - .fail() } } Ok(Output::new_with_affected_rows(0)) diff --git a/src/servers/src/postgres/handler.rs b/src/servers/src/postgres/handler.rs index 158e2cab4d..522c558cdc 100644 --- a/src/servers/src/postgres/handler.rs +++ b/src/servers/src/postgres/handler.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::fmt::Debug; use std::sync::Arc; use async_trait::async_trait; @@ -23,7 +24,7 @@ use common_telemetry::{debug, error, tracing}; use datafusion_common::ParamValues; use datatypes::prelude::ConcreteDataType; use datatypes::schema::SchemaRef; -use futures::{future, stream, Stream, StreamExt}; +use futures::{future, stream, Sink, SinkExt, Stream, StreamExt}; use pgwire::api::portal::{Format, Portal}; use pgwire::api::query::{ExtendedQueryHandler, SimpleQueryHandler}; use pgwire::api::results::{ @@ -32,6 +33,7 @@ use pgwire::api::results::{ use pgwire::api::stmt::{QueryParser, StoredStatement}; use pgwire::api::{ClientInfo, Type}; use pgwire::error::{ErrorInfo, PgWireError, PgWireResult}; +use pgwire::messages::PgWireBackendMessage; use query::query_engine::DescribeResult; use session::context::QueryContextRef; use session::Session; @@ -49,11 +51,13 @@ impl SimpleQueryHandler for PostgresServerHandlerInner { #[tracing::instrument(skip_all, fields(protocol = "postgres"))] async fn do_query<'a, C>( &self, - _client: &mut C, + client: &mut C, query: &'a str, ) -> PgWireResult>> where - C: ClientInfo + Unpin + Send + Sync, + C: ClientInfo + Sink + Unpin + Send + Sync, + C::Error: Debug, + PgWireError: From<>::Error>, { let query_ctx = self.session.new_query_context(); let db = query_ctx.get_db_string(); @@ -67,6 +71,7 @@ impl SimpleQueryHandler for PostgresServerHandlerInner { } if let Some(resps) = fixtures::process(query, query_ctx.clone()) { + send_warning_opt(client, query_ctx).await?; Ok(resps) } else { let outputs = self.query_handler.do_query(query, query_ctx.clone()).await; @@ -79,11 +84,34 @@ impl SimpleQueryHandler for PostgresServerHandlerInner { results.push(resp); } + send_warning_opt(client, query_ctx).await?; Ok(results) } } } +async fn send_warning_opt(client: &mut C, query_context: QueryContextRef) -> PgWireResult<()> +where + C: Sink + Unpin + Send + Sync, + C::Error: Debug, + PgWireError: From<>::Error>, +{ + if let Some(warning) = query_context.warning() { + client + .feed(PgWireBackendMessage::NoticeResponse( + ErrorInfo::new( + PgErrorSeverity::Warning.to_string(), + PgErrorCode::Ec01000.code(), + warning.to_string(), + ) + .into(), + )) + .await?; + } + + Ok(()) +} + pub(crate) fn output_to_query_response<'a>( query_ctx: QueryContextRef, output: Result, @@ -247,12 +275,14 @@ impl ExtendedQueryHandler for PostgresServerHandlerInner { async fn do_query<'a, C>( &self, - _client: &mut C, + client: &mut C, portal: &'a Portal, _max_rows: usize, ) -> PgWireResult> where - C: ClientInfo + Unpin + Send + Sync, + C: ClientInfo + Sink + Unpin + Send + Sync, + C::Error: Debug, + PgWireError: From<>::Error>, { let query_ctx = self.session.new_query_context(); let db = query_ctx.get_db_string(); @@ -268,6 +298,7 @@ impl ExtendedQueryHandler for PostgresServerHandlerInner { } if let Some(mut resps) = fixtures::process(&sql_plan.query, query_ctx.clone()) { + send_warning_opt(client, query_ctx).await?; // if the statement matches our predefined rules, return it early return Ok(resps.remove(0)); } @@ -297,6 +328,7 @@ impl ExtendedQueryHandler for PostgresServerHandlerInner { .remove(0) }; + send_warning_opt(client, query_ctx.clone()).await?; output_to_query_response(query_ctx, output, &portal.result_column_format) } diff --git a/src/servers/src/postgres/types.rs b/src/servers/src/postgres/types.rs index 2e4a805ef0..a5d1d392ac 100644 --- a/src/servers/src/postgres/types.rs +++ b/src/servers/src/postgres/types.rs @@ -37,7 +37,7 @@ use session::session_config::PGByteaOutputValue; use self::bytea::{EscapeOutputBytea, HexOutputBytea}; use self::datetime::{StylingDate, StylingDateTime}; -pub use self::error::PgErrorCode; +pub use self::error::{PgErrorCode, PgErrorSeverity}; use self::interval::PgInterval; use crate::error::{self as server_error, Error, Result}; use crate::SqlPlan; diff --git a/src/servers/src/postgres/types/error.rs b/src/servers/src/postgres/types/error.rs index 928c5454ce..9e6f570f26 100644 --- a/src/servers/src/postgres/types/error.rs +++ b/src/servers/src/postgres/types/error.rs @@ -19,7 +19,7 @@ use strum::{AsRefStr, Display, EnumIter, EnumMessage}; #[derive(Display, Debug, PartialEq)] #[allow(dead_code)] -enum ErrorSeverity { +pub enum PgErrorSeverity { #[strum(serialize = "INFO")] Info, #[strum(serialize = "DEBUG")] @@ -335,23 +335,23 @@ pub enum PgErrorCode { } impl PgErrorCode { - fn severity(&self) -> ErrorSeverity { + fn severity(&self) -> PgErrorSeverity { match self { - PgErrorCode::Ec00000 => ErrorSeverity::Info, - PgErrorCode::Ec01000 => ErrorSeverity::Warning, + PgErrorCode::Ec00000 => PgErrorSeverity::Info, + PgErrorCode::Ec01000 => PgErrorSeverity::Warning, PgErrorCode::EcXX000 | PgErrorCode::Ec42P14 | PgErrorCode::Ec22023 => { - ErrorSeverity::Error + PgErrorSeverity::Error } PgErrorCode::Ec28000 | PgErrorCode::Ec28P01 | PgErrorCode::Ec3D000 => { - ErrorSeverity::Fatal + PgErrorSeverity::Fatal } - _ => ErrorSeverity::Error, + _ => PgErrorSeverity::Error, } } - fn code(&self) -> String { + pub(crate) fn code(&self) -> String { self.as_ref()[2..].to_string() } @@ -428,19 +428,19 @@ mod tests { use common_error::status_code::StatusCode; use strum::{EnumMessage, IntoEnumIterator}; - use super::{ErrorInfo, ErrorSeverity, PgErrorCode}; + use super::{ErrorInfo, PgErrorCode, PgErrorSeverity}; #[test] fn test_error_severity() { // test for ErrorSeverity enum - assert_eq!("INFO", ErrorSeverity::Info.to_string()); - assert_eq!("DEBUG", ErrorSeverity::Debug.to_string()); - assert_eq!("NOTICE", ErrorSeverity::Notice.to_string()); - assert_eq!("WARNING", ErrorSeverity::Warning.to_string()); + assert_eq!("INFO", PgErrorSeverity::Info.to_string()); + assert_eq!("DEBUG", PgErrorSeverity::Debug.to_string()); + assert_eq!("NOTICE", PgErrorSeverity::Notice.to_string()); + assert_eq!("WARNING", PgErrorSeverity::Warning.to_string()); - assert_eq!("ERROR", ErrorSeverity::Error.to_string()); - assert_eq!("FATAL", ErrorSeverity::Fatal.to_string()); - assert_eq!("PANIC", ErrorSeverity::Panic.to_string()); + assert_eq!("ERROR", PgErrorSeverity::Error.to_string()); + assert_eq!("FATAL", PgErrorSeverity::Fatal.to_string()); + assert_eq!("PANIC", PgErrorSeverity::Panic.to_string()); // test for severity method for code in PgErrorCode::iter() { @@ -448,13 +448,13 @@ mod tests { assert_eq!("Ec", &name[0..2]); if name.starts_with("Ec00") { - assert_eq!(ErrorSeverity::Info, code.severity()); + assert_eq!(PgErrorSeverity::Info, code.severity()); } else if name.starts_with("Ec01") { - assert_eq!(ErrorSeverity::Warning, code.severity()); + assert_eq!(PgErrorSeverity::Warning, code.severity()); } else if name.starts_with("Ec28") || name.starts_with("Ec3D") { - assert_eq!(ErrorSeverity::Fatal, code.severity()); + assert_eq!(PgErrorSeverity::Fatal, code.severity()); } else { - assert_eq!(ErrorSeverity::Error, code.severity()); + assert_eq!(PgErrorSeverity::Error, code.severity()); } } } diff --git a/src/session/src/context.rs b/src/session/src/context.rs index 70168d9498..f85a8ceea3 100644 --- a/src/session/src/context.rs +++ b/src/session/src/context.rs @@ -40,7 +40,9 @@ pub struct QueryContext { current_catalog: String, // we use Arc> for modifiable fields #[builder(default)] - mutable_inner: Arc>, + mutable_session_data: Arc>, + #[builder(default)] + mutable_query_context_data: Arc>, sql_dialect: Arc, #[builder(default)] extensions: HashMap, @@ -52,6 +54,12 @@ pub struct QueryContext { channel: Channel, } +/// This fields hold data that is only valid to current query context +#[derive(Debug, Builder, Clone, Default)] +pub struct QueryContextMutableFields { + warning: Option, +} + impl Display for QueryContext { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!( @@ -65,21 +73,26 @@ impl Display for QueryContext { impl QueryContextBuilder { pub fn current_schema(mut self, schema: String) -> Self { - if self.mutable_inner.is_none() { - self.mutable_inner = Some(Arc::new(RwLock::new(MutableInner::default()))); + if self.mutable_session_data.is_none() { + self.mutable_session_data = Some(Arc::new(RwLock::new(MutableInner::default()))); } // safe for unwrap because previous none check - self.mutable_inner.as_mut().unwrap().write().unwrap().schema = schema; + self.mutable_session_data + .as_mut() + .unwrap() + .write() + .unwrap() + .schema = schema; self } pub fn timezone(mut self, timezone: Timezone) -> Self { - if self.mutable_inner.is_none() { - self.mutable_inner = Some(Arc::new(RwLock::new(MutableInner::default()))); + if self.mutable_session_data.is_none() { + self.mutable_session_data = Some(Arc::new(RwLock::new(MutableInner::default()))); } - self.mutable_inner + self.mutable_session_data .as_mut() .unwrap() .write() @@ -120,7 +133,7 @@ impl From for api::v1::QueryContext { fn from( QueryContext { current_catalog, - mutable_inner, + mutable_session_data: mutable_inner, extensions, channel, .. @@ -182,11 +195,11 @@ impl QueryContext { } pub fn current_schema(&self) -> String { - self.mutable_inner.read().unwrap().schema.clone() + self.mutable_session_data.read().unwrap().schema.clone() } pub fn set_current_schema(&self, new_schema: &str) { - self.mutable_inner.write().unwrap().schema = new_schema.to_string(); + self.mutable_session_data.write().unwrap().schema = new_schema.to_string(); } pub fn current_catalog(&self) -> &str { @@ -208,19 +221,19 @@ impl QueryContext { } pub fn timezone(&self) -> Timezone { - self.mutable_inner.read().unwrap().timezone.clone() + self.mutable_session_data.read().unwrap().timezone.clone() } pub fn set_timezone(&self, timezone: Timezone) { - self.mutable_inner.write().unwrap().timezone = timezone; + self.mutable_session_data.write().unwrap().timezone = timezone; } pub fn current_user(&self) -> UserInfoRef { - self.mutable_inner.read().unwrap().user_info.clone() + self.mutable_session_data.read().unwrap().user_info.clone() } pub fn set_current_user(&self, user: UserInfoRef) { - self.mutable_inner.write().unwrap().user_info = user; + self.mutable_session_data.write().unwrap().user_info = user; } pub fn set_extension, S2: Into>(&mut self, key: S1, value: S2) { @@ -257,6 +270,18 @@ impl QueryContext { pub fn set_channel(&mut self, channel: Channel) { self.channel = channel; } + + pub fn warning(&self) -> Option { + self.mutable_query_context_data + .read() + .unwrap() + .warning + .clone() + } + + pub fn set_warning(&self, msg: String) { + self.mutable_query_context_data.write().unwrap().warning = Some(msg); + } } impl QueryContextBuilder { @@ -266,7 +291,8 @@ impl QueryContextBuilder { current_catalog: self .current_catalog .unwrap_or_else(|| DEFAULT_CATALOG_NAME.to_string()), - mutable_inner: self.mutable_inner.unwrap_or_default(), + mutable_session_data: self.mutable_session_data.unwrap_or_default(), + mutable_query_context_data: self.mutable_query_context_data.unwrap_or_default(), sql_dialect: self .sql_dialect .unwrap_or_else(|| Arc::new(GreptimeDbDialect {})), diff --git a/src/session/src/lib.rs b/src/session/src/lib.rs index ecfc02f230..33bd140c70 100644 --- a/src/session/src/lib.rs +++ b/src/session/src/lib.rs @@ -76,7 +76,7 @@ impl Session { // catalog is not allowed for update in query context so we use // string here .current_catalog(self.catalog.read().unwrap().clone()) - .mutable_inner(self.mutable_inner.clone()) + .mutable_session_data(self.mutable_inner.clone()) .sql_dialect(self.conn_info.channel.dialect()) .configuration_parameter(self.configuration_variables.clone()) .channel(self.conn_info.channel) From cd4bf239d05f2b9747004c8a7fa3c744c32c2166 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Wed, 25 Sep 2024 10:45:18 +0800 Subject: [PATCH 035/128] chore: relax table name constraint (#4766) chore/relax-table-name-constraint: Updated NAME_PATTERN to allow '@' and '#' characters and adjusted tests for new table name validation rules. --- src/common/meta/src/key.rs | 2 +- src/operator/src/statement/ddl.rs | 8 +++++++- .../cases/standalone/common/create/create.result | 16 ++++++++++++++++ tests/cases/standalone/common/create/create.sql | 8 ++++++++ 4 files changed, 32 insertions(+), 2 deletions(-) diff --git a/src/common/meta/src/key.rs b/src/common/meta/src/key.rs index 3cf6f3a3b7..d864882da6 100644 --- a/src/common/meta/src/key.rs +++ b/src/common/meta/src/key.rs @@ -144,7 +144,7 @@ use crate::rpc::router::{region_distribution, RegionRoute, RegionStatus}; use crate::rpc::store::BatchDeleteRequest; use crate::DatanodeId; -pub const NAME_PATTERN: &str = r"[a-zA-Z_:-][a-zA-Z0-9_:\-\.]*"; +pub const NAME_PATTERN: &str = r"[a-zA-Z_:-][a-zA-Z0-9_:\-\.@#]*"; pub const MAINTENANCE_KEY: &str = "__maintenance"; const DATANODE_TABLE_KEY_PREFIX: &str = "__dn_table"; diff --git a/src/operator/src/statement/ddl.rs b/src/operator/src/statement/ddl.rs index e9b186bd63..af0f6883d8 100644 --- a/src/operator/src/statement/ddl.rs +++ b/src/operator/src/statement/ddl.rs @@ -229,7 +229,7 @@ impl StatementExecutor { ensure!( NAME_PATTERN_REG.is_match(&create_table.table_name), InvalidTableNameSnafu { - table_name: create_table.table_name.clone(), + table_name: &create_table.table_name, } ); @@ -1516,6 +1516,12 @@ mod test { assert!(!NAME_PATTERN_REG.is_match("/adaf")); assert!(!NAME_PATTERN_REG.is_match("🈲")); assert!(NAME_PATTERN_REG.is_match("hello")); + assert!(NAME_PATTERN_REG.is_match("test@")); + assert!(!NAME_PATTERN_REG.is_match("@test")); + assert!(NAME_PATTERN_REG.is_match("test#")); + assert!(!NAME_PATTERN_REG.is_match("#test")); + assert!(!NAME_PATTERN_REG.is_match("@")); + assert!(!NAME_PATTERN_REG.is_match("#")); } #[tokio::test] diff --git a/tests/cases/standalone/common/create/create.result b/tests/cases/standalone/common/create/create.result index 86a89483e8..d01ac4a764 100644 --- a/tests/cases/standalone/common/create/create.result +++ b/tests/cases/standalone/common/create/create.result @@ -54,6 +54,22 @@ CREATE TABLE 'N.~' (i TIMESTAMP TIME INDEX); Error: 1004(InvalidArguments), Invalid table name: N.~ +CREATE TABLE `p_perftest001@cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA` (i INTEGER, j TIMESTAMP TIME INDEX); + +Affected Rows: 0 + +DROP TABLE `p_perftest001@cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA`; + +Affected Rows: 0 + +CREATE TABLE `p_perftest001#cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA` (i INTEGER, j TIMESTAMP TIME INDEX); + +Affected Rows: 0 + +DROP TABLE `p_perftest001#cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA`; + +Affected Rows: 0 + CREATE TABLE neg_default_value_min(i TIMESTAMP TIME INDEX, j SMALLINT DEFAULT -32768); Affected Rows: 0 diff --git a/tests/cases/standalone/common/create/create.sql b/tests/cases/standalone/common/create/create.sql index f5ae083318..62237019ad 100644 --- a/tests/cases/standalone/common/create/create.sql +++ b/tests/cases/standalone/common/create/create.sql @@ -26,6 +26,14 @@ CREATE TABLE test2 (i INTEGER, j TIMESTAMP TIME INDEX); CREATE TABLE 'N.~' (i TIMESTAMP TIME INDEX); +CREATE TABLE `p_perftest001@cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA` (i INTEGER, j TIMESTAMP TIME INDEX); + +DROP TABLE `p_perftest001@cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA`; + +CREATE TABLE `p_perftest001#cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA` (i INTEGER, j TIMESTAMP TIME INDEX); + +DROP TABLE `p_perftest001#cc3kvQ_D1D9H9GOZGMIY97TWH20R1LRC8U0SFBA`; + CREATE TABLE neg_default_value_min(i TIMESTAMP TIME INDEX, j SMALLINT DEFAULT -32768); DESC TABLE neg_default_value_min; From 0274e752aeb1332fbd0a8b51882f7b9da0c73502 Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Thu, 26 Sep 2024 01:11:37 +0800 Subject: [PATCH 036/128] chore: make sure aws-lc-sys wouldn't be built (#4767) * chore: make sure aws-lc-sys wouldn't be built * fix: bash * refactor: multiple line bash --- .github/workflows/develop.yml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/.github/workflows/develop.yml b/.github/workflows/develop.yml index 5100b89754..073c407486 100644 --- a/.github/workflows/develop.yml +++ b/.github/workflows/develop.yml @@ -269,6 +269,13 @@ jobs: - name: Install cargo-gc-bin shell: bash run: cargo install cargo-gc-bin + - name: Check aws-lc-sys will not build + shell: bash + run: | + if cargo tree -i aws-lc-sys -e features | grep -q aws-lc-sys; then + echo "Found aws-lc-sys, which has compilation problems on older gcc versions. Please replace it with ring until its building experience improves." + exit 1 + fi - name: Build greptime bianry shell: bash # `cargo gc` will invoke `cargo build` with specified args From 627a3262735eaa490cee8dadf413b61a04948d16 Mon Sep 17 00:00:00 2001 From: localhost Date: Thu, 26 Sep 2024 03:32:34 +0800 Subject: [PATCH 037/128] refactor: Change the error type in the pipeline crate from String to Error (#4763) * chore: in process * chore: change pipeline crate error type * chore: improve event error * chore: fix by pr comment * chore: use snafu context replace ok_or_else * refactor: update snafu usage --------- Co-authored-by: Ning Sun --- src/pipeline/benches/processor.rs | 8 +- src/pipeline/src/etl.rs | 30 +- src/pipeline/src/etl/error.rs | 552 ++++++++++++++++++ src/pipeline/src/etl/field.rs | 14 +- src/pipeline/src/etl/processor.rs | 71 ++- src/pipeline/src/etl/processor/cmcd.rs | 69 ++- src/pipeline/src/etl/processor/csv.rs | 62 +- src/pipeline/src/etl/processor/date.rs | 63 +- src/pipeline/src/etl/processor/dissect.rs | 222 ++++--- src/pipeline/src/etl/processor/epoch.rs | 45 +- src/pipeline/src/etl/processor/gsub.rs | 65 ++- src/pipeline/src/etl/processor/join.rs | 47 +- src/pipeline/src/etl/processor/letter.rs | 44 +- src/pipeline/src/etl/processor/regex.rs | 79 +-- src/pipeline/src/etl/processor/timestamp.rs | 93 +-- src/pipeline/src/etl/processor/urlencoding.rs | 46 +- src/pipeline/src/etl/transform.rs | 45 +- src/pipeline/src/etl/transform/index.rs | 12 +- .../src/etl/transform/transformer/greptime.rs | 35 +- .../transform/transformer/greptime/coerce.rs | 80 +-- src/pipeline/src/etl/value.rs | 75 +-- src/pipeline/src/lib.rs | 1 + src/pipeline/src/manager/error.rs | 16 +- src/pipeline/src/manager/table.rs | 3 +- src/pipeline/tests/dissect.rs | 2 +- src/servers/src/http/event.rs | 37 +- 26 files changed, 1275 insertions(+), 541 deletions(-) create mode 100644 src/pipeline/src/etl/error.rs diff --git a/src/pipeline/benches/processor.rs b/src/pipeline/benches/processor.rs index 281d8ce0ef..09462753d8 100644 --- a/src/pipeline/benches/processor.rs +++ b/src/pipeline/benches/processor.rs @@ -13,13 +13,13 @@ // limitations under the License. use criterion::{black_box, criterion_group, criterion_main, Criterion}; -use pipeline::{parse, Content, GreptimeTransformer, Pipeline}; +use pipeline::{parse, Content, GreptimeTransformer, Pipeline, Result}; use serde_json::{Deserializer, Value}; fn processor_mut( pipeline: &Pipeline, input_values: Vec, -) -> Result, String> { +) -> Result> { let mut payload = pipeline.init_intermediate_state(); let mut result = Vec::with_capacity(input_values.len()); @@ -30,7 +30,7 @@ fn processor_mut( pipeline.reset_intermediate_state(&mut payload); } - Ok::, String>(result) + Ok(result) } fn prepare_pipeline() -> Pipeline { @@ -230,7 +230,7 @@ fn criterion_benchmark(c: &mut Criterion) { let input_value_str = include_str!("./data.log"); let input_value = Deserializer::from_str(input_value_str) .into_iter::() - .collect::, _>>() + .collect::, _>>() .unwrap(); let pipeline = prepare_pipeline(); let mut group = c.benchmark_group("pipeline"); diff --git a/src/pipeline/src/etl.rs b/src/pipeline/src/etl.rs index f6b3efd6e6..f29032e4f8 100644 --- a/src/pipeline/src/etl.rs +++ b/src/pipeline/src/etl.rs @@ -14,6 +14,7 @@ #![allow(dead_code)] +pub mod error; pub mod field; pub mod processor; pub mod transform; @@ -21,12 +22,16 @@ pub mod value; use ahash::HashSet; use common_telemetry::debug; +use error::{IntermediateKeyIndexSnafu, PrepareValueMustBeObjectSnafu, YamlLoadSnafu}; use itertools::Itertools; use processor::{Processor, ProcessorBuilder, Processors}; +use snafu::{OptionExt, ResultExt}; use transform::{TransformBuilders, Transformer, Transforms}; use value::Value; use yaml_rust::YamlLoader; +use crate::etl::error::Result; + const DESCRIPTION: &str = "description"; const PROCESSORS: &str = "processors"; const TRANSFORM: &str = "transform"; @@ -37,13 +42,13 @@ pub enum Content { Yaml(String), } -pub fn parse(input: &Content) -> Result, String> +pub fn parse(input: &Content) -> Result> where T: Transformer, { match input { Content::Yaml(str) => { - let docs = YamlLoader::load_from_str(str).map_err(|e| e.to_string())?; + let docs = YamlLoader::load_from_str(str).context(YamlLoadSnafu)?; let doc = &docs[0]; @@ -124,7 +129,7 @@ where .processor_builders .into_iter() .map(|builder| builder.build(&final_intermediate_keys)) - .collect::, _>>()?; + .collect::>>()?; let processors = Processors { processors: processors_kind_list, required_keys: processors_required_keys.clone(), @@ -136,7 +141,7 @@ where .builders .into_iter() .map(|builder| builder.build(&final_intermediate_keys, &output_keys)) - .collect::, String>>()?; + .collect::>>()?; let transformers = Transforms { transforms: transfor_list, @@ -197,7 +202,7 @@ impl Pipeline where T: Transformer, { - pub fn exec_mut(&self, val: &mut Vec) -> Result { + pub fn exec_mut(&self, val: &mut Vec) -> Result { for processor in self.processors.iter() { processor.exec_mut(val)?; } @@ -205,7 +210,7 @@ where self.transformer.transform_mut(val) } - pub fn prepare(&self, val: serde_json::Value, result: &mut [Value]) -> Result<(), String> { + pub fn prepare(&self, val: serde_json::Value, result: &mut [Value]) -> Result<()> { match val { serde_json::Value::Object(map) => { let mut search_from = 0; @@ -230,7 +235,7 @@ where result[0] = val.try_into()?; } _ => { - return Err("expect object".to_string()); + return PrepareValueMustBeObjectSnafu.fail(); } } Ok(()) @@ -274,18 +279,11 @@ where } } -pub(crate) fn find_key_index( - intermediate_keys: &[String], - key: &str, - kind: &str, -) -> Result { +pub(crate) fn find_key_index(intermediate_keys: &[String], key: &str, kind: &str) -> Result { intermediate_keys .iter() .position(|k| k == key) - .ok_or(format!( - "{} processor.{} not found in intermediate keys", - kind, key - )) + .context(IntermediateKeyIndexSnafu { kind, key }) } #[cfg(test)] diff --git a/src/pipeline/src/etl/error.rs b/src/pipeline/src/etl/error.rs new file mode 100644 index 0000000000..3680053ba0 --- /dev/null +++ b/src/pipeline/src/etl/error.rs @@ -0,0 +1,552 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::any::Any; + +use common_error::ext::ErrorExt; +use common_error::status_code::StatusCode; +use common_macro::stack_trace_debug; +use snafu::{Location, Snafu}; + +#[derive(Snafu)] +#[snafu(visibility(pub))] +#[stack_trace_debug] +pub enum Error { + #[snafu(display("Empty input field"))] + EmptyInputField { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Missing input field"))] + MissingInputField { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Processor must be a map"))] + ProcessorMustBeMap { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("{processor} processor: missing field: {field}"))] + ProcessorMissingField { + processor: String, + field: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("{processor} processor: expect string value, but got {v:?}"))] + ProcessorExpectString { + processor: String, + v: crate::etl::Value, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("{processor} processor: unsupported value {val}"))] + ProcessorUnsupportedValue { + processor: &'static str, + val: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("processor key must be a string"))] + ProcessorKeyMustBeString { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("{kind} processor: failed to parse {value}"))] + ProcessorFailedToParseString { + kind: String, + value: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("processor must have a string key"))] + ProcessorMustHaveStringKey { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("unsupported {processor} processor"))] + UnsupportedProcessor { + processor: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Field {field} must be a {ty}"))] + FieldMustBeType { + field: String, + ty: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Field parse from string failed: {field}"))] + FailedParseFieldFromString { + #[snafu(source)] + error: Box, + field: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("failed to parse {key} as int: {value}"))] + FailedToParseIntKey { + key: String, + value: String, + #[snafu(source)] + error: std::num::ParseIntError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to parse {value} to int"))] + FailedToParseInt { + value: String, + #[snafu(source)] + error: std::num::ParseIntError, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("failed to parse {key} as float: {value}"))] + FailedToParseFloatKey { + key: String, + value: String, + #[snafu(source)] + error: std::num::ParseFloatError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("{kind} processor.{key} not found in intermediate keys"))] + IntermediateKeyIndex { + kind: String, + key: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("{k} missing value in {s}"))] + CmcdMissingValue { + k: String, + s: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("{part} missing key in {s}"))] + CmcdMissingKey { + part: String, + s: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("key must be a string, but got {k:?}"))] + KeyMustBeString { + k: yaml_rust::Yaml, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("csv read error"))] + CsvRead { + #[snafu(implicit)] + location: Location, + #[snafu(source)] + error: csv::Error, + }, + #[snafu(display("expected at least one record from csv format, but got none"))] + CsvNoRecord { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("'{separator}' must be a single character, but got '{value}'"))] + CsvSeparatorName { + separator: &'static str, + value: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("'{quote}' must be a single character, but got '{value}'"))] + CsvQuoteName { + quote: &'static str, + value: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Parse date timezone error {value}"))] + DateParseTimezone { + value: String, + #[snafu(source)] + error: chrono_tz::ParseError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Parse date error {value}"))] + DateParse { + value: String, + #[snafu(source)] + error: chrono::ParseError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("failed to get local timezone"))] + DateFailedToGetLocalTimezone { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("failed to get timestamp"))] + DateFailedToGetTimestamp { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("{processor} processor: invalid format {s}"))] + DateInvalidFormat { + s: String, + processor: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Invalid Pattern: '{s}'. {detail}"))] + DissectInvalidPattern { + s: String, + detail: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Empty pattern is not allowed"))] + DissectEmptyPattern { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("'{split}' exceeds the input"))] + DissectSplitExceedsInput { + split: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("'{split}' does not match the input '{input}'"))] + DissectSplitNotMatchInput { + split: String, + input: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("consecutive names are not allowed: '{name1}' '{name2}'"))] + DissectConsecutiveNames { + name1: String, + name2: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("No matching pattern found"))] + DissectNoMatchingPattern { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("'{m}' modifier already set, but found {modifier}"))] + DissectModifierAlreadySet { + m: String, + modifier: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Append Order modifier is already set to '{n}', cannot be set to {order}"))] + DissectAppendOrderAlreadySet { + n: String, + order: u32, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Order can only be set to Append Modifier, current modifier is {m}"))] + DissectOrderOnlyAppend { + m: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Order can only be set to Append Modifier"))] + DissectOrderOnlyAppendModifier { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("End modifier already set: '{m}'"))] + DissectEndModifierAlreadySet { + m: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("invalid resolution: {resolution}"))] + EpochInvalidResolution { + resolution: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("pattern is required"))] + GsubPatternRequired { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("replacement is required"))] + GsubReplacementRequired { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("invalid regex pattern: {pattern}"))] + Regex { + #[snafu(source)] + error: regex::Error, + pattern: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("separator is required"))] + JoinSeparatorRequired { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("invalid method: {method}"))] + LetterInvalidMethod { + method: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("no named group found in regex {origin}"))] + RegexNamedGroupNotFound { + origin: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("no valid field found in {processor} processor"))] + RegexNoValidField { + processor: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("no valid pattern found in {processor} processor"))] + RegexNoValidPattern { + processor: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("invalid method: {s}"))] + UrlEncodingInvalidMethod { + s: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("url decoding error"))] + UrlEncodingDecode { + #[snafu(source)] + error: std::string::FromUtf8Error, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("invalid transform on_failure value: {value}"))] + TransformOnFailureInvalidValue { + value: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("transform element must be a map"))] + TransformElementMustBeMap { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("transform {fields:?} type MUST BE set before default {default}"))] + TransformTypeMustBeSet { + fields: String, + default: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("transform cannot be empty"))] + TransformEmpty { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("column name must be unique, but got duplicated: {duplicates}"))] + TransformColumnNameMustBeUnique { + duplicates: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display( + "Illegal to set multiple timestamp Index columns, please set only one: {columns}" + ))] + TransformMultipleTimestampIndex { + columns: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("transform must have exactly one field specified as timestamp Index, but got {count}: {columns}"))] + TransformTimestampIndexCount { + count: usize, + columns: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Null type not supported"))] + CoerceUnsupportedNullType { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Null type not supported when to coerce '{ty}' type"))] + CoerceUnsupportedNullTypeTo { + ty: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("{ty} value not supported for Epoch"))] + CoerceUnsupportedEpochType { + ty: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("failed to coerce string value '{s}' to type '{ty}'"))] + CoerceStringToType { + s: String, + ty: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display( + "invalid resolution: '{resolution}'. Available resolutions: {valid_resolution}" + ))] + ValueInvalidResolution { + resolution: String, + valid_resolution: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("failed to parse type: '{t}'"))] + ValueParseType { + t: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("failed to parse {ty}: {v}"))] + ValueParseInt { + ty: String, + v: String, + #[snafu(source)] + error: std::num::ParseIntError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("failed to parse {ty}: {v}"))] + ValueParseFloat { + ty: String, + v: String, + #[snafu(source)] + error: std::num::ParseFloatError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("failed to parse {ty}: {v}"))] + ValueParseBoolean { + ty: String, + v: String, + #[snafu(source)] + error: std::str::ParseBoolError, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("default value not unsupported for type {value}"))] + ValueDefaultValueUnsupported { + value: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("unsupported number type: {value}"))] + ValueUnsupportedNumberType { + value: serde_json::Number, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("unsupported yaml type: {value:?}"))] + ValueUnsupportedYamlType { + value: yaml_rust::Yaml, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("key in Hash must be a string, but got {value:?}"))] + ValueYamlKeyMustBeString { + value: yaml_rust::Yaml, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Yaml load error."))] + YamlLoad { + #[snafu(source)] + error: yaml_rust::ScanError, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Prepare value must be an object"))] + PrepareValueMustBeObject { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Column options error"))] + ColumnOptions { + #[snafu(source)] + source: api::error::Error, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("unsupported index type: {value}"))] + UnsupportedIndexType { + value: String, + #[snafu(implicit)] + location: Location, + }, +} + +pub type Result = std::result::Result; + +impl ErrorExt for Error { + fn status_code(&self) -> StatusCode { + StatusCode::InvalidArguments + } + + fn as_any(&self) -> &dyn Any { + self + } +} diff --git a/src/pipeline/src/etl/field.rs b/src/pipeline/src/etl/field.rs index ff2f1ee7b5..84d94040b5 100644 --- a/src/pipeline/src/etl/field.rs +++ b/src/pipeline/src/etl/field.rs @@ -15,6 +15,10 @@ use std::ops::Deref; use std::str::FromStr; +use snafu::OptionExt; + +use super::error::{EmptyInputFieldSnafu, MissingInputFieldSnafu}; +use crate::etl::error::{Error, Result}; use crate::etl::find_key_index; /// Information about the input field including the name and index in intermediate keys. @@ -56,7 +60,7 @@ impl OneInputOneOutputField { intermediate_keys: &[String], input_field: &str, target_field: &str, - ) -> Result { + ) -> Result { let input_index = find_key_index(intermediate_keys, input_field, processor_kind)?; let input_field_info = InputFieldInfo::new(input_field, input_index); @@ -145,19 +149,19 @@ pub struct Field { } impl FromStr for Field { - type Err = String; + type Err = Error; - fn from_str(s: &str) -> Result { + fn from_str(s: &str) -> Result { let mut parts = s.split(','); let input_field = parts .next() - .ok_or("input field is missing")? + .context(MissingInputFieldSnafu)? .trim() .to_string(); let target_field = parts.next().map(|x| x.trim().to_string()); if input_field.is_empty() { - return Err("input field is empty".to_string()); + return EmptyInputFieldSnafu.fail(); } Ok(Field { diff --git a/src/pipeline/src/etl/processor.rs b/src/pipeline/src/etl/processor.rs index 257cce4dfc..afda8f399e 100644 --- a/src/pipeline/src/etl/processor.rs +++ b/src/pipeline/src/etl/processor.rs @@ -36,10 +36,16 @@ use itertools::Itertools; use join::{JoinProcessor, JoinProcessorBuilder}; use letter::{LetterProcessor, LetterProcessorBuilder}; use regex::{RegexProcessor, RegexProcessorBuilder}; +use snafu::{OptionExt, ResultExt}; use timestamp::{TimestampProcessor, TimestampProcessorBuilder}; use urlencoding::{UrlEncodingProcessor, UrlEncodingProcessorBuilder}; +use super::error::{ + FailedParseFieldFromStringSnafu, FieldMustBeTypeSnafu, ProcessorKeyMustBeStringSnafu, + ProcessorMustBeMapSnafu, ProcessorMustHaveStringKeySnafu, UnsupportedProcessorSnafu, +}; use super::field::{Field, Fields}; +use crate::etl::error::{Error, Result}; use crate::etl::value::Value; const FIELD_NAME: &str = "field"; @@ -70,7 +76,7 @@ pub trait Processor: std::fmt::Debug + Send + Sync + 'static { fn ignore_missing(&self) -> bool; /// Execute the processor on a vector which be preprocessed by the pipeline - fn exec_mut(&self, val: &mut Vec) -> Result<(), String>; + fn exec_mut(&self, val: &mut Vec) -> Result<()>; } #[derive(Debug)] @@ -98,7 +104,7 @@ pub trait ProcessorBuilder: std::fmt::Debug + Send + Sync + 'static { /// Get the processor's input keys fn input_keys(&self) -> HashSet<&str>; /// Build the processor - fn build(self, intermediate_keys: &[String]) -> Result; + fn build(self, intermediate_keys: &[String]) -> Result; } #[derive(Debug)] @@ -171,9 +177,9 @@ impl Processors { } impl TryFrom<&Vec> for ProcessorBuilderList { - type Error = String; + type Error = Error; - fn try_from(vec: &Vec) -> Result { + fn try_from(vec: &Vec) -> Result { let mut processors_builders = vec![]; let mut all_output_keys = HashSet::with_capacity(50); let mut all_required_keys = HashSet::with_capacity(50); @@ -226,13 +232,10 @@ impl TryFrom<&Vec> for ProcessorBuilderList { } } -fn parse_processor(doc: &yaml_rust::Yaml) -> Result { - let map = doc.as_hash().ok_or("processor must be a map".to_string())?; +fn parse_processor(doc: &yaml_rust::Yaml) -> Result { + let map = doc.as_hash().context(ProcessorMustBeMapSnafu)?; - let key = map - .keys() - .next() - .ok_or("processor must have a string key".to_string())?; + let key = map.keys().next().context(ProcessorMustHaveStringKeySnafu)?; let value = map .get(key) @@ -240,9 +243,7 @@ fn parse_processor(doc: &yaml_rust::Yaml) -> Result { .as_hash() .expect("processor value must be a map"); - let str_key = key - .as_str() - .ok_or("processor key must be a string".to_string())?; + let str_key = key.as_str().context(ProcessorKeyMustBeStringSnafu)?; let processor = match str_key { cmcd::PROCESSOR_CMCD => ProcessorBuilders::Cmcd(CmcdProcessorBuilder::try_from(value)?), @@ -264,58 +265,72 @@ fn parse_processor(doc: &yaml_rust::Yaml) -> Result { urlencoding::PROCESSOR_URL_ENCODING => { ProcessorBuilders::UrlEncoding(UrlEncodingProcessorBuilder::try_from(value)?) } - _ => return Err(format!("unsupported {} processor", str_key)), + _ => return UnsupportedProcessorSnafu { processor: str_key }.fail(), }; Ok(processor) } -pub(crate) fn yaml_string(v: &yaml_rust::Yaml, field: &str) -> Result { +pub(crate) fn yaml_string(v: &yaml_rust::Yaml, field: &str) -> Result { v.as_str() .map(|s| s.to_string()) - .ok_or(format!("'{field}' must be a string")) + .context(FieldMustBeTypeSnafu { + field, + ty: "string", + }) } -pub(crate) fn yaml_strings(v: &yaml_rust::Yaml, field: &str) -> Result, String> { +pub(crate) fn yaml_strings(v: &yaml_rust::Yaml, field: &str) -> Result> { let vec = v .as_vec() - .ok_or(format!("'{field}' must be a list of strings",))? + .context(FieldMustBeTypeSnafu { + field, + ty: "list of string", + })? .iter() .map(|v| v.as_str().unwrap_or_default().into()) .collect(); Ok(vec) } -pub(crate) fn yaml_bool(v: &yaml_rust::Yaml, field: &str) -> Result { - v.as_bool().ok_or(format!("'{field}' must be a boolean")) +pub(crate) fn yaml_bool(v: &yaml_rust::Yaml, field: &str) -> Result { + v.as_bool().context(FieldMustBeTypeSnafu { + field, + ty: "boolean", + }) } -pub(crate) fn yaml_parse_string(v: &yaml_rust::Yaml, field: &str) -> Result +pub(crate) fn yaml_parse_string(v: &yaml_rust::Yaml, field: &str) -> Result where T: std::str::FromStr, - T::Err: ToString, + T::Err: std::error::Error + Send + Sync + 'static, { yaml_string(v, field)? .parse::() - .map_err(|e| e.to_string()) + .map_err(|e| Box::new(e) as Box) + .context(FailedParseFieldFromStringSnafu { field }) } -pub(crate) fn yaml_parse_strings(v: &yaml_rust::Yaml, field: &str) -> Result, String> +pub(crate) fn yaml_parse_strings(v: &yaml_rust::Yaml, field: &str) -> Result> where T: std::str::FromStr, - T::Err: ToString, + T::Err: std::error::Error + Send + Sync + 'static, { yaml_strings(v, field).and_then(|v| { v.into_iter() - .map(|s| s.parse::().map_err(|e| e.to_string())) + .map(|s| { + s.parse::() + .map_err(|e| Box::new(e) as Box) + .context(FailedParseFieldFromStringSnafu { field }) + }) .collect() }) } -pub(crate) fn yaml_new_fields(v: &yaml_rust::Yaml, field: &str) -> Result { +pub(crate) fn yaml_new_fields(v: &yaml_rust::Yaml, field: &str) -> Result { yaml_parse_strings(v, field).map(Fields::new) } -pub(crate) fn yaml_new_field(v: &yaml_rust::Yaml, field: &str) -> Result { +pub(crate) fn yaml_new_field(v: &yaml_rust::Yaml, field: &str) -> Result { yaml_parse_string(v, field) } diff --git a/src/pipeline/src/etl/processor/cmcd.rs b/src/pipeline/src/etl/processor/cmcd.rs index 1556829d65..f43186b94a 100644 --- a/src/pipeline/src/etl/processor/cmcd.rs +++ b/src/pipeline/src/etl/processor/cmcd.rs @@ -15,8 +15,14 @@ use std::collections::BTreeMap; use ahash::HashSet; +use snafu::{OptionExt, ResultExt}; use urlencoding::decode; +use crate::etl::error::{ + CmcdMissingKeySnafu, CmcdMissingValueSnafu, Error, FailedToParseFloatKeySnafu, + FailedToParseIntKeySnafu, KeyMustBeStringSnafu, ProcessorExpectStringSnafu, + ProcessorMissingFieldSnafu, Result, +}; use crate::etl::field::{Field, Fields, InputFieldInfo, OneInputMultiOutputField}; use crate::etl::find_key_index; use crate::etl::processor::{ @@ -82,7 +88,7 @@ impl CmcdProcessorBuilder { pub(super) fn build_cmcd_outputs( field: &Field, intermediate_keys: &[String], - ) -> Result<(BTreeMap, Vec), String> { + ) -> Result<(BTreeMap, Vec)> { let mut output_index = BTreeMap::new(); let mut cmcd_field_outputs = Vec::with_capacity(CMCD_KEYS.len()); for cmcd in CMCD_KEYS { @@ -119,7 +125,7 @@ impl CmcdProcessorBuilder { } /// build CmcdProcessor from CmcdProcessorBuilder - pub fn build(self, intermediate_keys: &[String]) -> Result { + pub fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; let mut cmcd_outputs = Vec::with_capacity(CMCD_KEYS.len()); for field in self.fields.into_iter() { @@ -151,7 +157,7 @@ impl ProcessorBuilder for CmcdProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Cmcd) } } @@ -170,7 +176,7 @@ pub(super) struct CmcdOutputInfo { /// index in intermediate_keys index: usize, /// function to resolve value - f: fn(&str, &str, Option<&str>) -> Result, + f: fn(&str, &str, Option<&str>) -> Result, } impl CmcdOutputInfo { @@ -178,7 +184,7 @@ impl CmcdOutputInfo { final_key: String, key: &'static str, index: usize, - f: fn(&str, &str, Option<&str>) -> Result, + f: fn(&str, &str, Option<&str>) -> Result, ) -> Self { Self { final_key, @@ -201,28 +207,28 @@ impl Default for CmcdOutputInfo { } /// function to resolve CMCD_KEY_BS | CMCD_KEY_SU -fn bs_su(_: &str, _: &str, _: Option<&str>) -> Result { +fn bs_su(_: &str, _: &str, _: Option<&str>) -> Result { Ok(Value::Boolean(true)) } /// function to resolve CMCD_KEY_BR | CMCD_KEY_BL | CMCD_KEY_D | CMCD_KEY_DL | CMCD_KEY_MTP | CMCD_KEY_RTP | CMCD_KEY_TB -fn br_tb(s: &str, k: &str, v: Option<&str>) -> Result { - let v = v.ok_or(format!("{k} missing value in {s}"))?; +fn br_tb(s: &str, k: &str, v: Option<&str>) -> Result { + let v = v.context(CmcdMissingValueSnafu { k, s })?; let val: i64 = v .parse() - .map_err(|_| format!("failed to parse {v} as i64"))?; + .context(FailedToParseIntKeySnafu { key: k, value: v })?; Ok(Value::Int64(val)) } /// function to resolve CMCD_KEY_CID | CMCD_KEY_NRR | CMCD_KEY_OT | CMCD_KEY_SF | CMCD_KEY_SID | CMCD_KEY_V -fn cid_v(s: &str, k: &str, v: Option<&str>) -> Result { - let v = v.ok_or(format!("{k} missing value in {s}"))?; +fn cid_v(s: &str, k: &str, v: Option<&str>) -> Result { + let v = v.context(CmcdMissingValueSnafu { k, s })?; Ok(Value::String(v.to_string())) } /// function to resolve CMCD_KEY_NOR -fn nor(s: &str, k: &str, v: Option<&str>) -> Result { - let v = v.ok_or(format!("{k} missing value in {s}"))?; +fn nor(s: &str, k: &str, v: Option<&str>) -> Result { + let v = v.context(CmcdMissingValueSnafu { k, s })?; let val = match decode(v) { Ok(val) => val.to_string(), Err(_) => v.to_string(), @@ -231,11 +237,11 @@ fn nor(s: &str, k: &str, v: Option<&str>) -> Result { } /// function to resolve CMCD_KEY_PR -fn pr(s: &str, k: &str, v: Option<&str>) -> Result { - let v = v.ok_or(format!("{k} missing value in {s}"))?; +fn pr(s: &str, k: &str, v: Option<&str>) -> Result { + let v = v.context(CmcdMissingValueSnafu { k, s })?; let val: f64 = v .parse() - .map_err(|_| format!("failed to parse {v} as f64"))?; + .context(FailedToParseFloatKeySnafu { key: k, value: v })?; Ok(Value::Float64(val)) } @@ -287,12 +293,12 @@ impl CmcdProcessor { format!("{}_{}", prefix, key) } - fn parse(&self, field_index: usize, s: &str) -> Result, String> { + fn parse(&self, field_index: usize, s: &str) -> Result> { let parts = s.split(','); let mut result = Vec::new(); for part in parts { let mut kv = part.split('='); - let k = kv.next().ok_or(format!("{part} missing key in {s}"))?; + let k = kv.next().context(CmcdMissingKeySnafu { part, s })?; let v = kv.next(); for cmcd_key in self.cmcd_outputs[field_index].iter() { @@ -308,16 +314,16 @@ impl CmcdProcessor { } impl TryFrom<&yaml_rust::yaml::Hash> for CmcdProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut ignore_missing = false; for (k, v) in value.iter() { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { fields = Fields::one(yaml_new_field(v, FIELD_NAME)?); @@ -362,7 +368,7 @@ impl Processor for CmcdProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for (field_index, field) in self.fields.iter().enumerate() { let field_value_index = field.input_index(); match val.get(field_value_index) { @@ -374,18 +380,19 @@ impl Processor for CmcdProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind().to_string(), + field: field.input_name().to_string(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind().to_string(), + v: v.clone(), + } + .fail(); } } } diff --git a/src/pipeline/src/etl/processor/csv.rs b/src/pipeline/src/etl/processor/csv.rs index fb1fca2bfb..c9cb5f847d 100644 --- a/src/pipeline/src/etl/processor/csv.rs +++ b/src/pipeline/src/etl/processor/csv.rs @@ -18,7 +18,12 @@ use ahash::HashSet; use csv::{ReaderBuilder, Trim}; use itertools::EitherOrBoth::{Both, Left, Right}; use itertools::Itertools; +use snafu::{OptionExt, ResultExt}; +use crate::etl::error::{ + CsvNoRecordSnafu, CsvQuoteNameSnafu, CsvReadSnafu, CsvSeparatorNameSnafu, Error, + KeyMustBeStringSnafu, ProcessorExpectStringSnafu, ProcessorMissingFieldSnafu, Result, +}; use crate::etl::field::{Fields, InputFieldInfo, OneInputMultiOutputField}; use crate::etl::find_key_index; use crate::etl::processor::{ @@ -53,7 +58,7 @@ pub struct CsvProcessorBuilder { } impl CsvProcessorBuilder { - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields { @@ -68,7 +73,7 @@ impl CsvProcessorBuilder { .target_fields .iter() .map(|f| find_key_index(intermediate_keys, f, "csv")) - .collect::, String>>()?; + .collect::>>()?; Ok(CsvProcessor { reader: self.reader, fields: real_fields, @@ -88,7 +93,7 @@ impl ProcessorBuilder for CsvProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Csv) } } @@ -114,11 +119,11 @@ pub struct CsvProcessor { impl CsvProcessor { // process the csv format string to a map with target_fields as keys - fn process(&self, val: &str) -> Result, String> { + fn process(&self, val: &str) -> Result> { let mut reader = self.reader.from_reader(val.as_bytes()); if let Some(result) = reader.records().next() { - let record: csv::StringRecord = result.map_err(|e| e.to_string())?; + let record: csv::StringRecord = result.context(CsvReadSnafu)?; let values: Vec<(usize, Value)> = self .output_index_info @@ -142,15 +147,15 @@ impl CsvProcessor { Ok(values) } else { - Err("expected at least one record from csv format, but got none".into()) + CsvNoRecordSnafu.fail() } } } impl TryFrom<&yaml_rust::yaml::Hash> for CsvProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { let mut reader = ReaderBuilder::new(); reader.has_headers(false); @@ -162,7 +167,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for CsvProcessorBuilder { for (k, v) in hash { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { fields = Fields::one(yaml_new_field(v, FIELD_NAME)?); @@ -180,10 +185,11 @@ impl TryFrom<&yaml_rust::yaml::Hash> for CsvProcessorBuilder { SEPARATOR_NAME => { let separator = yaml_string(v, SEPARATOR_NAME)?; if separator.len() != 1 { - return Err(format!( - "'{}' must be a single character, but got '{}'", - SEPARATOR_NAME, separator - )); + return CsvSeparatorNameSnafu { + separator: SEPARATOR_NAME, + value: separator, + } + .fail(); } else { reader.delimiter(separator.as_bytes()[0]); } @@ -191,10 +197,11 @@ impl TryFrom<&yaml_rust::yaml::Hash> for CsvProcessorBuilder { QUOTE_NAME => { let quote = yaml_string(v, QUOTE_NAME)?; if quote.len() != 1 { - return Err(format!( - "'{}' must be a single character, but got '{}'", - QUOTE_NAME, quote - )); + return CsvQuoteNameSnafu { + quote: QUOTE_NAME, + value: quote, + } + .fail(); } else { reader.quote(quote.as_bytes()[0]); } @@ -240,7 +247,7 @@ impl Processor for CsvProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { @@ -252,18 +259,19 @@ impl Processor for CsvProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind().to_string(), + field: field.input_name().to_string(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind().to_string(), + v: v.clone(), + } + .fail(); } } } diff --git a/src/pipeline/src/etl/processor/date.rs b/src/pipeline/src/etl/processor/date.rs index b9bfcf3b6c..fa202a0edf 100644 --- a/src/pipeline/src/etl/processor/date.rs +++ b/src/pipeline/src/etl/processor/date.rs @@ -18,7 +18,13 @@ use ahash::HashSet; use chrono::{DateTime, NaiveDateTime}; use chrono_tz::Tz; use lazy_static::lazy_static; +use snafu::{OptionExt, ResultExt}; +use crate::etl::error::{ + DateFailedToGetLocalTimezoneSnafu, DateFailedToGetTimestampSnafu, DateParseSnafu, + DateParseTimezoneSnafu, Error, KeyMustBeStringSnafu, ProcessorExpectStringSnafu, + ProcessorFailedToParseStringSnafu, ProcessorMissingFieldSnafu, Result, +}; use crate::etl::field::{Fields, OneInputOneOutputField}; use crate::etl::processor::{ yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, yaml_strings, Processor, @@ -103,13 +109,13 @@ impl ProcessorBuilder for DateProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Date) } } impl DateProcessorBuilder { - pub fn build(self, intermediate_keys: &[String]) -> Result { + pub fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input = OneInputOneOutputField::build( @@ -131,9 +137,9 @@ impl DateProcessorBuilder { } impl TryFrom<&yaml_rust::yaml::Hash> for DateProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut formats = Formats::default(); let mut timezone = None; @@ -143,7 +149,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for DateProcessorBuilder { for (k, v) in hash { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { @@ -205,10 +211,12 @@ pub struct DateProcessor { } impl DateProcessor { - fn parse(&self, val: &str) -> Result { + fn parse(&self, val: &str) -> Result { let mut tz = Tz::UTC; if let Some(timezone) = &self.timezone { - tz = timezone.parse::().map_err(|e| e.to_string())?; + tz = timezone.parse::().context(DateParseTimezoneSnafu { + value: timezone.as_ref(), + })?; } for fmt in self.formats.iter() { @@ -217,7 +225,11 @@ impl DateProcessor { } } - Err(format!("{} processor: failed to parse {val}", self.kind(),)) + ProcessorFailedToParseStringSnafu { + kind: PROCESSOR_DATE.to_string(), + value: val.to_string(), + } + .fail() } } @@ -230,7 +242,7 @@ impl Processor for DateProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { @@ -241,18 +253,19 @@ impl Processor for DateProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind().to_string(), + field: field.input_name().to_string(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind().to_string(), + v: v.clone(), + } + .fail(); } } } @@ -261,16 +274,20 @@ impl Processor for DateProcessor { } /// try to parse val with timezone first, if failed, parse without timezone -fn try_parse(val: &str, fmt: &str, tz: Tz) -> Result { +fn try_parse(val: &str, fmt: &str, tz: Tz) -> Result { if let Ok(dt) = DateTime::parse_from_str(val, fmt) { - Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + Ok(dt + .timestamp_nanos_opt() + .context(DateFailedToGetTimestampSnafu)?) } else { let dt = NaiveDateTime::parse_from_str(val, fmt) - .map_err(|e| e.to_string())? + .context(DateParseSnafu { value: val })? .and_local_timezone(tz) .single() - .ok_or("failed to get local timezone")?; - Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + .context(DateFailedToGetLocalTimezoneSnafu)?; + Ok(dt + .timestamp_nanos_opt() + .context(DateFailedToGetTimestampSnafu)?) } } diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs index f9925916fc..09c6fc93d0 100644 --- a/src/pipeline/src/etl/processor/dissect.rs +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -16,7 +16,15 @@ use std::ops::Deref; use ahash::{HashMap, HashMapExt, HashSet, HashSetExt}; use itertools::Itertools; +use snafu::OptionExt; +use crate::etl::error::{ + DissectAppendOrderAlreadySetSnafu, DissectConsecutiveNamesSnafu, DissectEmptyPatternSnafu, + DissectEndModifierAlreadySetSnafu, DissectInvalidPatternSnafu, DissectModifierAlreadySetSnafu, + DissectNoMatchingPatternSnafu, DissectOrderOnlyAppendModifierSnafu, + DissectOrderOnlyAppendSnafu, DissectSplitExceedsInputSnafu, DissectSplitNotMatchInputSnafu, + Error, KeyMustBeStringSnafu, ProcessorExpectStringSnafu, ProcessorMissingFieldSnafu, Result, +}; use crate::etl::field::{Fields, InputFieldInfo, OneInputMultiOutputField}; use crate::etl::find_key_index; use crate::etl::processor::{ @@ -77,9 +85,13 @@ impl NameInfo { self.name.is_empty() && self.start_modifier.is_none() && self.end_modifier.is_none() } - fn try_start_modifier(&mut self, modifier: StartModifier) -> Result<(), String> { + fn try_start_modifier(&mut self, modifier: StartModifier) -> Result<()> { match &self.start_modifier { - Some(m) => Err(format!("'{m}' modifier already set, but found {modifier}",)), + Some(m) => DissectModifierAlreadySetSnafu { + m: m.to_string(), + modifier: modifier.to_string(), + } + .fail(), None => { self.start_modifier = Some(modifier); Ok(()) @@ -87,27 +99,27 @@ impl NameInfo { } } - fn try_append_order(&mut self, order: u32) -> Result<(), String> { + fn try_append_order(&mut self, order: u32) -> Result<()> { match &mut self.start_modifier { Some(StartModifier::Append(o)) => match o { - Some(n) => Err(format!( - "Append Order modifier is already set to '{n}', cannot be set to {order}" - )), + Some(n) => DissectAppendOrderAlreadySetSnafu { + n: n.to_string(), + order, + } + .fail(), None => { *o = Some(order); Ok(()) } }, - Some(m) => Err(format!( - "Order can only be set to Append Modifier, current modifier is {m}" - )), - None => Err("Order can only be set to Append Modifier".to_string()), + Some(m) => DissectOrderOnlyAppendSnafu { m: m.to_string() }.fail(), + None => DissectOrderOnlyAppendModifierSnafu.fail(), } } - fn try_end_modifier(&mut self) -> Result<(), String> { + fn try_end_modifier(&mut self) -> Result<()> { match &self.end_modifier { - Some(m) => Err(format!("End modifier already set: '{m}'")), + Some(m) => DissectEndModifierAlreadySetSnafu { m: m.to_string() }.fail(), None => { self.end_modifier = Some(EndModifier); Ok(()) @@ -290,9 +302,9 @@ impl std::ops::DerefMut for PatternInfo { } impl std::str::FromStr for PatternInfo { - type Err = String; + type Err = Error; - fn from_str(s: &str) -> Result { + fn from_str(s: &str) -> Result { let mut parts = vec![]; let mut cursor = PartInfo::empty_split(); @@ -338,9 +350,11 @@ impl std::str::FromStr for PatternInfo { } if j == pos + 1 { - return Err(format!( - "Invalid Pattern: '{s}'. Digit order must be set after '/'", - )); + return DissectInvalidPatternSnafu { + s, + detail: "Digit order must be set after '/'", + } + .fail(); } name.try_append_order(order)?; @@ -358,14 +372,20 @@ impl std::str::FromStr for PatternInfo { ('-', PartInfo::Name(name)) if !name.is_end_modifier_set() => { if let Some('>') = chars.get(pos + 1) { } else { - return Err(format!( - "Invalid Pattern: '{s}'. expected '->' but only '-'", - )); + return DissectInvalidPatternSnafu { + s, + detail: "Expected '->' but only '-'", + } + .fail(); } if let Some('}') = chars.get(pos + 2) { } else { - return Err(format!("Invalid Pattern: '{s}'. expected '}}' after '->'",)); + return DissectInvalidPatternSnafu { + s, + detail: "Expected '}' after '->'", + } + .fail(); } name.try_end_modifier()?; @@ -377,7 +397,7 @@ impl std::str::FromStr for PatternInfo { } else { format!("Invalid '{ch}' in '{name}'") }; - return Err(format!("Invalid Pattern: '{s}'. {tail}")); + return DissectInvalidPatternSnafu { s, detail: tail }.fail(); } (_, PartInfo::Name(_)) => { cursor.push(ch); @@ -390,7 +410,11 @@ impl std::str::FromStr for PatternInfo { match cursor { PartInfo::Split(ref split) if !split.is_empty() => parts.push(cursor), PartInfo::Name(name) if !name.is_empty() => { - return Err(format!("Invalid Pattern: '{s}'. '{name}' is not closed")) + return DissectInvalidPatternSnafu { + s, + detail: format!("'{name}' is not closed"), + } + .fail(); } _ => {} } @@ -402,9 +426,9 @@ impl std::str::FromStr for PatternInfo { } impl PatternInfo { - fn check(&self) -> Result<(), String> { + fn check(&self) -> Result<()> { if self.len() == 0 { - return Err("Empty pattern is not allowed".to_string()); + return DissectEmptyPatternSnafu.fail(); } let mut map_keys = HashSet::new(); @@ -415,42 +439,47 @@ impl PatternInfo { let next_part = self.get(i + 1); match (this_part, next_part) { (PartInfo::Split(split), _) if split.is_empty() => { - return Err(format!( - "Invalid Pattern: '{}'. Empty split is not allowed", - self.origin - )); + return DissectInvalidPatternSnafu { + s: &self.origin, + detail: "Empty split is not allowed", + } + .fail(); } (PartInfo::Name(name1), Some(PartInfo::Name(name2))) => { - return Err(format!( - "Invalid Pattern: '{}'. consecutive names are not allowed: '{}' '{}'", - self.origin, name1, name2 - )); + return DissectInvalidPatternSnafu { + s: &self.origin, + detail: format!("consecutive names are not allowed: '{name1}' '{name2}'",), + } + .fail(); } (PartInfo::Name(name), _) if name.is_name_empty() => { if let Some(ref m) = name.start_modifier { - return Err(format!( - "Invalid Pattern: '{}'. only '{}' modifier is invalid", - self.origin, m - )); + return DissectInvalidPatternSnafu { + s: &self.origin, + detail: format!("only '{m}' modifier is invalid"), + } + .fail(); } } (PartInfo::Name(name), _) => match name.start_modifier { Some(StartModifier::MapKey) => { if map_keys.contains(&name.name) { - return Err(format!( - "Invalid Pattern: '{}'. Duplicate map key: '{}'", - self.origin, name.name - )); + return DissectInvalidPatternSnafu { + s: &self.origin, + detail: format!("Duplicate map key: '{}'", name.name), + } + .fail(); } else { map_keys.insert(&name.name); } } Some(StartModifier::MapVal) => { if map_vals.contains(&name.name) { - return Err(format!( - "Invalid Pattern: '{}'. Duplicate map val: '{}'", - self.origin, name.name - )); + return DissectInvalidPatternSnafu { + s: &self.origin, + detail: format!("Duplicate map val: '{}'", name.name), + } + .fail(); } else { map_vals.insert(&name.name); } @@ -462,15 +491,18 @@ impl PatternInfo { } if map_keys != map_vals { - return Err(format!( - "Invalid Pattern: '{}'. key and value not matched: '{}'", - self.origin, - map_keys - .symmetric_difference(&map_vals) - .map(|s| s.as_str()) - .collect::>() - .join(",") - )); + return DissectInvalidPatternSnafu { + s: &self.origin, + detail: format!( + "key and value not matched: '{}'", + map_keys + .symmetric_difference(&map_vals) + .map(|s| s.as_str()) + .collect::>() + .join(",") + ), + } + .fail(); } Ok(()) @@ -516,10 +548,7 @@ impl DissectProcessorBuilder { .collect() } - fn part_info_to_part( - part_info: PartInfo, - intermediate_keys: &[String], - ) -> Result { + fn part_info_to_part(part_info: PartInfo, intermediate_keys: &[String]) -> Result { match part_info { PartInfo::Split(s) => Ok(Part::Split(s)), PartInfo::Name(n) => match n.start_modifier { @@ -545,13 +574,13 @@ impl DissectProcessorBuilder { fn pattern_info_to_pattern( pattern_info: PatternInfo, intermediate_keys: &[String], - ) -> Result { + ) -> Result { let original = pattern_info.origin; let pattern = pattern_info .parts .into_iter() .map(|part_info| Self::part_info_to_part(part_info, intermediate_keys)) - .collect::, String>>()?; + .collect::>>()?; Ok(Pattern { origin: original, parts: pattern, @@ -561,7 +590,7 @@ impl DissectProcessorBuilder { fn build_patterns_from_pattern_infos( patterns: Vec, intermediate_keys: &[String], - ) -> Result, String> { + ) -> Result> { patterns .into_iter() .map(|pattern_info| Self::pattern_info_to_pattern(pattern_info, intermediate_keys)) @@ -578,7 +607,7 @@ impl ProcessorBuilder for DissectProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input_index = find_key_index(intermediate_keys, field.input_field(), "dissect")?; @@ -610,11 +639,7 @@ pub struct DissectProcessor { } impl DissectProcessor { - fn process_pattern( - &self, - chs: &[char], - pattern: &Pattern, - ) -> Result, String> { + fn process_pattern(&self, chs: &[char], pattern: &Pattern) -> Result> { let mut map = Vec::new(); let mut pos = 0; @@ -668,23 +693,26 @@ impl DissectProcessor { let split_chs = split.chars().collect::>(); let split_len = split_chs.len(); if pos + split_len > chs.len() { - return Err(format!("'{split}' exceeds the input",)); + return DissectSplitExceedsInputSnafu { split }.fail(); } if &chs[pos..pos + split_len] != split_chs.as_slice() { - return Err(format!( - "'{split}' does not match the input '{}'", - chs[pos..pos + split_len].iter().collect::() - )); + return DissectSplitNotMatchInputSnafu { + split, + input: chs[pos..pos + split_len].iter().collect::(), + } + .fail(); } pos += split_len; } (Part::Name(name1), Some(Part::Name(name2))) => { - return Err(format!( - "consecutive names are not allowed: '{name1}' '{name2}'" - )); + return DissectConsecutiveNamesSnafu { + name1: name1.to_string(), + name2: name2.to_string(), + } + .fail(); } // if Name part is the last part, then the rest of the input is the value @@ -695,10 +723,10 @@ impl DissectProcessor { // if Name part, and next part is Split, then find the matched value of the name (Part::Name(name), Some(Part::Split(split))) => { - let stop = split - .chars() - .next() - .ok_or("Empty split is not allowed".to_string())?; // this won't happen + let stop = split.chars().next().context(DissectInvalidPatternSnafu { + s: &pattern.origin, + detail: "Empty split is not allowed", + })?; // this won't happen let mut end = pos; while end < chs.len() && chs[end] != stop { end += 1; @@ -737,7 +765,7 @@ impl DissectProcessor { Ok(map) } - fn process(&self, val: &str) -> Result, String> { + fn process(&self, val: &str) -> Result> { let chs = val.chars().collect::>(); for pattern in &self.patterns { @@ -745,15 +773,14 @@ impl DissectProcessor { return Ok(map); } } - - Err("No matching pattern found".to_string()) + DissectNoMatchingPatternSnafu.fail() } } impl TryFrom<&yaml_rust::yaml::Hash> for DissectProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut patterns = vec![]; let mut ignore_missing = false; @@ -762,7 +789,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for DissectProcessorBuilder { for (k, v) in value.iter() { let key = k .as_str() - .ok_or(format!("key must be a string, but got '{k:?}'"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { @@ -809,7 +836,7 @@ impl Processor for DissectProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { @@ -821,18 +848,19 @@ impl Processor for DissectProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind(), + v: v.clone(), + } + .fail(); } } } @@ -1123,7 +1151,7 @@ mod tests { ), ( "%{->clientip} ", - "Invalid Pattern: '%{->clientip} '. expected '}' after '->'", + "Invalid Pattern: '%{->clientip} '. Expected '}' after '->'", ), ( "%{/clientip} ", @@ -1185,7 +1213,7 @@ mod tests { for (pattern, expected) in cases.into_iter() { let err = pattern.parse::().unwrap_err(); - assert_eq!(err, expected); + assert_eq!(err.to_string(), expected); } } diff --git a/src/pipeline/src/etl/processor/epoch.rs b/src/pipeline/src/etl/processor/epoch.rs index 32c7d61786..f2c03fd120 100644 --- a/src/pipeline/src/etl/processor/epoch.rs +++ b/src/pipeline/src/etl/processor/epoch.rs @@ -13,7 +13,12 @@ // limitations under the License. use ahash::HashSet; +use snafu::{OptionExt, ResultExt}; +use crate::etl::error::{ + EpochInvalidResolutionSnafu, Error, FailedToParseIntSnafu, KeyMustBeStringSnafu, + ProcessorMissingFieldSnafu, ProcessorUnsupportedValueSnafu, Result, +}; use crate::etl::field::{Fields, OneInputOneOutputField}; use crate::etl::processor::{ yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, Processor, ProcessorBuilder, @@ -39,15 +44,15 @@ enum Resolution { } impl TryFrom<&str> for Resolution { - type Error = String; + type Error = Error; - fn try_from(s: &str) -> Result { + fn try_from(s: &str) -> Result { match s { SECOND_RESOLUTION | SEC_RESOLUTION | S_RESOLUTION => Ok(Resolution::Second), MILLISECOND_RESOLUTION | MILLI_RESOLUTION | MS_RESOLUTION => Ok(Resolution::Milli), MICROSECOND_RESOLUTION | MICRO_RESOLUTION | US_RESOLUTION => Ok(Resolution::Micro), NANOSECOND_RESOLUTION | NANO_RESOLUTION | NS_RESOLUTION => Ok(Resolution::Nano), - _ => Err(format!("invalid resolution: {s}")), + _ => EpochInvalidResolutionSnafu { resolution: s }.fail(), } } } @@ -71,13 +76,13 @@ impl ProcessorBuilder for EpochProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Epoch) } } impl EpochProcessorBuilder { - pub fn build(self, intermediate_keys: &[String]) -> Result { + pub fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input = OneInputOneOutputField::build( @@ -112,11 +117,11 @@ pub struct EpochProcessor { } impl EpochProcessor { - fn parse(&self, val: &Value) -> Result { + fn parse(&self, val: &Value) -> Result { let t: i64 = match val { Value::String(s) => s .parse::() - .map_err(|e| format!("Failed to parse {} to number: {}", s, e))?, + .context(FailedToParseIntSnafu { value: s })?, Value::Int16(i) => *i as i64, Value::Int32(i) => *i as i64, Value::Int64(i) => *i, @@ -135,9 +140,11 @@ impl EpochProcessor { }, _ => { - return Err(format!( - "{PROCESSOR_EPOCH} processor: unsupported value {val}" - )) + return ProcessorUnsupportedValueSnafu { + processor: PROCESSOR_EPOCH, + val: val.to_string(), + } + .fail(); } }; @@ -151,9 +158,9 @@ impl EpochProcessor { } impl TryFrom<&yaml_rust::yaml::Hash> for EpochProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut resolution = Resolution::default(); let mut ignore_missing = false; @@ -161,7 +168,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for EpochProcessorBuilder { for (k, v) in hash { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { @@ -200,17 +207,17 @@ impl Processor for EpochProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { diff --git a/src/pipeline/src/etl/processor/gsub.rs b/src/pipeline/src/etl/processor/gsub.rs index 1b8e581e6a..b5a328c6fa 100644 --- a/src/pipeline/src/etl/processor/gsub.rs +++ b/src/pipeline/src/etl/processor/gsub.rs @@ -14,11 +14,16 @@ use ahash::HashSet; use regex::Regex; +use snafu::{OptionExt, ResultExt}; +use crate::etl::error::{ + Error, GsubPatternRequiredSnafu, GsubReplacementRequiredSnafu, KeyMustBeStringSnafu, + ProcessorExpectStringSnafu, ProcessorMissingFieldSnafu, RegexSnafu, Result, +}; use crate::etl::field::{Fields, OneInputOneOutputField}; use crate::etl::processor::{ - yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, Processor, ProcessorBuilder, - ProcessorKind, FIELDS_NAME, FIELD_NAME, IGNORE_MISSING_NAME, PATTERN_NAME, + yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, ProcessorBuilder, ProcessorKind, + FIELDS_NAME, FIELD_NAME, IGNORE_MISSING_NAME, PATTERN_NAME, }; use crate::etl::value::Value; @@ -46,25 +51,25 @@ impl ProcessorBuilder for GsubProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Gsub) } } impl GsubProcessorBuilder { - fn check(self) -> Result { + fn check(self) -> Result { if self.pattern.is_none() { - return Err("pattern is required".to_string()); + return GsubPatternRequiredSnafu.fail(); } if self.replacement.is_none() { - return Err("replacement is required".to_string()); + return GsubReplacementRequiredSnafu.fail(); } Ok(self) } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input = OneInputOneOutputField::build( @@ -94,19 +99,19 @@ pub struct GsubProcessor { } impl GsubProcessor { - fn check(self) -> Result { + fn check(self) -> Result { if self.pattern.is_none() { - return Err("pattern is required".to_string()); + return GsubPatternRequiredSnafu.fail(); } if self.replacement.is_none() { - return Err("replacement is required".to_string()); + return GsubReplacementRequiredSnafu.fail(); } Ok(self) } - fn process_string(&self, val: &str) -> Result { + fn process_string(&self, val: &str) -> Result { let replacement = self.replacement.as_ref().unwrap(); let new_val = self .pattern @@ -119,21 +124,26 @@ impl GsubProcessor { Ok(val) } - fn process(&self, val: &Value) -> Result { + fn process(&self, val: &Value) -> Result { match val { Value::String(val) => self.process_string(val), - _ => Err(format!( - "{} processor: expect string or array string, but got {val:?}", - self.kind() - )), + _ => ProcessorExpectStringSnafu { + processor: PROCESSOR_GSUB, + v: val.clone(), + } + .fail(), + // Err(format!( + // "{} processor: expect string or array string, but got {val:?}", + // self.kind() + // )), } } } impl TryFrom<&yaml_rust::yaml::Hash> for GsubProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut ignore_missing = false; let mut pattern = None; @@ -142,7 +152,8 @@ impl TryFrom<&yaml_rust::yaml::Hash> for GsubProcessorBuilder { for (k, v) in value.iter() { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; + match key { FIELD_NAME => { fields = Fields::one(yaml_new_field(v, FIELD_NAME)?); @@ -152,7 +163,9 @@ impl TryFrom<&yaml_rust::yaml::Hash> for GsubProcessorBuilder { } PATTERN_NAME => { let pattern_str = yaml_string(v, PATTERN_NAME)?; - pattern = Some(Regex::new(&pattern_str).map_err(|e| e.to_string())?); + pattern = Some(Regex::new(&pattern_str).context(RegexSnafu { + pattern: pattern_str, + })?); } REPLACEMENT_NAME => { let replacement_str = yaml_string(v, REPLACEMENT_NAME)?; @@ -187,17 +200,17 @@ impl crate::etl::processor::Processor for GsubProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { diff --git a/src/pipeline/src/etl/processor/join.rs b/src/pipeline/src/etl/processor/join.rs index d4b309d5c2..ddbc086ab8 100644 --- a/src/pipeline/src/etl/processor/join.rs +++ b/src/pipeline/src/etl/processor/join.rs @@ -13,7 +13,12 @@ // limitations under the License. use ahash::HashSet; +use snafu::OptionExt; +use crate::etl::error::{ + Error, JoinSeparatorRequiredSnafu, KeyMustBeStringSnafu, ProcessorExpectStringSnafu, + ProcessorMissingFieldSnafu, Result, +}; use crate::etl::field::{Fields, OneInputOneOutputField}; use crate::etl::processor::{ yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, Processor, ProcessorBuilder, @@ -42,21 +47,21 @@ impl ProcessorBuilder for JoinProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Join) } } impl JoinProcessorBuilder { - fn check(self) -> Result { + fn check(self) -> Result { if self.separator.is_none() { - return Err("separator is required".to_string()); + return JoinSeparatorRequiredSnafu.fail(); } Ok(self) } - pub fn build(self, intermediate_keys: &[String]) -> Result { + pub fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input = OneInputOneOutputField::build( @@ -85,7 +90,7 @@ pub struct JoinProcessor { } impl JoinProcessor { - fn process(&self, arr: &Array) -> Result { + fn process(&self, arr: &Array) -> Result { let sep = self.separator.as_ref().unwrap(); let val = arr .iter() @@ -96,9 +101,9 @@ impl JoinProcessor { Ok(Value::String(val)) } - fn check(self) -> Result { + fn check(self) -> Result { if self.separator.is_none() { - return Err("separator is required".to_string()); + return JoinSeparatorRequiredSnafu.fail(); } Ok(self) @@ -106,9 +111,9 @@ impl JoinProcessor { } impl TryFrom<&yaml_rust::yaml::Hash> for JoinProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut separator = None; let mut ignore_missing = false; @@ -116,7 +121,8 @@ impl TryFrom<&yaml_rust::yaml::Hash> for JoinProcessorBuilder { for (k, v) in value.iter() { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; + match key { FIELD_NAME => { fields = Fields::one(yaml_new_field(v, FIELD_NAME)?); @@ -152,7 +158,7 @@ impl Processor for JoinProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { @@ -163,18 +169,19 @@ impl Processor for JoinProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind(), + v: v.clone(), + } + .fail(); } } } diff --git a/src/pipeline/src/etl/processor/letter.rs b/src/pipeline/src/etl/processor/letter.rs index f388b5a2a9..8eb9399181 100644 --- a/src/pipeline/src/etl/processor/letter.rs +++ b/src/pipeline/src/etl/processor/letter.rs @@ -13,7 +13,12 @@ // limitations under the License. use ahash::HashSet; +use snafu::OptionExt; +use crate::etl::error::{ + Error, KeyMustBeStringSnafu, LetterInvalidMethodSnafu, ProcessorExpectStringSnafu, + ProcessorMissingFieldSnafu, Result, +}; use crate::etl::field::{Fields, OneInputOneOutputField}; use crate::etl::processor::{ yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, Processor, ProcessorBuilder, @@ -42,14 +47,14 @@ impl std::fmt::Display for Method { } impl std::str::FromStr for Method { - type Err = String; + type Err = Error; - fn from_str(s: &str) -> Result { + fn from_str(s: &str) -> Result { match s.to_lowercase().as_str() { "upper" => Ok(Method::Upper), "lower" => Ok(Method::Lower), "capital" => Ok(Method::Capital), - _ => Err(format!("invalid method: {s}")), + _ => LetterInvalidMethodSnafu { method: s }.fail(), } } } @@ -73,13 +78,13 @@ impl ProcessorBuilder for LetterProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Letter) } } impl LetterProcessorBuilder { - pub fn build(self, intermediate_keys: &[String]) -> Result { + pub fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input = OneInputOneOutputField::build( @@ -108,7 +113,7 @@ pub struct LetterProcessor { } impl LetterProcessor { - fn process_field(&self, val: &str) -> Result { + fn process_field(&self, val: &str) -> Result { let processed = match self.method { Method::Upper => val.to_uppercase(), Method::Lower => val.to_lowercase(), @@ -121,9 +126,9 @@ impl LetterProcessor { } impl TryFrom<&yaml_rust::yaml::Hash> for LetterProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut method = Method::Lower; let mut ignore_missing = false; @@ -131,7 +136,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for LetterProcessorBuilder { for (k, v) in value.iter() { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { fields = Fields::one(yaml_new_field(v, FIELD_NAME)?); @@ -166,7 +171,7 @@ impl Processor for LetterProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { @@ -177,18 +182,19 @@ impl Processor for LetterProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - &field.input().name - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind(), + v: v.clone(), + } + .fail(); } } } diff --git a/src/pipeline/src/etl/processor/regex.rs b/src/pipeline/src/etl/processor/regex.rs index a1de2ea76d..a74c19140c 100644 --- a/src/pipeline/src/etl/processor/regex.rs +++ b/src/pipeline/src/etl/processor/regex.rs @@ -21,7 +21,13 @@ pub(crate) const PROCESSOR_REGEX: &str = "regex"; use ahash::{HashSet, HashSetExt}; use lazy_static::lazy_static; use regex::Regex; +use snafu::{OptionExt, ResultExt}; +use crate::etl::error::{ + Error, KeyMustBeStringSnafu, ProcessorExpectStringSnafu, ProcessorMissingFieldSnafu, + RegexNamedGroupNotFoundSnafu, RegexNoValidFieldSnafu, RegexNoValidPatternSnafu, RegexSnafu, + Result, +}; use crate::etl::field::{Fields, InputFieldInfo, OneInputMultiOutputField}; use crate::etl::find_key_index; use crate::etl::processor::{ @@ -60,15 +66,15 @@ impl std::fmt::Display for GroupRegex { } impl std::str::FromStr for GroupRegex { - type Err = String; + type Err = Error; - fn from_str(origin: &str) -> Result { + fn from_str(origin: &str) -> Result { let groups = get_regex_group_names(origin); if groups.is_empty() { - return Err(format!("no named group found in regex {origin}")); + return RegexNamedGroupNotFoundSnafu { origin }.fail(); } - let regex = Regex::new(origin).map_err(|e| e.to_string())?; + let regex = Regex::new(origin).context(RegexSnafu { pattern: origin })?; Ok(GroupRegex { origin: origin.into(), regex, @@ -94,25 +100,25 @@ impl ProcessorBuilder for RegexProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Regex) } } impl RegexProcessorBuilder { - fn check(self) -> Result { + fn check(self) -> Result { if self.fields.is_empty() { - return Err(format!( - "no valid field found in {} processor", - PROCESSOR_REGEX - )); + return RegexNoValidFieldSnafu { + processor: PROCESSOR_REGEX, + } + .fail(); } if self.patterns.is_empty() { - return Err(format!( - "no valid pattern found in {} processor", - PROCESSOR_REGEX - )); + return RegexNoValidPatternSnafu { + processor: PROCESSOR_REGEX, + } + .fail(); } Ok(self) @@ -122,7 +128,7 @@ impl RegexProcessorBuilder { group_regex: &GroupRegex, om_field: &OneInputMultiOutputField, intermediate_keys: &[String], - ) -> Result, String> { + ) -> Result> { group_regex .groups .iter() @@ -135,35 +141,35 @@ impl RegexProcessorBuilder { index, }) }) - .collect::, String>>() + .collect::>>() } fn build_group_output_infos( patterns: &[GroupRegex], om_field: &OneInputMultiOutputField, intermediate_keys: &[String], - ) -> Result>, String> { + ) -> Result>> { patterns .iter() .map(|group_regex| { Self::build_group_output_info(group_regex, om_field, intermediate_keys) }) - .collect::, String>>() + .collect::>>() } fn build_output_info( real_fields: &[OneInputMultiOutputField], patterns: &[GroupRegex], intermediate_keys: &[String], - ) -> Result { + ) -> Result { let inner = real_fields .iter() .map(|om_field| Self::build_group_output_infos(patterns, om_field, intermediate_keys)) - .collect::, String>>(); + .collect::>>(); inner.map(|inner| RegexProcessorOutputInfo { inner }) } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input_index = find_key_index(intermediate_keys, field.input_field(), "regex")?; @@ -184,9 +190,9 @@ impl RegexProcessorBuilder { } impl TryFrom<&yaml_rust::yaml::Hash> for RegexProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut patterns: Vec = vec![]; let mut ignore_missing = false; @@ -194,7 +200,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for RegexProcessorBuilder { for (k, v) in value.iter() { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { fields = Fields::one(yaml_new_field(v, FIELD_NAME)?); @@ -275,7 +281,7 @@ pub struct RegexProcessor { } impl RegexProcessor { - fn try_with_patterns(&mut self, patterns: Vec) -> Result<(), String> { + fn try_with_patterns(&mut self, patterns: Vec) -> Result<()> { let mut rs = vec![]; for pattern in patterns { let gr = pattern.parse()?; @@ -290,7 +296,7 @@ impl RegexProcessor { val: &str, gr: &GroupRegex, index: (usize, usize), - ) -> Result, String> { + ) -> Result> { let mut result = Vec::new(); if let Some(captures) = gr.regex.captures(val) { for (group_index, group) in gr.groups.iter().enumerate() { @@ -316,7 +322,7 @@ impl Processor for RegexProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for (field_index, field) in self.fields.iter().enumerate() { let index = field.input_index(); let mut result_list = None; @@ -346,18 +352,19 @@ impl Processor for RegexProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.input_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind(), + v: v.clone(), + } + .fail(); } } // safety here diff --git a/src/pipeline/src/etl/processor/timestamp.rs b/src/pipeline/src/etl/processor/timestamp.rs index 7ab9571101..3f9621c19d 100644 --- a/src/pipeline/src/etl/processor/timestamp.rs +++ b/src/pipeline/src/etl/processor/timestamp.rs @@ -18,7 +18,14 @@ use ahash::HashSet; use chrono::{DateTime, NaiveDateTime}; use chrono_tz::Tz; use lazy_static::lazy_static; +use snafu::{OptionExt, ResultExt}; +use crate::etl::error::{ + DateFailedToGetLocalTimezoneSnafu, DateFailedToGetTimestampSnafu, DateInvalidFormatSnafu, + DateParseSnafu, DateParseTimezoneSnafu, EpochInvalidResolutionSnafu, Error, + KeyMustBeStringSnafu, ProcessorFailedToParseStringSnafu, ProcessorMissingFieldSnafu, + ProcessorUnsupportedValueSnafu, Result, +}; use crate::etl::field::{Fields, OneInputOneOutputField}; use crate::etl::processor::{ yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, yaml_strings, Processor, @@ -69,15 +76,15 @@ enum Resolution { } impl TryFrom<&str> for Resolution { - type Error = String; + type Error = Error; - fn try_from(s: &str) -> Result { + fn try_from(s: &str) -> Result { match s { SECOND_RESOLUTION | SEC_RESOLUTION | S_RESOLUTION => Ok(Resolution::Second), MILLISECOND_RESOLUTION | MILLI_RESOLUTION | MS_RESOLUTION => Ok(Resolution::Milli), MICROSECOND_RESOLUTION | MICRO_RESOLUTION | US_RESOLUTION => Ok(Resolution::Micro), NANOSECOND_RESOLUTION | NANO_RESOLUTION | NS_RESOLUTION => Ok(Resolution::Nano), - _ => Err(format!("invalid resolution: {s}")), + _ => EpochInvalidResolutionSnafu { resolution: s }.fail(), } } } @@ -127,13 +134,13 @@ impl ProcessorBuilder for TimestampProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys).map(ProcessorKind::Timestamp) } } impl TimestampProcessorBuilder { - pub fn build(self, intermediate_keys: &[String]) -> Result { + pub fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input = OneInputOneOutputField::build( @@ -169,29 +176,37 @@ pub struct TimestampProcessor { impl TimestampProcessor { /// try to parse val with timezone first, if failed, parse without timezone - fn try_parse(val: &str, fmt: &str, tz: Tz) -> Result { + fn try_parse(val: &str, fmt: &str, tz: Tz) -> Result { if let Ok(dt) = DateTime::parse_from_str(val, fmt) { - Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + Ok(dt + .timestamp_nanos_opt() + .context(DateFailedToGetTimestampSnafu)?) } else { let dt = NaiveDateTime::parse_from_str(val, fmt) - .map_err(|e| e.to_string())? + .context(DateParseSnafu { value: val })? .and_local_timezone(tz) .single() - .ok_or("failed to get local timezone")?; - Ok(dt.timestamp_nanos_opt().ok_or("failed to get timestamp")?) + .context(DateFailedToGetLocalTimezoneSnafu)?; + Ok(dt + .timestamp_nanos_opt() + .context(DateFailedToGetTimestampSnafu)?) } } - fn parse_time_str(&self, val: &str) -> Result { + fn parse_time_str(&self, val: &str) -> Result { for (fmt, tz) in self.formats.iter() { if let Ok(ns) = Self::try_parse(val, fmt, *tz) { return Ok(ns); } } - Err(format!("{} processor: failed to parse {val}", self.kind(),)) + ProcessorFailedToParseStringSnafu { + kind: PROCESSOR_TIMESTAMP, + value: val.to_string(), + } + .fail() } - fn parse(&self, val: &Value) -> Result { + fn parse(&self, val: &Value) -> Result { let t: i64 = match val { Value::String(s) => { let t = s.parse::(); @@ -221,9 +236,11 @@ impl TimestampProcessor { }, _ => { - return Err(format!( - "{PROCESSOR_TIMESTAMP} processor: unsupported value {val}" - )) + return ProcessorUnsupportedValueSnafu { + processor: PROCESSOR_TIMESTAMP, + val: val.to_string(), + } + .fail(); } }; @@ -236,7 +253,7 @@ impl TimestampProcessor { } } -fn parse_formats(yaml: &yaml_rust::yaml::Yaml) -> Result, Tz)>, String> { +fn parse_formats(yaml: &yaml_rust::yaml::Yaml) -> Result, Tz)>> { return match yaml.as_vec() { Some(formats_yaml) => { let mut formats = Vec::with_capacity(formats_yaml.len()); @@ -244,32 +261,38 @@ fn parse_formats(yaml: &yaml_rust::yaml::Yaml) -> Result, Tz)>, let s = yaml_strings(v, FORMATS_NAME) .or(yaml_string(v, FORMATS_NAME).map(|s| vec![s]))?; if s.len() != 1 && s.len() != 2 { - return Err(format!( - "{PROCESSOR_TIMESTAMP} processor: invalid format {s:?}" - )); + return DateInvalidFormatSnafu { + processor: PROCESSOR_TIMESTAMP, + s: format!("{s:?}"), + } + .fail(); } let mut iter = s.into_iter(); // safety: unwrap is safe here let formatter = iter.next().unwrap(); let tz = iter .next() - .map(|tz| tz.parse::()) - .unwrap_or(Ok(Tz::UTC)) - .map_err(|e| e.to_string())?; + .map(|tz| { + tz.parse::() + .context(DateParseTimezoneSnafu { value: tz }) + }) + .unwrap_or(Ok(Tz::UTC))?; formats.push((Arc::new(formatter), tz)); } Ok(formats) } - None => Err(format!( - "{PROCESSOR_TIMESTAMP} processor: invalid format {yaml:?}" - )), + None => DateInvalidFormatSnafu { + processor: PROCESSOR_TIMESTAMP, + s: format!("{yaml:?}"), + } + .fail(), }; } impl TryFrom<&yaml_rust::yaml::Hash> for TimestampProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut formats = Formats::default(); let mut resolution = Resolution::default(); @@ -278,7 +301,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for TimestampProcessorBuilder { for (k, v) in hash { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { @@ -321,17 +344,17 @@ impl Processor for TimestampProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input().index; match val.get(index) { Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - &field.input().name - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { diff --git a/src/pipeline/src/etl/processor/urlencoding.rs b/src/pipeline/src/etl/processor/urlencoding.rs index 7db9d092f2..ca42aae236 100644 --- a/src/pipeline/src/etl/processor/urlencoding.rs +++ b/src/pipeline/src/etl/processor/urlencoding.rs @@ -13,8 +13,13 @@ // limitations under the License. use ahash::HashSet; +use snafu::{OptionExt, ResultExt}; use urlencoding::{decode, encode}; +use crate::etl::error::{ + Error, KeyMustBeStringSnafu, ProcessorExpectStringSnafu, ProcessorMissingFieldSnafu, Result, + UrlEncodingDecodeSnafu, UrlEncodingInvalidMethodSnafu, +}; use crate::etl::field::{Fields, OneInputOneOutputField}; use crate::etl::processor::{ yaml_bool, yaml_new_field, yaml_new_fields, yaml_string, ProcessorBuilder, ProcessorKind, @@ -41,13 +46,13 @@ impl std::fmt::Display for Method { } impl std::str::FromStr for Method { - type Err = String; + type Err = Error; - fn from_str(s: &str) -> Result { + fn from_str(s: &str) -> Result { match s { "decode" => Ok(Method::Decode), "encode" => Ok(Method::Encode), - _ => Err(format!("invalid method: {s}")), + _ => UrlEncodingInvalidMethodSnafu { s }.fail(), } } } @@ -71,14 +76,14 @@ impl ProcessorBuilder for UrlEncodingProcessorBuilder { self.fields.iter().map(|f| f.input_field()).collect() } - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { self.build(intermediate_keys) .map(ProcessorKind::UrlEncoding) } } impl UrlEncodingProcessorBuilder { - fn build(self, intermediate_keys: &[String]) -> Result { + fn build(self, intermediate_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields.into_iter() { let input = OneInputOneOutputField::build( @@ -106,19 +111,19 @@ pub struct UrlEncodingProcessor { } impl UrlEncodingProcessor { - fn process_field(&self, val: &str) -> Result { + fn process_field(&self, val: &str) -> Result { let processed = match self.method { Method::Encode => encode(val).to_string(), - Method::Decode => decode(val).map_err(|e| e.to_string())?.into_owned(), + Method::Decode => decode(val).context(UrlEncodingDecodeSnafu)?.into_owned(), }; Ok(Value::String(processed)) } } impl TryFrom<&yaml_rust::yaml::Hash> for UrlEncodingProcessorBuilder { - type Error = String; + type Error = Error; - fn try_from(value: &yaml_rust::yaml::Hash) -> Result { + fn try_from(value: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut method = Method::Decode; let mut ignore_missing = false; @@ -126,7 +131,7 @@ impl TryFrom<&yaml_rust::yaml::Hash> for UrlEncodingProcessorBuilder { for (k, v) in value.iter() { let key = k .as_str() - .ok_or(format!("key must be a string, but got {k:?}"))?; + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { FIELD_NAME => { fields = Fields::one(yaml_new_field(v, FIELD_NAME)?); @@ -166,7 +171,7 @@ impl crate::etl::processor::Processor for UrlEncodingProcessor { self.ignore_missing } - fn exec_mut(&self, val: &mut Vec) -> Result<(), String> { + fn exec_mut(&self, val: &mut Vec) -> Result<()> { for field in self.fields.iter() { let index = field.input_index(); match val.get(index) { @@ -177,18 +182,19 @@ impl crate::etl::processor::Processor for UrlEncodingProcessor { } Some(Value::Null) | None => { if !self.ignore_missing { - return Err(format!( - "{} processor: missing field: {}", - self.kind(), - field.output_name() - )); + return ProcessorMissingFieldSnafu { + processor: self.kind(), + field: field.input_name(), + } + .fail(); } } Some(v) => { - return Err(format!( - "{} processor: expect string value, but got {v:?}", - self.kind() - )); + return ProcessorExpectStringSnafu { + processor: self.kind(), + v: v.clone(), + } + .fail(); } } } diff --git a/src/pipeline/src/etl/transform.rs b/src/pipeline/src/etl/transform.rs index 15d1bf3378..4408523c27 100644 --- a/src/pipeline/src/etl/transform.rs +++ b/src/pipeline/src/etl/transform.rs @@ -16,7 +16,9 @@ pub mod index; pub mod transformer; use itertools::Itertools; +use snafu::OptionExt; +use crate::etl::error::{Error, Result}; use crate::etl::find_key_index; use crate::etl::processor::yaml_string; use crate::etl::transform::index::Index; @@ -31,6 +33,10 @@ const TRANSFORM_ON_FAILURE: &str = "on_failure"; pub use transformer::greptime::GreptimeTransformer; +use super::error::{ + KeyMustBeStringSnafu, TransformElementMustBeMapSnafu, TransformOnFailureInvalidValueSnafu, + TransformTypeMustBeSetSnafu, +}; use super::field::{Fields, InputFieldInfo, OneInputOneOutputField}; use super::processor::{yaml_new_field, yaml_new_fields}; @@ -38,11 +44,11 @@ pub trait Transformer: std::fmt::Display + Sized + Send + Sync + 'static { type Output; type VecOutput; - fn new(transforms: Transforms) -> Result; + fn new(transforms: Transforms) -> Result; fn schemas(&self) -> &Vec; fn transforms(&self) -> &Transforms; fn transforms_mut(&mut self) -> &mut Transforms; - fn transform_mut(&self, val: &mut Vec) -> Result; + fn transform_mut(&self, val: &mut Vec) -> Result; } /// On Failure behavior when transform fails @@ -57,13 +63,13 @@ pub enum OnFailure { } impl std::str::FromStr for OnFailure { - type Err = String; + type Err = Error; - fn from_str(s: &str) -> Result { + fn from_str(s: &str) -> Result { match s { "ignore" => Ok(OnFailure::Ignore), "default" => Ok(OnFailure::Default), - _ => Err(format!("invalid transform on_failure value: {}", s)), + _ => TransformOnFailureInvalidValueSnafu { value: s }.fail(), } } } @@ -139,16 +145,16 @@ impl std::ops::DerefMut for Transforms { } impl TryFrom<&Vec> for TransformBuilders { - type Error = String; + type Error = Error; - fn try_from(docs: &Vec) -> Result { + fn try_from(docs: &Vec) -> Result { let mut transforms = Vec::with_capacity(100); let mut all_output_keys: Vec = Vec::with_capacity(100); let mut all_required_keys = Vec::with_capacity(100); for doc in docs { let transform_builder: TransformBuilder = doc .as_hash() - .ok_or("transform element must be a map".to_string())? + .context(TransformElementMustBeMapSnafu)? .try_into()?; let mut transform_output_keys = transform_builder .fields @@ -187,11 +193,7 @@ pub struct TransformBuilder { } impl TransformBuilder { - pub fn build( - self, - intermediate_keys: &[String], - output_keys: &[String], - ) -> Result { + pub fn build(self, intermediate_keys: &[String], output_keys: &[String]) -> Result { let mut real_fields = vec![]; for field in self.fields { let input_index = find_key_index(intermediate_keys, field.input_field(), "transform")?; @@ -277,9 +279,9 @@ impl Transform { } impl TryFrom<&yaml_rust::yaml::Hash> for TransformBuilder { - type Error = String; + type Error = Error; - fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { + fn try_from(hash: &yaml_rust::yaml::Hash) -> Result { let mut fields = Fields::default(); let mut type_ = Value::Null; let mut default = None; @@ -287,7 +289,9 @@ impl TryFrom<&yaml_rust::yaml::Hash> for TransformBuilder { let mut on_failure = None; for (k, v) in hash { - let key = k.as_str().ok_or("key must be a string")?; + let key = k + .as_str() + .with_context(|| KeyMustBeStringSnafu { k: k.clone() })?; match key { TRANSFORM_FIELD => { fields = Fields::one(yaml_new_field(v, TRANSFORM_FIELD)?); @@ -324,10 +328,11 @@ impl TryFrom<&yaml_rust::yaml::Hash> for TransformBuilder { if let Some(default_value) = default { match (&type_, &default_value) { (Value::Null, _) => { - return Err(format!( - "transform {:?} type MUST BE set before default {}", - fields, &default_value, - )); + return TransformTypeMustBeSetSnafu { + fields: format!("{:?}", fields), + default: default_value.to_string(), + } + .fail(); } (_, Value::Null) => {} // if default is not set, then it will be regarded as default null (_, _) => { diff --git a/src/pipeline/src/etl/transform/index.rs b/src/pipeline/src/etl/transform/index.rs index f26ca4828a..6af41990a0 100644 --- a/src/pipeline/src/etl/transform/index.rs +++ b/src/pipeline/src/etl/transform/index.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::etl::error::{Error, Result, UnsupportedIndexTypeSnafu}; + const INDEX_TIMESTAMP: &str = "timestamp"; const INDEX_TIMEINDEX: &str = "time"; const INDEX_TAG: &str = "tag"; @@ -38,22 +40,22 @@ impl std::fmt::Display for Index { } impl TryFrom for Index { - type Error = String; + type Error = Error; - fn try_from(value: String) -> Result { + fn try_from(value: String) -> Result { Index::try_from(value.as_str()) } } impl TryFrom<&str> for Index { - type Error = String; + type Error = Error; - fn try_from(value: &str) -> Result { + fn try_from(value: &str) -> Result { match value { INDEX_TIMESTAMP | INDEX_TIMEINDEX => Ok(Index::Time), INDEX_TAG => Ok(Index::Tag), INDEX_FULLTEXT => Ok(Index::Fulltext), - _ => Err(format!("unsupported index type: {}", value)), + _ => UnsupportedIndexTypeSnafu { value }.fail(), } } } diff --git a/src/pipeline/src/etl/transform/transformer/greptime.rs b/src/pipeline/src/etl/transform/transformer/greptime.rs index 9753b01004..000a2ddc26 100644 --- a/src/pipeline/src/etl/transform/transformer/greptime.rs +++ b/src/pipeline/src/etl/transform/transformer/greptime.rs @@ -20,6 +20,10 @@ use coerce::{coerce_columns, coerce_value}; use greptime_proto::v1::{ColumnSchema, Row, Rows, Value as GreptimeValue}; use itertools::Itertools; +use crate::etl::error::{ + Result, TransformColumnNameMustBeUniqueSnafu, TransformEmptySnafu, + TransformMultipleTimestampIndexSnafu, TransformTimestampIndexCountSnafu, +}; use crate::etl::field::{InputFieldInfo, OneInputOneOutputField}; use crate::etl::transform::index::Index; use crate::etl::transform::{Transform, Transformer, Transforms}; @@ -71,7 +75,7 @@ impl GreptimeTransformer { } /// Generate the schema for the GreptimeTransformer - fn schemas(transforms: &Transforms) -> Result, String> { + fn schemas(transforms: &Transforms) -> Result> { let mut schema = vec![]; for transform in transforms.iter() { schema.extend(coerce_columns(transform)?); @@ -90,9 +94,9 @@ impl Transformer for GreptimeTransformer { type Output = Rows; type VecOutput = Row; - fn new(mut transforms: Transforms) -> Result { + fn new(mut transforms: Transforms) -> Result { if transforms.is_empty() { - return Err("transform cannot be empty".to_string()); + return TransformEmptySnafu.fail(); } let mut column_names_set = HashSet::new(); @@ -108,9 +112,7 @@ impl Transformer for GreptimeTransformer { let intersections: Vec<_> = column_names_set.intersection(&target_fields_set).collect(); if !intersections.is_empty() { let duplicates = intersections.iter().join(","); - return Err(format!( - "column name must be unique, but got duplicated: {duplicates}" - )); + return TransformColumnNameMustBeUniqueSnafu { duplicates }.fail(); } column_names_set.extend(target_fields_set); @@ -121,10 +123,14 @@ impl Transformer for GreptimeTransformer { 1 => timestamp_columns .push(transform.real_fields.first().unwrap().input_name()), _ => { - return Err(format!( - "Illegal to set multiple timestamp Index columns, please set only one: {}", - transform.real_fields.iter().map(|x|x.input_name()).join(", ") - )) + return TransformMultipleTimestampIndexSnafu { + columns: transform + .real_fields + .iter() + .map(|x| x.input_name()) + .join(", "), + } + .fail(); } } } @@ -145,14 +151,12 @@ impl Transformer for GreptimeTransformer { _ => { let columns: String = timestamp_columns.iter().map(|s| s.to_string()).join(", "); let count = timestamp_columns.len(); - Err( - format!("transform must have exactly one field specified as timestamp Index, but got {count}: {columns}") - ) + TransformTimestampIndexCountSnafu { count, columns }.fail() } } } - fn transform_mut(&self, val: &mut Vec) -> Result { + fn transform_mut(&self, val: &mut Vec) -> Result { let mut values = vec![GreptimeValue { value_data: None }; self.schema.len()]; for transform in self.transforms.iter() { for field in transform.real_fields.iter() { @@ -160,8 +164,7 @@ impl Transformer for GreptimeTransformer { let output_index = field.output_index(); match val.get(index) { Some(v) => { - let value_data = coerce_value(v, transform) - .map_err(|e| format!("{} processor: {}", field.input_name(), e))?; + let value_data = coerce_value(v, transform)?; // every transform fields has only one output field values[output_index] = GreptimeValue { value_data }; } diff --git a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs index 8c7efef22f..827613b02b 100644 --- a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs +++ b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs @@ -17,17 +17,22 @@ use api::v1::ColumnOptions; use datatypes::schema::FulltextOptions; use greptime_proto::v1::value::ValueData; use greptime_proto::v1::{ColumnDataType, ColumnSchema, SemanticType}; +use snafu::ResultExt; +use crate::etl::error::{ + CoerceStringToTypeSnafu, CoerceUnsupportedEpochTypeSnafu, CoerceUnsupportedNullTypeSnafu, + CoerceUnsupportedNullTypeToSnafu, ColumnOptionsSnafu, Error, Result, +}; use crate::etl::transform::index::Index; use crate::etl::transform::{OnFailure, Transform}; use crate::etl::value::{Timestamp, Value}; impl TryFrom for ValueData { - type Error = String; + type Error = Error; - fn try_from(value: Value) -> Result { + fn try_from(value: Value) -> Result { match value { - Value::Null => Err("Null type not supported".to_string()), + Value::Null => CoerceUnsupportedNullTypeSnafu.fail(), Value::Int8(v) => Ok(ValueData::I32Value(v as i32)), Value::Int16(v) => Ok(ValueData::I32Value(v as i32)), @@ -63,7 +68,7 @@ impl TryFrom for ValueData { } // TODO(yuanbohan): add fulltext support in datatype_extension -pub(crate) fn coerce_columns(transform: &Transform) -> Result, String> { +pub(crate) fn coerce_columns(transform: &Transform) -> Result> { let mut columns = Vec::new(); for field in transform.real_fields.iter() { @@ -94,19 +99,19 @@ fn coerce_semantic_type(transform: &Transform) -> SemanticType { } } -fn coerce_options(transform: &Transform) -> Result, String> { +fn coerce_options(transform: &Transform) -> Result> { if let Some(Index::Fulltext) = transform.index { options_from_fulltext(&FulltextOptions { enable: true, ..Default::default() }) - .map_err(|e| e.to_string()) + .context(ColumnOptionsSnafu) } else { Ok(None) } } -fn coerce_type(transform: &Transform) -> Result { +fn coerce_type(transform: &Transform) -> Result { match transform.type_ { Value::Int8(_) => Ok(ColumnDataType::Int8), Value::Int16(_) => Ok(ColumnDataType::Int16), @@ -132,17 +137,14 @@ fn coerce_type(transform: &Transform) -> Result { Value::Array(_) => unimplemented!("Array"), Value::Map(_) => unimplemented!("Object"), - Value::Null => Err(format!( - "Null type not supported when to coerce '{}' type", - transform.type_.to_str_type() - )), + Value::Null => CoerceUnsupportedNullTypeToSnafu { + ty: transform.type_.to_str_type(), + } + .fail(), } } -pub(crate) fn coerce_value( - val: &Value, - transform: &Transform, -) -> Result, String> { +pub(crate) fn coerce_value(val: &Value, transform: &Transform) -> Result> { match val { Value::Null => match &transform.default { Some(default) => coerce_value(default, transform), @@ -190,7 +192,7 @@ pub(crate) fn coerce_value( } } -fn coerce_bool_value(b: bool, transform: &Transform) -> Result, String> { +fn coerce_bool_value(b: bool, transform: &Transform) -> Result> { let val = match transform.type_ { Value::Int8(_) => ValueData::I8Value(b as i32), Value::Int16(_) => ValueData::I16Value(b as i32), @@ -211,9 +213,11 @@ fn coerce_bool_value(b: bool, transform: &Transform) -> Result Value::Timestamp(_) => match transform.on_failure { Some(OnFailure::Ignore) => return Ok(None), Some(OnFailure::Default) => { - return Err("default value not supported for Epoch".to_string()) + return CoerceUnsupportedEpochTypeSnafu { ty: "Default" }.fail(); + } + None => { + return CoerceUnsupportedEpochTypeSnafu { ty: "Boolean" }.fail(); } - None => return Err("Boolean type not supported for Epoch".to_string()), }, Value::Array(_) => unimplemented!("Array type not supported"), @@ -225,7 +229,7 @@ fn coerce_bool_value(b: bool, transform: &Transform) -> Result Ok(Some(val)) } -fn coerce_i64_value(n: i64, transform: &Transform) -> Result, String> { +fn coerce_i64_value(n: i64, transform: &Transform) -> Result> { let val = match transform.type_ { Value::Int8(_) => ValueData::I8Value(n as i32), Value::Int16(_) => ValueData::I16Value(n as i32), @@ -246,9 +250,11 @@ fn coerce_i64_value(n: i64, transform: &Transform) -> Result, Value::Timestamp(_) => match transform.on_failure { Some(OnFailure::Ignore) => return Ok(None), Some(OnFailure::Default) => { - return Err("default value not supported for Epoch".to_string()) + return CoerceUnsupportedEpochTypeSnafu { ty: "Default" }.fail(); + } + None => { + return CoerceUnsupportedEpochTypeSnafu { ty: "Integer" }.fail(); } - None => return Err("Integer type not supported for Epoch".to_string()), }, Value::Array(_) => unimplemented!("Array type not supported"), @@ -260,7 +266,7 @@ fn coerce_i64_value(n: i64, transform: &Transform) -> Result, Ok(Some(val)) } -fn coerce_u64_value(n: u64, transform: &Transform) -> Result, String> { +fn coerce_u64_value(n: u64, transform: &Transform) -> Result> { let val = match transform.type_ { Value::Int8(_) => ValueData::I8Value(n as i32), Value::Int16(_) => ValueData::I16Value(n as i32), @@ -281,9 +287,11 @@ fn coerce_u64_value(n: u64, transform: &Transform) -> Result, Value::Timestamp(_) => match transform.on_failure { Some(OnFailure::Ignore) => return Ok(None), Some(OnFailure::Default) => { - return Err("default value not supported for Epoch".to_string()) + return CoerceUnsupportedEpochTypeSnafu { ty: "Default" }.fail(); + } + None => { + return CoerceUnsupportedEpochTypeSnafu { ty: "Integer" }.fail(); } - None => return Err("Integer type not supported for Epoch".to_string()), }, Value::Array(_) => unimplemented!("Array type not supported"), @@ -295,7 +303,7 @@ fn coerce_u64_value(n: u64, transform: &Transform) -> Result, Ok(Some(val)) } -fn coerce_f64_value(n: f64, transform: &Transform) -> Result, String> { +fn coerce_f64_value(n: f64, transform: &Transform) -> Result> { let val = match transform.type_ { Value::Int8(_) => ValueData::I8Value(n as i32), Value::Int16(_) => ValueData::I16Value(n as i32), @@ -316,9 +324,11 @@ fn coerce_f64_value(n: f64, transform: &Transform) -> Result, Value::Timestamp(_) => match transform.on_failure { Some(OnFailure::Ignore) => return Ok(None), Some(OnFailure::Default) => { - return Err("default value not supported for Epoch".to_string()) + return CoerceUnsupportedEpochTypeSnafu { ty: "Default" }.fail(); + } + None => { + return CoerceUnsupportedEpochTypeSnafu { ty: "Float" }.fail(); } - None => return Err("Float type not supported for Epoch".to_string()), }, Value::Array(_) => unimplemented!("Array type not supported"), @@ -340,17 +350,17 @@ macro_rules! coerce_string_value { Some(default) => coerce_value(default, $transform), None => coerce_value($transform.get_type_matched_default_val(), $transform), }, - None => Err(format!( - "failed to coerce string value '{}' to type '{}'", - $s, - $transform.type_.to_str_type() - )), + None => CoerceStringToTypeSnafu { + s: $s, + ty: $transform.type_.to_str_type(), + } + .fail(), }, } }; } -fn coerce_string_value(s: &String, transform: &Transform) -> Result, String> { +fn coerce_string_value(s: &String, transform: &Transform) -> Result> { match transform.type_ { Value::Int8(_) => { coerce_string_value!(s, transform, i32, I8Value) @@ -393,8 +403,8 @@ fn coerce_string_value(s: &String, transform: &Transform) -> Result match transform.on_failure { Some(OnFailure::Ignore) => Ok(None), - Some(OnFailure::Default) => Err("default value not supported for Epoch".to_string()), - None => Err("String type not supported for Epoch".to_string()), + Some(OnFailure::Default) => CoerceUnsupportedEpochTypeSnafu { ty: "Default" }.fail(), + None => CoerceUnsupportedEpochTypeSnafu { ty: "String" }.fail(), }, Value::Array(_) => unimplemented!("Array type not supported"), diff --git a/src/pipeline/src/etl/value.rs b/src/pipeline/src/etl/value.rs index 42390a9b73..3adde2514b 100644 --- a/src/pipeline/src/etl/value.rs +++ b/src/pipeline/src/etl/value.rs @@ -19,8 +19,16 @@ pub mod time; use ahash::{HashMap, HashMapExt}; pub use array::Array; pub use map::Map; +use snafu::{OptionExt, ResultExt}; pub use time::Timestamp; +use super::error::{ + ValueDefaultValueUnsupportedSnafu, ValueInvalidResolutionSnafu, ValueParseBooleanSnafu, + ValueParseFloatSnafu, ValueParseIntSnafu, ValueParseTypeSnafu, ValueUnsupportedNumberTypeSnafu, + ValueUnsupportedYamlTypeSnafu, ValueYamlKeyMustBeStringSnafu, +}; +use crate::etl::error::{Error, Result}; + /// Value can be used as type /// acts as value: the enclosed value is the actual value /// acts as type: the enclosed value is the default value @@ -58,7 +66,7 @@ impl Value { matches!(self, Value::Null) } - pub fn parse_str_type(t: &str) -> Result { + pub fn parse_str_type(t: &str) -> Result { let mut parts = t.splitn(2, ','); let head = parts.next().unwrap_or_default(); let tail = parts.next().map(|s| s.trim().to_string()); @@ -93,10 +101,11 @@ impl Value { time::SECOND_RESOLUTION | time::SEC_RESOLUTION | time::S_RESOLUTION => { Ok(Value::Timestamp(Timestamp::Second(0))) } - _ => Err(format!( - "invalid resolution: '{resolution}'. Available resolutions: {}", - time::VALID_RESOLUTIONS.join(",") - )), + _ => ValueInvalidResolutionSnafu { + resolution, + valid_resolution: time::VALID_RESOLUTIONS.join(","), + } + .fail(), }, _ => Ok(Value::Timestamp(Timestamp::Nanosecond(0))), }, @@ -104,65 +113,68 @@ impl Value { "array" => Ok(Value::Array(Array::default())), "map" => Ok(Value::Map(Map::default())), - _ => Err(format!("failed to parse type: '{t}'")), + _ => ValueParseTypeSnafu { t }.fail(), } } /// only support string, bool, number, null - pub fn parse_str_value(&self, v: &str) -> Result { + pub fn parse_str_value(&self, v: &str) -> Result { match self { Value::Int8(_) => v .parse::() .map(Value::Int8) - .map_err(|e| format!("failed to parse int8: {}", e)), + .context(ValueParseIntSnafu { ty: "int8", v }), Value::Int16(_) => v .parse::() .map(Value::Int16) - .map_err(|e| format!("failed to parse int16: {}", e)), + .context(ValueParseIntSnafu { ty: "int16", v }), Value::Int32(_) => v .parse::() .map(Value::Int32) - .map_err(|e| format!("failed to parse int32: {}", e)), + .context(ValueParseIntSnafu { ty: "int32", v }), Value::Int64(_) => v .parse::() .map(Value::Int64) - .map_err(|e| format!("failed to parse int64: {}", e)), + .context(ValueParseIntSnafu { ty: "int64", v }), Value::Uint8(_) => v .parse::() .map(Value::Uint8) - .map_err(|e| format!("failed to parse uint8: {}", e)), + .context(ValueParseIntSnafu { ty: "uint8", v }), Value::Uint16(_) => v .parse::() .map(Value::Uint16) - .map_err(|e| format!("failed to parse uint16: {}", e)), + .context(ValueParseIntSnafu { ty: "uint16", v }), Value::Uint32(_) => v .parse::() .map(Value::Uint32) - .map_err(|e| format!("failed to parse uint32: {}", e)), + .context(ValueParseIntSnafu { ty: "uint32", v }), Value::Uint64(_) => v .parse::() .map(Value::Uint64) - .map_err(|e| format!("failed to parse uint64: {}", e)), + .context(ValueParseIntSnafu { ty: "uint64", v }), Value::Float32(_) => v .parse::() .map(Value::Float32) - .map_err(|e| format!("failed to parse float32: {}", e)), + .context(ValueParseFloatSnafu { ty: "float32", v }), Value::Float64(_) => v .parse::() .map(Value::Float64) - .map_err(|e| format!("failed to parse float64: {}", e)), + .context(ValueParseFloatSnafu { ty: "float64", v }), Value::Boolean(_) => v .parse::() .map(Value::Boolean) - .map_err(|e| format!("failed to parse bool: {}", e)), + .context(ValueParseBooleanSnafu { ty: "boolean", v }), Value::String(_) => Ok(Value::String(v.to_string())), Value::Null => Ok(Value::Null), - _ => Err(format!("default value not unsupported for type {}", self)), + _ => ValueDefaultValueUnsupportedSnafu { + value: format!("{:?}", self), + } + .fail(), } } @@ -249,9 +261,9 @@ impl std::fmt::Display for Value { } impl TryFrom for Value { - type Error = String; + type Error = Error; - fn try_from(v: serde_json::Value) -> Result { + fn try_from(v: serde_json::Value) -> Result { match v { serde_json::Value::Null => Ok(Value::Null), serde_json::Value::Bool(v) => Ok(Value::Boolean(v)), @@ -263,7 +275,7 @@ impl TryFrom for Value { } else if let Some(v) = v.as_f64() { Ok(Value::Float64(v)) } else { - Err(format!("unsupported number type: {}", v)) + ValueUnsupportedNumberTypeSnafu { value: v }.fail() } } serde_json::Value::String(v) => Ok(Value::String(v)), @@ -286,20 +298,17 @@ impl TryFrom for Value { } impl TryFrom<&yaml_rust::Yaml> for Value { - type Error = String; + type Error = Error; - fn try_from(v: &yaml_rust::Yaml) -> Result { + fn try_from(v: &yaml_rust::Yaml) -> Result { match v { yaml_rust::Yaml::Null => Ok(Value::Null), yaml_rust::Yaml::Boolean(v) => Ok(Value::Boolean(*v)), yaml_rust::Yaml::Integer(v) => Ok(Value::Int64(*v)), - yaml_rust::Yaml::Real(v) => { - if let Ok(v) = v.parse() { - Ok(Value::Float64(v)) - } else { - Err(format!("failed to parse float64: {}", v)) - } - } + yaml_rust::Yaml::Real(v) => match v.parse::() { + Ok(v) => Ok(Value::Float64(v)), + Err(e) => Err(e).context(ValueParseFloatSnafu { ty: "float64", v }), + }, yaml_rust::Yaml::String(v) => Ok(Value::String(v.to_string())), yaml_rust::Yaml::Array(arr) => { let mut values = vec![]; @@ -313,12 +322,12 @@ impl TryFrom<&yaml_rust::Yaml> for Value { for (k, v) in v { let key = k .as_str() - .ok_or(format!("key in Hash must be a string, but got {v:?}"))?; + .with_context(|| ValueYamlKeyMustBeStringSnafu { value: v.clone() })?; values.insert(key.to_string(), Value::try_from(v)?); } Ok(Value::Map(Map { values })) } - _ => Err(format!("unsupported yaml type: {v:?}")), + _ => ValueUnsupportedYamlTypeSnafu { value: v.clone() }.fail(), } } } diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index 86367232bd..8fc72c5844 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -16,6 +16,7 @@ mod etl; mod manager; mod metrics; +pub use etl::error::Result; pub use etl::processor::Processor; pub use etl::transform::{GreptimeTransformer, Transformer}; pub use etl::value::{Array, Map, Value}; diff --git a/src/pipeline/src/manager/error.rs b/src/pipeline/src/manager/error.rs index 4467b42b51..a51ad61dac 100644 --- a/src/pipeline/src/manager/error.rs +++ b/src/pipeline/src/manager/error.rs @@ -32,14 +32,16 @@ pub enum Error { #[snafu(display("Failed to insert pipeline to pipelines table"))] InsertPipeline { + #[snafu(source)] source: operator::error::Error, #[snafu(implicit)] location: Location, }, - #[snafu(display("Failed to parse pipeline: {}", reason))] + #[snafu(display("Failed to parse pipeline"))] CompilePipeline { - reason: String, + #[snafu(source)] + source: crate::etl::error::Error, #[snafu(implicit)] location: Location, }, @@ -56,6 +58,7 @@ pub enum Error { CollectRecords { #[snafu(implicit)] location: Location, + #[snafu(source)] source: common_recordbatch::error::Error, }, @@ -76,6 +79,7 @@ pub enum Error { #[snafu(display("Failed to execute internal statement"))] ExecuteInternalStatement { + #[snafu(source)] source: query::error::Error, #[snafu(implicit)] location: Location, @@ -83,6 +87,7 @@ pub enum Error { #[snafu(display("Failed to create dataframe"))] DataFrame { + #[snafu(source)] source: query::error::Error, #[snafu(implicit)] location: Location, @@ -90,6 +95,7 @@ pub enum Error { #[snafu(display("General catalog error"))] Catalog { + #[snafu(source)] source: catalog::error::Error, #[snafu(implicit)] location: Location, @@ -97,14 +103,16 @@ pub enum Error { #[snafu(display("Failed to create table"))] CreateTable { + #[snafu(source)] source: operator::error::Error, #[snafu(implicit)] location: Location, }, - #[snafu(display("Failed to execute pipeline, reason: {}", reason))] + #[snafu(display("Failed to execute pipeline"))] PipelineTransform { - reason: String, + #[snafu(source)] + source: crate::etl::error::Error, #[snafu(implicit)] location: Location, }, diff --git a/src/pipeline/src/manager/table.rs b/src/pipeline/src/manager/table.rs index 8989412c3e..7b3719b667 100644 --- a/src/pipeline/src/manager/table.rs +++ b/src/pipeline/src/manager/table.rs @@ -204,8 +204,7 @@ impl PipelineTable { /// Compile a pipeline from a string. pub fn compile_pipeline(pipeline: &str) -> Result> { let yaml_content = Content::Yaml(pipeline.into()); - parse::(&yaml_content) - .map_err(|e| CompilePipelineSnafu { reason: e }.build()) + parse::(&yaml_content).context(CompilePipelineSnafu) } /// Insert a pipeline into the pipeline table. diff --git a/src/pipeline/tests/dissect.rs b/src/pipeline/tests/dissect.rs index 22cf14c46b..7577d58080 100644 --- a/src/pipeline/tests/dissect.rs +++ b/src/pipeline/tests/dissect.rs @@ -279,5 +279,5 @@ transform: let row = pipeline.exec_mut(&mut result); assert!(row.is_err()); - assert_eq!(row.err().unwrap(), "No matching pattern found"); + assert_eq!(row.err().unwrap().to_string(), "No matching pattern found"); } diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index f0a0902837..7877d2b84a 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -115,19 +115,18 @@ pub async fn add_pipeline( ) -> Result { let start = Instant::now(); let handler = state.log_handler; - if pipeline_name.is_empty() { - return Err(InvalidParameterSnafu { + ensure!( + !pipeline_name.is_empty(), + InvalidParameterSnafu { reason: "pipeline_name is required in path", } - .build()); - } - - if payload.is_empty() { - return Err(InvalidParameterSnafu { + ); + ensure!( + !payload.is_empty(), + InvalidParameterSnafu { reason: "pipeline is required in body", } - .build()); - } + ); query_ctx.set_channel(Channel::Http); let query_ctx = Arc::new(query_ctx); @@ -252,12 +251,12 @@ pub async fn pipeline_dryrun( let value = extract_pipeline_value_by_content_type(content_type, payload, ignore_errors)?; - if value.len() > 10 { - return Err(InvalidParameterSnafu { + ensure!( + value.len() <= 10, + InvalidParameterSnafu { reason: "too many rows for dryrun", } - .build()); - } + ); query_ctx.set_channel(Channel::Http); let query_ctx = Arc::new(query_ctx); @@ -272,11 +271,11 @@ pub async fn pipeline_dryrun( for v in value { pipeline .prepare(v, &mut intermediate_state) - .map_err(|reason| PipelineTransformSnafu { reason }.build()) + .context(PipelineTransformSnafu) .context(PipelineSnafu)?; let r = pipeline .exec_mut(&mut intermediate_state) - .map_err(|reason| PipelineTransformSnafu { reason }.build()) + .context(PipelineTransformSnafu) .context(PipelineSnafu)?; results.push(r); pipeline.reset_intermediate_state(&mut intermediate_state); @@ -438,21 +437,21 @@ async fn ingest_logs_inner( for v in pipeline_data { pipeline .prepare(v, &mut intermediate_state) - .map_err(|reason| { + .inspect_err(|_| { METRIC_HTTP_LOGS_TRANSFORM_ELAPSED .with_label_values(&[db.as_str(), METRIC_FAILURE_VALUE]) .observe(transform_timer.elapsed().as_secs_f64()); - PipelineTransformSnafu { reason }.build() }) + .context(PipelineTransformSnafu) .context(PipelineSnafu)?; let r = pipeline .exec_mut(&mut intermediate_state) - .map_err(|reason| { + .inspect_err(|_| { METRIC_HTTP_LOGS_TRANSFORM_ELAPSED .with_label_values(&[db.as_str(), METRIC_FAILURE_VALUE]) .observe(transform_timer.elapsed().as_secs_f64()); - PipelineTransformSnafu { reason }.build() }) + .context(PipelineTransformSnafu) .context(PipelineSnafu)?; results.push(r); pipeline.reset_intermediate_state(&mut intermediate_state); From cc4106cbd2965f8ffe5bbe9aaa9a7a7baba6ffd0 Mon Sep 17 00:00:00 2001 From: Lanqing Yang Date: Thu, 26 Sep 2024 23:12:57 -0700 Subject: [PATCH 038/128] feat: protect datanode with concurrency limit. (#4699) Adding parallelism in region server to protect datanode from query overload. --- config/config.md | 2 + config/datanode.example.toml | 3 ++ config/standalone.example.toml | 3 ++ src/datanode/src/config.rs | 2 + src/datanode/src/datanode.rs | 5 ++- src/datanode/src/error.rs | 20 +++++++++ src/datanode/src/region_server.rs | 75 ++++++++++++++++++++++++++++++- 7 files changed, 108 insertions(+), 2 deletions(-) diff --git a/config/config.md b/config/config.md index 235b1f86fd..641eee4b58 100644 --- a/config/config.md +++ b/config/config.md @@ -17,6 +17,7 @@ | `default_timezone` | String | Unset | The default timezone of the server. | | `init_regions_in_background` | Bool | `false` | Initialize all regions in the background during the startup.
By default, it provides services after all regions have been initialized. | | `init_regions_parallelism` | Integer | `16` | Parallelism of initializing regions. | +| `max_concurrent_queries` | Integer | `0` | The maximum current queries allowed to be executed. Zero means unlimited. | | `runtime` | -- | -- | The runtime options. | | `runtime.global_rt_size` | Integer | `8` | The number of threads to execute the runtime for global read operations. | | `runtime.compact_rt_size` | Integer | `4` | The number of threads to execute the runtime for global write operations. | @@ -335,6 +336,7 @@ | `init_regions_in_background` | Bool | `false` | Initialize all regions in the background during the startup.
By default, it provides services after all regions have been initialized. | | `enable_telemetry` | Bool | `true` | Enable telemetry to collect anonymous usage data. | | `init_regions_parallelism` | Integer | `16` | Parallelism of initializing regions. | +| `max_concurrent_queries` | Integer | `0` | The maximum current queries allowed to be executed. Zero means unlimited. | | `rpc_addr` | String | Unset | Deprecated, use `grpc.addr` instead. | | `rpc_hostname` | String | Unset | Deprecated, use `grpc.hostname` instead. | | `rpc_runtime_size` | Integer | Unset | Deprecated, use `grpc.runtime_size` instead. | diff --git a/config/datanode.example.toml b/config/datanode.example.toml index 4388c4420f..e4a3dca6d3 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -19,6 +19,9 @@ enable_telemetry = true ## Parallelism of initializing regions. init_regions_parallelism = 16 +## The maximum current queries allowed to be executed. Zero means unlimited. +max_concurrent_queries = 0 + ## Deprecated, use `grpc.addr` instead. ## @toml2docs:none-default rpc_addr = "127.0.0.1:3001" diff --git a/config/standalone.example.toml b/config/standalone.example.toml index 040e1e62c2..1cd75e6414 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -15,6 +15,9 @@ init_regions_in_background = false ## Parallelism of initializing regions. init_regions_parallelism = 16 +## The maximum current queries allowed to be executed. Zero means unlimited. +max_concurrent_queries = 0 + ## The runtime options. #+ [runtime] ## The number of threads to execute the runtime for global read operations. diff --git a/src/datanode/src/config.rs b/src/datanode/src/config.rs index 7cb18131ed..70be2513b2 100644 --- a/src/datanode/src/config.rs +++ b/src/datanode/src/config.rs @@ -305,6 +305,7 @@ pub struct DatanodeOptions { pub meta_client: Option, pub wal: DatanodeWalConfig, pub storage: StorageConfig, + pub max_concurrent_queries: usize, /// Options for different store engines. pub region_engine: Vec, pub logging: LoggingOptions, @@ -339,6 +340,7 @@ impl Default for DatanodeOptions { meta_client: None, wal: DatanodeWalConfig::default(), storage: StorageConfig::default(), + max_concurrent_queries: 0, region_engine: vec![ RegionEngineConfig::Mito(MitoConfig::default()), RegionEngineConfig::File(FileEngineConfig::default()), diff --git a/src/datanode/src/datanode.rs b/src/datanode/src/datanode.rs index 149aa44ebe..f5d7bd9fc6 100644 --- a/src/datanode/src/datanode.rs +++ b/src/datanode/src/datanode.rs @@ -314,7 +314,7 @@ impl DatanodeBuilder { &self, event_listener: RegionServerEventListenerRef, ) -> Result { - let opts = &self.opts; + let opts: &DatanodeOptions = &self.opts; let query_engine_factory = QueryEngineFactory::new_with_plugins( // query engine in datanode only executes plan with resolved table source. @@ -334,6 +334,9 @@ impl DatanodeBuilder { common_runtime::global_runtime(), event_listener, table_provider_factory, + opts.max_concurrent_queries, + //TODO: revaluate the hardcoded timeout on the next version of datanode concurrency limiter. + Duration::from_millis(100), ); let object_store_manager = Self::build_object_store_manager(&opts.storage).await?; diff --git a/src/datanode/src/error.rs b/src/datanode/src/error.rs index 5717d0a403..0b36245924 100644 --- a/src/datanode/src/error.rs +++ b/src/datanode/src/error.rs @@ -22,6 +22,7 @@ use common_macro::stack_trace_debug; use snafu::{Location, Snafu}; use store_api::storage::RegionId; use table::error::Error as TableError; +use tokio::time::error::Elapsed; /// Business error of datanode. #[derive(Snafu)] @@ -347,6 +348,22 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("Failed to acquire permit, source closed"))] + ConcurrentQueryLimiterClosed { + #[snafu(source)] + error: tokio::sync::AcquireError, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to acquire permit under timeouts"))] + ConcurrentQueryLimiterTimeout { + #[snafu(source)] + error: Elapsed, + #[snafu(implicit)] + location: Location, + }, } pub type Result = std::result::Result; @@ -411,6 +428,9 @@ impl ErrorExt for Error { FindLogicalRegions { source, .. } => source.status_code(), BuildMitoEngine { source, .. } => source.status_code(), + ConcurrentQueryLimiterClosed { .. } | ConcurrentQueryLimiterTimeout { .. } => { + StatusCode::RegionBusy + } } } diff --git a/src/datanode/src/region_server.rs b/src/datanode/src/region_server.rs index 54e1cdbafd..4fcdfb86af 100644 --- a/src/datanode/src/region_server.rs +++ b/src/datanode/src/region_server.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::fmt::Debug; use std::ops::Deref; use std::sync::{Arc, RwLock}; +use std::time::Duration; use api::region::RegionResponse; use api::v1::region::{region_request, RegionResponse as RegionResponseV1}; @@ -58,10 +59,13 @@ use store_api::region_request::{ AffectedRows, RegionCloseRequest, RegionOpenRequest, RegionRequest, }; use store_api::storage::RegionId; +use tokio::sync::{Semaphore, SemaphorePermit}; +use tokio::time::timeout; use tonic::{Request, Response, Result as TonicResult}; use crate::error::{ - self, BuildRegionRequestsSnafu, DataFusionSnafu, DecodeLogicalPlanSnafu, + self, BuildRegionRequestsSnafu, ConcurrentQueryLimiterClosedSnafu, + ConcurrentQueryLimiterTimeoutSnafu, DataFusionSnafu, DecodeLogicalPlanSnafu, ExecuteLogicalPlanSnafu, FindLogicalRegionsSnafu, HandleBatchOpenRequestSnafu, HandleRegionRequestSnafu, NewPlanDecoderSnafu, RegionEngineNotFoundSnafu, RegionNotFoundSnafu, RegionNotReadySnafu, Result, StopRegionEngineSnafu, UnexpectedSnafu, UnsupportedOutputSnafu, @@ -90,6 +94,8 @@ impl RegionServer { runtime, event_listener, Arc::new(DummyTableProviderFactory), + 0, + Duration::from_millis(0), ) } @@ -98,6 +104,8 @@ impl RegionServer { runtime: Runtime, event_listener: RegionServerEventListenerRef, table_provider_factory: TableProviderFactoryRef, + max_concurrent_queries: usize, + concurrent_query_limiter_timeout: Duration, ) -> Self { Self { inner: Arc::new(RegionServerInner::new( @@ -105,6 +113,10 @@ impl RegionServer { runtime, event_listener, table_provider_factory, + RegionServerParallelism::from_opts( + max_concurrent_queries, + concurrent_query_limiter_timeout, + ), )), } } @@ -167,6 +179,11 @@ impl RegionServer { &self, request: api::v1::region::QueryRequest, ) -> Result { + let _permit = if let Some(p) = &self.inner.parallelism { + Some(p.acquire().await?) + } else { + None + }; let region_id = RegionId::from_u64(request.region_id); let provider = self.table_provider(region_id).await?; let catalog_list = Arc::new(DummyCatalogList::with_table_provider(provider)); @@ -200,6 +217,11 @@ impl RegionServer { #[tracing::instrument(skip_all)] pub async fn handle_read(&self, request: QueryRequest) -> Result { + let _permit = if let Some(p) = &self.inner.parallelism { + Some(p.acquire().await?) + } else { + None + }; let provider = self.table_provider(request.region_id).await?; struct RegionDataSourceInjector { @@ -450,6 +472,36 @@ struct RegionServerInner { runtime: Runtime, event_listener: RegionServerEventListenerRef, table_provider_factory: TableProviderFactoryRef, + // The number of queries allowed to be executed at the same time. + // Act as last line of defense on datanode to prevent query overloading. + parallelism: Option, +} + +struct RegionServerParallelism { + semaphore: Semaphore, + timeout: Duration, +} + +impl RegionServerParallelism { + pub fn from_opts( + max_concurrent_queries: usize, + concurrent_query_limiter_timeout: Duration, + ) -> Option { + if max_concurrent_queries == 0 { + return None; + } + Some(RegionServerParallelism { + semaphore: Semaphore::new(max_concurrent_queries), + timeout: concurrent_query_limiter_timeout, + }) + } + + pub async fn acquire(&self) -> Result { + timeout(self.timeout, self.semaphore.acquire()) + .await + .context(ConcurrentQueryLimiterTimeoutSnafu)? + .context(ConcurrentQueryLimiterClosedSnafu) + } } enum CurrentEngine { @@ -478,6 +530,7 @@ impl RegionServerInner { runtime: Runtime, event_listener: RegionServerEventListenerRef, table_provider_factory: TableProviderFactoryRef, + parallelism: Option, ) -> Self { Self { engines: RwLock::new(HashMap::new()), @@ -486,6 +539,7 @@ impl RegionServerInner { runtime, event_listener, table_provider_factory, + parallelism, } } @@ -1284,4 +1338,23 @@ mod tests { assert(result); } } + + #[tokio::test] + async fn test_region_server_parallism() { + let p = RegionServerParallelism::from_opts(2, Duration::from_millis(1)).unwrap(); + let first_query = p.acquire().await; + assert!(first_query.is_ok()); + let second_query = p.acquire().await; + assert!(second_query.is_ok()); + let third_query = p.acquire().await; + assert!(third_query.is_err()); + let err = third_query.unwrap_err(); + assert_eq!( + err.output_msg(), + "Failed to acquire permit under timeouts: deadline has elapsed".to_string() + ); + drop(first_query); + let forth_query = p.acquire().await; + assert!(forth_query.is_ok()); + } } From 4045298cb2b39ca8dbc98d4c25bafa3fc910bb25 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 27 Sep 2024 17:54:52 +0800 Subject: [PATCH 039/128] feat: add `region_statistics` table (#4771) * refactor: introduce `region_statistic` * refactor: move DatanodeStat related structs to common_meta * chore: add comments * feat: implement `list_region_stats` for `ClusterInfo` trait * feat: add `region_statistics` table * feat: add table_id and region_number fields * chore: rename unused snafu * chore: udpate sqlness results * chore: avoid to print source in error msg * chore: move `procedure_info` under `greptime` catalog * chore: apply suggestions from CR * Update src/common/meta/src/datanode.rs Co-authored-by: jeremyhi --------- Co-authored-by: jeremyhi --- Cargo.lock | 1 + src/catalog/src/error.rs | 10 +- src/catalog/src/kvbackend/manager.rs | 5 + .../src/system_schema/information_schema.rs | 18 +- .../information_schema/region_statistics.rs | 257 +++++++++++ .../information_schema/table_names.rs | 1 + src/common/catalog/src/consts.rs | 3 + src/common/meta/src/cluster.rs | 4 + src/common/meta/src/datanode.rs | 413 ++++++++++++++++++ src/common/meta/src/error.rs | 32 +- src/common/meta/src/lib.rs | 1 + src/datanode/Cargo.toml | 1 + src/datanode/src/heartbeat.rs | 31 +- src/datanode/src/region_server.rs | 6 +- src/datanode/src/tests.rs | 6 +- src/file-engine/src/engine.rs | 5 +- src/meta-client/src/client.rs | 23 + src/meta-srv/src/cluster.rs | 20 +- src/meta-srv/src/error.rs | 28 +- src/meta-srv/src/handler.rs | 3 +- .../src/handler/collect_stats_handler.rs | 8 +- .../src/handler/extract_stat_handler.rs | 2 +- src/meta-srv/src/handler/failure_handler.rs | 6 +- src/meta-srv/src/handler/node_stat.rs | 170 ------- .../src/handler/region_lease_handler.rs | 6 +- src/meta-srv/src/key/datanode.rs | 192 +------- src/meta-srv/src/region/supervisor.rs | 2 +- src/meta-srv/src/selector/load_based.rs | 7 +- src/meta-srv/src/selector/weight_compute.rs | 17 +- src/meta-srv/src/service/admin/heartbeat.rs | 6 +- src/metric-engine/src/engine.rs | 12 +- src/mito2/src/engine.rs | 23 +- src/mito2/src/engine/basic_test.rs | 18 +- src/mito2/src/region.rs | 17 +- src/query/src/optimizer/test_util.rs | 6 +- src/store-api/src/region_engine.rs | 40 +- .../common/show/show_databases_tables.result | 3 + .../common/system/information_schema.result | 9 + .../standalone/common/view/create.result | 1 + 39 files changed, 939 insertions(+), 474 deletions(-) create mode 100644 src/catalog/src/system_schema/information_schema/region_statistics.rs create mode 100644 src/common/meta/src/datanode.rs delete mode 100644 src/meta-srv/src/handler/node_stat.rs diff --git a/Cargo.lock b/Cargo.lock index 991829a758..3d5783dcd4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3202,6 +3202,7 @@ dependencies = [ "query", "reqwest", "serde", + "serde_json", "servers", "session", "snafu 0.8.4", diff --git a/src/catalog/src/error.rs b/src/catalog/src/error.rs index 14748a4695..dd7071b095 100644 --- a/src/catalog/src/error.rs +++ b/src/catalog/src/error.rs @@ -50,13 +50,20 @@ pub enum Error { source: BoxedError, }, - #[snafu(display("Failed to list nodes in cluster: {source}"))] + #[snafu(display("Failed to list nodes in cluster"))] ListNodes { #[snafu(implicit)] location: Location, source: BoxedError, }, + #[snafu(display("Failed to region stats in cluster"))] + ListRegionStats { + #[snafu(implicit)] + location: Location, + source: BoxedError, + }, + #[snafu(display("Failed to list flows in catalog {catalog}"))] ListFlows { #[snafu(implicit)] @@ -314,6 +321,7 @@ impl ErrorExt for Error { | Error::ListTables { source, .. } | Error::ListFlows { source, .. } | Error::ListProcedures { source, .. } + | Error::ListRegionStats { source, .. } | Error::ConvertProtoData { source, .. } => source.status_code(), Error::CreateTable { source, .. } => source.status_code(), diff --git a/src/catalog/src/kvbackend/manager.rs b/src/catalog/src/kvbackend/manager.rs index 2495a95255..e3377afbe1 100644 --- a/src/catalog/src/kvbackend/manager.rs +++ b/src/catalog/src/kvbackend/manager.rs @@ -64,12 +64,17 @@ use crate::CatalogManager; #[derive(Clone)] pub struct KvBackendCatalogManager { mode: Mode, + /// Only available in `Distributed` mode. meta_client: Option>, + /// Manages partition rules. partition_manager: PartitionRuleManagerRef, + /// Manages table metadata. table_metadata_manager: TableMetadataManagerRef, /// A sub-CatalogManager that handles system tables system_catalog: SystemCatalog, + /// Cache registry for all caches. cache_registry: LayeredCacheRegistryRef, + /// Only available in `Standalone` mode. procedure_manager: Option, } diff --git a/src/catalog/src/system_schema/information_schema.rs b/src/catalog/src/system_schema/information_schema.rs index 83f2ff4926..9fa31b85fd 100644 --- a/src/catalog/src/system_schema/information_schema.rs +++ b/src/catalog/src/system_schema/information_schema.rs @@ -20,6 +20,7 @@ pub mod key_column_usage; mod partitions; mod procedure_info; mod region_peers; +mod region_statistics; mod runtime_metrics; pub mod schemata; mod table_constraints; @@ -194,6 +195,11 @@ impl SystemSchemaProviderInner for InformationSchemaProvider { self.catalog_manager.clone(), )) as _, ), + REGION_STATISTICS => Some(Arc::new( + region_statistics::InformationSchemaRegionStatistics::new( + self.catalog_manager.clone(), + ), + ) as _), _ => None, } } @@ -241,6 +247,14 @@ impl InformationSchemaProvider { CLUSTER_INFO.to_string(), self.build_table(CLUSTER_INFO).unwrap(), ); + tables.insert( + PROCEDURE_INFO.to_string(), + self.build_table(PROCEDURE_INFO).unwrap(), + ); + tables.insert( + REGION_STATISTICS.to_string(), + self.build_table(REGION_STATISTICS).unwrap(), + ); } tables.insert(TABLES.to_string(), self.build_table(TABLES).unwrap()); @@ -256,10 +270,6 @@ impl InformationSchemaProvider { self.build_table(TABLE_CONSTRAINTS).unwrap(), ); tables.insert(FLOWS.to_string(), self.build_table(FLOWS).unwrap()); - tables.insert( - PROCEDURE_INFO.to_string(), - self.build_table(PROCEDURE_INFO).unwrap(), - ); // Add memory tables for name in MEMORY_TABLES.iter() { tables.insert((*name).to_string(), self.build_table(name).expect(name)); diff --git a/src/catalog/src/system_schema/information_schema/region_statistics.rs b/src/catalog/src/system_schema/information_schema/region_statistics.rs new file mode 100644 index 0000000000..07b94ede54 --- /dev/null +++ b/src/catalog/src/system_schema/information_schema/region_statistics.rs @@ -0,0 +1,257 @@ +// 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::sync::{Arc, Weak}; + +use arrow_schema::SchemaRef as ArrowSchemaRef; +use common_catalog::consts::INFORMATION_SCHEMA_REGION_STATISTICS_TABLE_ID; +use common_config::Mode; +use common_error::ext::BoxedError; +use common_meta::cluster::ClusterInfo; +use common_meta::datanode::RegionStat; +use common_recordbatch::adapter::RecordBatchStreamAdapter; +use common_recordbatch::{DfSendableRecordBatchStream, RecordBatch, SendableRecordBatchStream}; +use common_telemetry::tracing::warn; +use datafusion::execution::TaskContext; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter; +use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream; +use datatypes::prelude::{ConcreteDataType, ScalarVectorBuilder, VectorRef}; +use datatypes::schema::{ColumnSchema, Schema, SchemaRef}; +use datatypes::value::Value; +use datatypes::vectors::{StringVectorBuilder, UInt32VectorBuilder, UInt64VectorBuilder}; +use snafu::ResultExt; +use store_api::storage::{ScanRequest, TableId}; + +use super::{InformationTable, REGION_STATISTICS}; +use crate::error::{CreateRecordBatchSnafu, InternalSnafu, ListRegionStatsSnafu, Result}; +use crate::information_schema::Predicates; +use crate::system_schema::utils; +use crate::CatalogManager; + +const REGION_ID: &str = "region_id"; +const TABLE_ID: &str = "table_id"; +const REGION_NUMBER: &str = "region_number"; +const MEMTABLE_SIZE: &str = "memtable_size"; +const MANIFEST_SIZE: &str = "manifest_size"; +const SST_SIZE: &str = "sst_size"; +const ENGINE: &str = "engine"; +const REGION_ROLE: &str = "region_role"; + +const INIT_CAPACITY: usize = 42; + +/// The `REGION_STATISTICS` table provides information about the region statistics. Including fields: +/// +/// - `region_id`: The region id. +/// - `table_id`: The table id. +/// - `region_number`: The region number. +/// - `memtable_size`: The memtable size in bytes. +/// - `manifest_size`: The manifest size in bytes. +/// - `sst_size`: The sst size in bytes. +/// - `engine`: The engine type. +/// - `region_role`: The region role. +/// +pub(super) struct InformationSchemaRegionStatistics { + schema: SchemaRef, + catalog_manager: Weak, +} + +impl InformationSchemaRegionStatistics { + pub(super) fn new(catalog_manager: Weak) -> Self { + Self { + schema: Self::schema(), + catalog_manager, + } + } + + pub(crate) fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + ColumnSchema::new(REGION_ID, ConcreteDataType::uint64_datatype(), false), + ColumnSchema::new(TABLE_ID, ConcreteDataType::uint32_datatype(), false), + ColumnSchema::new(REGION_NUMBER, ConcreteDataType::uint32_datatype(), false), + ColumnSchema::new(MEMTABLE_SIZE, ConcreteDataType::uint64_datatype(), true), + ColumnSchema::new(MANIFEST_SIZE, ConcreteDataType::uint64_datatype(), true), + ColumnSchema::new(SST_SIZE, ConcreteDataType::uint64_datatype(), true), + ColumnSchema::new(ENGINE, ConcreteDataType::string_datatype(), true), + ColumnSchema::new(REGION_ROLE, ConcreteDataType::string_datatype(), true), + ])) + } + + fn builder(&self) -> InformationSchemaRegionStatisticsBuilder { + InformationSchemaRegionStatisticsBuilder::new( + self.schema.clone(), + self.catalog_manager.clone(), + ) + } +} + +impl InformationTable for InformationSchemaRegionStatistics { + fn table_id(&self) -> TableId { + INFORMATION_SCHEMA_REGION_STATISTICS_TABLE_ID + } + + fn table_name(&self) -> &'static str { + REGION_STATISTICS + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn to_stream(&self, request: ScanRequest) -> Result { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + + let stream = Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_region_statistics(Some(request)) + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )); + + Ok(Box::pin( + RecordBatchStreamAdapter::try_new(stream) + .map_err(BoxedError::new) + .context(InternalSnafu)?, + )) + } +} + +struct InformationSchemaRegionStatisticsBuilder { + schema: SchemaRef, + catalog_manager: Weak, + + region_ids: UInt64VectorBuilder, + table_ids: UInt32VectorBuilder, + region_numbers: UInt32VectorBuilder, + memtable_sizes: UInt64VectorBuilder, + manifest_sizes: UInt64VectorBuilder, + sst_sizes: UInt64VectorBuilder, + engines: StringVectorBuilder, + region_roles: StringVectorBuilder, +} + +impl InformationSchemaRegionStatisticsBuilder { + fn new(schema: SchemaRef, catalog_manager: Weak) -> Self { + Self { + schema, + catalog_manager, + region_ids: UInt64VectorBuilder::with_capacity(INIT_CAPACITY), + table_ids: UInt32VectorBuilder::with_capacity(INIT_CAPACITY), + region_numbers: UInt32VectorBuilder::with_capacity(INIT_CAPACITY), + memtable_sizes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY), + manifest_sizes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY), + sst_sizes: UInt64VectorBuilder::with_capacity(INIT_CAPACITY), + engines: StringVectorBuilder::with_capacity(INIT_CAPACITY), + region_roles: StringVectorBuilder::with_capacity(INIT_CAPACITY), + } + } + + /// Construct a new `InformationSchemaRegionStatistics` from the collected data. + async fn make_region_statistics( + &mut self, + request: Option, + ) -> Result { + let predicates = Predicates::from_scan_request(&request); + let mode = utils::running_mode(&self.catalog_manager)?.unwrap_or(Mode::Standalone); + + match mode { + Mode::Standalone => { + // TODO(weny): implement it + } + Mode::Distributed => { + if let Some(meta_client) = utils::meta_client(&self.catalog_manager)? { + let region_stats = meta_client + .list_region_stats() + .await + .map_err(BoxedError::new) + .context(ListRegionStatsSnafu)?; + for region_stat in region_stats { + self.add_region_statistic(&predicates, region_stat); + } + } else { + warn!("Meta client is not available"); + } + } + } + + self.finish() + } + + fn add_region_statistic(&mut self, predicate: &Predicates, region_stat: RegionStat) { + let row = [ + (REGION_ID, &Value::from(region_stat.id.as_u64())), + (TABLE_ID, &Value::from(region_stat.id.table_id())), + (REGION_NUMBER, &Value::from(region_stat.id.region_number())), + (MEMTABLE_SIZE, &Value::from(region_stat.memtable_size)), + (MANIFEST_SIZE, &Value::from(region_stat.manifest_size)), + (SST_SIZE, &Value::from(region_stat.sst_size)), + (ENGINE, &Value::from(region_stat.engine.as_str())), + (REGION_ROLE, &Value::from(region_stat.role.to_string())), + ]; + + if !predicate.eval(&row) { + return; + } + + self.region_ids.push(Some(region_stat.id.as_u64())); + self.table_ids.push(Some(region_stat.id.table_id())); + self.region_numbers + .push(Some(region_stat.id.region_number())); + self.memtable_sizes.push(Some(region_stat.memtable_size)); + self.manifest_sizes.push(Some(region_stat.manifest_size)); + self.sst_sizes.push(Some(region_stat.sst_size)); + self.engines.push(Some(®ion_stat.engine)); + self.region_roles.push(Some(®ion_stat.role.to_string())); + } + + fn finish(&mut self) -> Result { + let columns: Vec = vec![ + Arc::new(self.region_ids.finish()), + Arc::new(self.table_ids.finish()), + Arc::new(self.region_numbers.finish()), + Arc::new(self.memtable_sizes.finish()), + Arc::new(self.manifest_sizes.finish()), + Arc::new(self.sst_sizes.finish()), + Arc::new(self.engines.finish()), + Arc::new(self.region_roles.finish()), + ]; + + RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu) + } +} + +impl DfPartitionStream for InformationSchemaRegionStatistics { + fn schema(&self) -> &ArrowSchemaRef { + self.schema.arrow_schema() + } + + fn execute(&self, _: Arc) -> DfSendableRecordBatchStream { + let schema = self.schema.arrow_schema().clone(); + let mut builder = self.builder(); + Box::pin(DfRecordBatchStreamAdapter::new( + schema, + futures::stream::once(async move { + builder + .make_region_statistics(None) + .await + .map(|x| x.into_df_record_batch()) + .map_err(Into::into) + }), + )) + } +} diff --git a/src/catalog/src/system_schema/information_schema/table_names.rs b/src/catalog/src/system_schema/information_schema/table_names.rs index a62f4ddb40..53541bf955 100644 --- a/src/catalog/src/system_schema/information_schema/table_names.rs +++ b/src/catalog/src/system_schema/information_schema/table_names.rs @@ -46,3 +46,4 @@ pub const CLUSTER_INFO: &str = "cluster_info"; pub const VIEWS: &str = "views"; pub const FLOWS: &str = "flows"; pub const PROCEDURE_INFO: &str = "procedure_info"; +pub const REGION_STATISTICS: &str = "region_statistics"; diff --git a/src/common/catalog/src/consts.rs b/src/common/catalog/src/consts.rs index 2a8e2fc0e4..9cf02b81ee 100644 --- a/src/common/catalog/src/consts.rs +++ b/src/common/catalog/src/consts.rs @@ -100,6 +100,9 @@ pub const INFORMATION_SCHEMA_VIEW_TABLE_ID: u32 = 32; pub const INFORMATION_SCHEMA_FLOW_TABLE_ID: u32 = 33; /// id for information_schema.procedure_info pub const INFORMATION_SCHEMA_PROCEDURE_INFO_TABLE_ID: u32 = 34; +/// id for information_schema.region_statistics +pub const INFORMATION_SCHEMA_REGION_STATISTICS_TABLE_ID: u32 = 35; + /// ----- End of information_schema tables ----- /// ----- Begin of pg_catalog tables ----- diff --git a/src/common/meta/src/cluster.rs b/src/common/meta/src/cluster.rs index 0cf593f1ca..785a28f17d 100644 --- a/src/common/meta/src/cluster.rs +++ b/src/common/meta/src/cluster.rs @@ -20,6 +20,7 @@ use regex::Regex; use serde::{Deserialize, Serialize}; use snafu::{ensure, OptionExt, ResultExt}; +use crate::datanode::RegionStat; use crate::error::{ DecodeJsonSnafu, EncodeJsonSnafu, Error, FromUtf8Snafu, InvalidNodeInfoKeySnafu, InvalidRoleSnafu, ParseNumSnafu, Result, @@ -47,6 +48,9 @@ pub trait ClusterInfo { role: Option, ) -> std::result::Result, Self::Error>; + /// List all region stats in the cluster. + async fn list_region_stats(&self) -> std::result::Result, Self::Error>; + // TODO(jeremy): Other info, like region status, etc. } diff --git a/src/common/meta/src/datanode.rs b/src/common/meta/src/datanode.rs new file mode 100644 index 0000000000..4551b8de2f --- /dev/null +++ b/src/common/meta/src/datanode.rs @@ -0,0 +1,413 @@ +// 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::HashSet; +use std::str::FromStr; + +use api::v1::meta::{HeartbeatRequest, RequestHeader}; +use common_time::util as time_util; +use lazy_static::lazy_static; +use regex::Regex; +use serde::{Deserialize, Serialize}; +use snafu::{ensure, OptionExt, ResultExt}; +use store_api::region_engine::{RegionRole, RegionStatistic}; +use store_api::storage::RegionId; +use table::metadata::TableId; + +use crate::error::Result; +use crate::{error, ClusterId}; + +pub(crate) const DATANODE_LEASE_PREFIX: &str = "__meta_datanode_lease"; +const INACTIVE_REGION_PREFIX: &str = "__meta_inactive_region"; + +const DATANODE_STAT_PREFIX: &str = "__meta_datanode_stat"; + +pub const REGION_STATISTIC_KEY: &str = "__region_statistic"; + +lazy_static! { + pub(crate) static ref DATANODE_LEASE_KEY_PATTERN: Regex = + Regex::new(&format!("^{DATANODE_LEASE_PREFIX}-([0-9]+)-([0-9]+)$")).unwrap(); + static ref DATANODE_STAT_KEY_PATTERN: Regex = + Regex::new(&format!("^{DATANODE_STAT_PREFIX}-([0-9]+)-([0-9]+)$")).unwrap(); + static ref INACTIVE_REGION_KEY_PATTERN: Regex = Regex::new(&format!( + "^{INACTIVE_REGION_PREFIX}-([0-9]+)-([0-9]+)-([0-9]+)$" + )) + .unwrap(); +} + +/// The key of the datanode stat in the storage. +/// +/// The format is `__meta_datanode_stat-{cluster_id}-{node_id}`. +#[derive(Debug, Clone, Default, Serialize, Deserialize)] +pub struct Stat { + pub timestamp_millis: i64, + pub cluster_id: ClusterId, + // The datanode Id. + pub id: u64, + // The datanode address. + pub addr: String, + /// The read capacity units during this period + pub rcus: i64, + /// The write capacity units during this period + pub wcus: i64, + /// How many regions on this node + pub region_num: u64, + pub region_stats: Vec, + // The node epoch is used to check whether the node has restarted or redeployed. + pub node_epoch: u64, +} + +/// The statistics of a region. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct RegionStat { + /// The region_id. + pub id: RegionId, + /// The read capacity units during this period + pub rcus: i64, + /// The write capacity units during this period + pub wcus: i64, + /// Approximate bytes of this region + pub approximate_bytes: i64, + /// The engine name. + pub engine: String, + /// The region role. + pub role: RegionRole, + /// The size of the memtable in bytes. + pub memtable_size: u64, + /// The size of the manifest in bytes. + pub manifest_size: u64, + /// The size of the SST files in bytes. + pub sst_size: u64, +} + +impl Stat { + #[inline] + pub fn is_empty(&self) -> bool { + self.region_stats.is_empty() + } + + pub fn stat_key(&self) -> DatanodeStatKey { + DatanodeStatKey { + cluster_id: self.cluster_id, + node_id: self.id, + } + } + + /// Returns a tuple array containing [RegionId] and [RegionRole]. + pub fn regions(&self) -> Vec<(RegionId, RegionRole)> { + self.region_stats.iter().map(|s| (s.id, s.role)).collect() + } + + /// Returns all table ids in the region stats. + pub fn table_ids(&self) -> HashSet { + self.region_stats.iter().map(|s| s.id.table_id()).collect() + } + + /// Retains the active region stats and updates the rcus, wcus, and region_num. + pub fn retain_active_region_stats(&mut self, inactive_region_ids: &HashSet) { + if inactive_region_ids.is_empty() { + return; + } + + self.region_stats + .retain(|r| !inactive_region_ids.contains(&r.id)); + self.rcus = self.region_stats.iter().map(|s| s.rcus).sum(); + self.wcus = self.region_stats.iter().map(|s| s.wcus).sum(); + self.region_num = self.region_stats.len() as u64; + } +} + +impl TryFrom<&HeartbeatRequest> for Stat { + type Error = Option; + + fn try_from(value: &HeartbeatRequest) -> std::result::Result { + let HeartbeatRequest { + header, + peer, + region_stats, + node_epoch, + .. + } = value; + + match (header, peer) { + (Some(header), Some(peer)) => { + let region_stats = region_stats + .iter() + .map(RegionStat::from) + .collect::>(); + + Ok(Self { + timestamp_millis: time_util::current_time_millis(), + cluster_id: header.cluster_id, + // datanode id + id: peer.id, + // datanode address + addr: peer.addr.clone(), + rcus: region_stats.iter().map(|s| s.rcus).sum(), + wcus: region_stats.iter().map(|s| s.wcus).sum(), + region_num: region_stats.len() as u64, + region_stats, + node_epoch: *node_epoch, + }) + } + (header, _) => Err(header.clone()), + } + } +} + +impl From<&api::v1::meta::RegionStat> for RegionStat { + fn from(value: &api::v1::meta::RegionStat) -> Self { + let region_stat = value + .extensions + .get(REGION_STATISTIC_KEY) + .and_then(|value| RegionStatistic::deserialize_from_slice(value)) + .unwrap_or_default(); + + Self { + id: RegionId::from_u64(value.region_id), + rcus: value.rcus, + wcus: value.wcus, + approximate_bytes: value.approximate_bytes, + engine: value.engine.to_string(), + role: RegionRole::from(value.role()), + memtable_size: region_stat.memtable_size, + manifest_size: region_stat.manifest_size, + sst_size: region_stat.sst_size, + } + } +} + +/// The key of the datanode stat in the memory store. +/// +/// The format is `__meta_datanode_stat-{cluster_id}-{node_id}`. +#[derive(Debug, Clone, Copy, Eq, PartialEq, Hash)] +pub struct DatanodeStatKey { + pub cluster_id: ClusterId, + pub node_id: u64, +} + +impl DatanodeStatKey { + /// The key prefix. + pub fn prefix_key() -> Vec { + format!("{DATANODE_STAT_PREFIX}-").into_bytes() + } + + /// The key prefix with the cluster id. + pub fn key_prefix_with_cluster_id(cluster_id: ClusterId) -> String { + format!("{DATANODE_STAT_PREFIX}-{cluster_id}-") + } +} + +impl From for Vec { + fn from(value: DatanodeStatKey) -> Self { + format!( + "{}-{}-{}", + DATANODE_STAT_PREFIX, value.cluster_id, value.node_id + ) + .into_bytes() + } +} + +impl FromStr for DatanodeStatKey { + type Err = error::Error; + + fn from_str(key: &str) -> Result { + let caps = DATANODE_STAT_KEY_PATTERN + .captures(key) + .context(error::InvalidStatKeySnafu { key })?; + + ensure!(caps.len() == 3, error::InvalidStatKeySnafu { key }); + + let cluster_id = caps[1].to_string(); + let node_id = caps[2].to_string(); + let cluster_id: u64 = cluster_id.parse().context(error::ParseNumSnafu { + err_msg: format!("invalid cluster_id: {cluster_id}"), + })?; + let node_id: u64 = node_id.parse().context(error::ParseNumSnafu { + err_msg: format!("invalid node_id: {node_id}"), + })?; + + Ok(Self { + cluster_id, + node_id, + }) + } +} + +impl TryFrom> for DatanodeStatKey { + type Error = error::Error; + + fn try_from(bytes: Vec) -> Result { + String::from_utf8(bytes) + .context(error::FromUtf8Snafu { + name: "DatanodeStatKey", + }) + .map(|x| x.parse())? + } +} + +/// The value of the datanode stat in the memory store. +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(transparent)] +pub struct DatanodeStatValue { + pub stats: Vec, +} + +impl DatanodeStatValue { + /// Get the latest number of regions. + pub fn region_num(&self) -> Option { + self.stats.last().map(|x| x.region_num) + } + + /// Get the latest node addr. + pub fn node_addr(&self) -> Option { + self.stats.last().map(|x| x.addr.clone()) + } +} + +impl TryFrom for Vec { + type Error = error::Error; + + fn try_from(stats: DatanodeStatValue) -> Result { + Ok(serde_json::to_string(&stats) + .context(error::SerializeToJsonSnafu { + input: format!("{stats:?}"), + })? + .into_bytes()) + } +} + +impl FromStr for DatanodeStatValue { + type Err = error::Error; + + fn from_str(value: &str) -> Result { + serde_json::from_str(value).context(error::DeserializeFromJsonSnafu { input: value }) + } +} + +impl TryFrom> for DatanodeStatValue { + type Error = error::Error; + + fn try_from(value: Vec) -> Result { + String::from_utf8(value) + .context(error::FromUtf8Snafu { + name: "DatanodeStatValue", + }) + .map(|x| x.parse())? + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_stat_key() { + let stat = Stat { + cluster_id: 3, + id: 101, + region_num: 10, + ..Default::default() + }; + + let stat_key = stat.stat_key(); + + assert_eq!(3, stat_key.cluster_id); + assert_eq!(101, stat_key.node_id); + } + + #[test] + fn test_stat_val_round_trip() { + let stat = Stat { + cluster_id: 0, + id: 101, + region_num: 100, + ..Default::default() + }; + + let stat_val = DatanodeStatValue { stats: vec![stat] }; + + let bytes: Vec = stat_val.try_into().unwrap(); + let stat_val: DatanodeStatValue = bytes.try_into().unwrap(); + let stats = stat_val.stats; + + assert_eq!(1, stats.len()); + + let stat = stats.first().unwrap(); + assert_eq!(0, stat.cluster_id); + assert_eq!(101, stat.id); + assert_eq!(100, stat.region_num); + } + + #[test] + fn test_get_addr_from_stat_val() { + let empty = DatanodeStatValue { stats: vec![] }; + let addr = empty.node_addr(); + assert!(addr.is_none()); + + let stat_val = DatanodeStatValue { + stats: vec![ + Stat { + addr: "1".to_string(), + ..Default::default() + }, + Stat { + addr: "2".to_string(), + ..Default::default() + }, + Stat { + addr: "3".to_string(), + ..Default::default() + }, + ], + }; + let addr = stat_val.node_addr().unwrap(); + assert_eq!("3", addr); + } + + #[test] + fn test_get_region_num_from_stat_val() { + let empty = DatanodeStatValue { stats: vec![] }; + let region_num = empty.region_num(); + assert!(region_num.is_none()); + + let wrong = DatanodeStatValue { + stats: vec![Stat { + region_num: 0, + ..Default::default() + }], + }; + let right = wrong.region_num(); + assert_eq!(Some(0), right); + + let stat_val = DatanodeStatValue { + stats: vec![ + Stat { + region_num: 1, + ..Default::default() + }, + Stat { + region_num: 0, + ..Default::default() + }, + Stat { + region_num: 2, + ..Default::default() + }, + ], + }; + let region_num = stat_val.region_num().unwrap(); + assert_eq!(2, region_num); + } +} diff --git a/src/common/meta/src/error.rs b/src/common/meta/src/error.rs index b244b1e310..849ee28948 100644 --- a/src/common/meta/src/error.rs +++ b/src/common/meta/src/error.rs @@ -218,6 +218,24 @@ pub enum Error { error: JsonError, }, + #[snafu(display("Failed to serialize to json: {}", input))] + SerializeToJson { + input: String, + #[snafu(source)] + error: serde_json::error::Error, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to deserialize from json: {}", input))] + DeserializeFromJson { + input: String, + #[snafu(source)] + error: serde_json::error::Error, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Payload not exist"))] PayloadNotExist { #[snafu(implicit)] @@ -531,13 +549,20 @@ pub enum Error { location: Location, }, - #[snafu(display("Invalid node info key: {}", key))] + #[snafu(display("Invalid node info key: {}", key))] InvalidNodeInfoKey { key: String, #[snafu(implicit)] location: Location, }, + #[snafu(display("Invalid node stat key: {}", key))] + InvalidStatKey { + key: String, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Failed to parse number: {}", err_msg))] ParseNum { err_msg: String, @@ -627,7 +652,9 @@ impl ErrorExt for Error { | EtcdTxnFailed { .. } | ConnectEtcd { .. } | MoveValues { .. } - | GetCache { .. } => StatusCode::Internal, + | GetCache { .. } + | SerializeToJson { .. } + | DeserializeFromJson { .. } => StatusCode::Internal, ValueNotExist { .. } => StatusCode::Unexpected, @@ -700,6 +727,7 @@ impl ErrorExt for Error { | InvalidNumTopics { .. } | SchemaNotFound { .. } | InvalidNodeInfoKey { .. } + | InvalidStatKey { .. } | ParseNum { .. } | InvalidRole { .. } | EmptyDdlTasks { .. } => StatusCode::InvalidArguments, diff --git a/src/common/meta/src/lib.rs b/src/common/meta/src/lib.rs index 78d111c479..1dd658890c 100644 --- a/src/common/meta/src/lib.rs +++ b/src/common/meta/src/lib.rs @@ -22,6 +22,7 @@ pub mod cache; pub mod cache_invalidator; pub mod cluster; +pub mod datanode; pub mod ddl; pub mod ddl_manager; pub mod distributed_time_constants; diff --git a/src/datanode/Cargo.toml b/src/datanode/Cargo.toml index 26a7ccb675..c7ef0a7242 100644 --- a/src/datanode/Cargo.toml +++ b/src/datanode/Cargo.toml @@ -53,6 +53,7 @@ prost.workspace = true query.workspace = true reqwest.workspace = true serde.workspace = true +serde_json.workspace = true servers.workspace = true session.workspace = true snafu.workspace = true diff --git a/src/datanode/src/heartbeat.rs b/src/datanode/src/heartbeat.rs index 6633d0ab62..d84552a8d2 100644 --- a/src/datanode/src/heartbeat.rs +++ b/src/datanode/src/heartbeat.rs @@ -12,11 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::Arc; use std::time::Duration; use api::v1::meta::{HeartbeatRequest, NodeInfo, Peer, RegionRole, RegionStat}; +use common_meta::datanode::REGION_STATISTIC_KEY; use common_meta::distributed_time_constants::META_KEEP_ALIVE_INTERVAL_SECS; use common_meta::heartbeat::handler::parse_mailbox_message::ParseMailboxMessageHandler; use common_meta::heartbeat::handler::{ @@ -320,16 +322,25 @@ impl HeartbeatTask { region_server .reportable_regions() .into_iter() - .map(|stat| RegionStat { - region_id: stat.region_id.as_u64(), - engine: stat.engine, - role: RegionRole::from(stat.role).into(), - // TODO(weny): w/rcus - rcus: 0, - wcus: 0, - approximate_bytes: region_server.region_disk_usage(stat.region_id).unwrap_or(0), - // TODO(weny): add extensions - extensions: Default::default(), + .map(|stat| { + let region_stat = region_server + .region_statistic(stat.region_id) + .unwrap_or_default(); + let mut extensions = HashMap::new(); + if let Some(serialized) = region_stat.serialize_to_vec() { + extensions.insert(REGION_STATISTIC_KEY.to_string(), serialized); + } + + RegionStat { + region_id: stat.region_id.as_u64(), + engine: stat.engine, + role: RegionRole::from(stat.role).into(), + // TODO(weny): w/rcus + rcus: 0, + wcus: 0, + approximate_bytes: region_stat.estimated_disk_size() as i64, + extensions, + } }) .collect() } diff --git a/src/datanode/src/region_server.rs b/src/datanode/src/region_server.rs index 4fcdfb86af..aa80f52a5c 100644 --- a/src/datanode/src/region_server.rs +++ b/src/datanode/src/region_server.rs @@ -54,7 +54,7 @@ use snafu::{ensure, OptionExt, ResultExt}; use store_api::metric_engine_consts::{ FILE_ENGINE_NAME, LOGICAL_TABLE_METADATA_KEY, METRIC_ENGINE_NAME, }; -use store_api::region_engine::{RegionEngineRef, RegionRole, SetReadonlyResponse}; +use store_api::region_engine::{RegionEngineRef, RegionRole, RegionStatistic, SetReadonlyResponse}; use store_api::region_request::{ AffectedRows, RegionCloseRequest, RegionOpenRequest, RegionRequest, }; @@ -312,9 +312,9 @@ impl RegionServer { self.inner.runtime.clone() } - pub fn region_disk_usage(&self, region_id: RegionId) -> Option { + pub fn region_statistic(&self, region_id: RegionId) -> Option { match self.inner.region_map.get(®ion_id) { - Some(e) => e.region_disk_usage(region_id), + Some(e) => e.region_statistic(region_id), None => None, } } diff --git a/src/datanode/src/tests.rs b/src/datanode/src/tests.rs index 8966dc4932..35f513bc83 100644 --- a/src/datanode/src/tests.rs +++ b/src/datanode/src/tests.rs @@ -31,7 +31,9 @@ use query::query_engine::{DescribeResult, QueryEngineState}; use query::{QueryEngine, QueryEngineContext}; use session::context::QueryContextRef; use store_api::metadata::RegionMetadataRef; -use store_api::region_engine::{RegionEngine, RegionRole, RegionScannerRef, SetReadonlyResponse}; +use store_api::region_engine::{ + RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetReadonlyResponse, +}; use store_api::region_request::{AffectedRows, RegionRequest}; use store_api::storage::{RegionId, ScanRequest}; use table::TableRef; @@ -210,7 +212,7 @@ impl RegionEngine for MockRegionEngine { unimplemented!() } - fn region_disk_usage(&self, _region_id: RegionId) -> Option { + fn region_statistic(&self, _region_id: RegionId) -> Option { unimplemented!() } diff --git a/src/file-engine/src/engine.rs b/src/file-engine/src/engine.rs index f8cbbe7ddc..32e1a1d58d 100644 --- a/src/file-engine/src/engine.rs +++ b/src/file-engine/src/engine.rs @@ -26,7 +26,8 @@ use object_store::ObjectStore; use snafu::{ensure, OptionExt}; use store_api::metadata::RegionMetadataRef; use store_api::region_engine::{ - RegionEngine, RegionRole, RegionScannerRef, SetReadonlyResponse, SinglePartitionScanner, + RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetReadonlyResponse, + SinglePartitionScanner, }; use store_api::region_request::{ AffectedRows, RegionCloseRequest, RegionCreateRequest, RegionDropRequest, RegionOpenRequest, @@ -108,7 +109,7 @@ impl RegionEngine for FileRegionEngine { self.inner.stop().await.map_err(BoxedError::new) } - fn region_disk_usage(&self, _: RegionId) -> Option { + fn region_statistic(&self, _: RegionId) -> Option { None } diff --git a/src/meta-client/src/client.rs b/src/meta-client/src/client.rs index d5f7c6aaaf..1c5b96a684 100644 --- a/src/meta-client/src/client.rs +++ b/src/meta-client/src/client.rs @@ -29,6 +29,7 @@ use common_grpc::channel_manager::{ChannelConfig, ChannelManager}; use common_meta::cluster::{ ClusterInfo, MetasrvStatus, NodeInfo, NodeInfoKey, NodeStatus, Role as ClusterRole, }; +use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue, RegionStat}; use common_meta::ddl::{ExecutorContext, ProcedureExecutor}; use common_meta::error::{self as meta_error, Result as MetaResult}; use common_meta::rpc::ddl::{SubmitDdlTaskRequest, SubmitDdlTaskResponse}; @@ -327,6 +328,28 @@ impl ClusterInfo for MetaClient { Ok(nodes) } + + async fn list_region_stats(&self) -> Result> { + let cluster_client = self.cluster_client()?; + let range_prefix = DatanodeStatKey::key_prefix_with_cluster_id(self.id.0); + let req = RangeRequest::new().with_prefix(range_prefix); + let mut datanode_stats = cluster_client + .range(req) + .await? + .kvs + .into_iter() + .map(|kv| DatanodeStatValue::try_from(kv.value).context(ConvertMetaRequestSnafu)) + .collect::>>()?; + let region_stats = datanode_stats + .iter_mut() + .flat_map(|datanode_stat| { + let last = datanode_stat.stats.pop(); + last.map(|stat| stat.region_stats).unwrap_or_default() + }) + .collect::>(); + + Ok(region_stats) + } } impl MetaClient { diff --git a/src/meta-srv/src/cluster.rs b/src/meta-srv/src/cluster.rs index ebad9e3e6b..9a6cecbd36 100644 --- a/src/meta-srv/src/cluster.rs +++ b/src/meta-srv/src/cluster.rs @@ -23,6 +23,7 @@ use api::v1::meta::{ RangeRequest as PbRangeRequest, RangeResponse as PbRangeResponse, ResponseHeader, }; use common_grpc::channel_manager::ChannelManager; +use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue}; use common_meta::kv_backend::{KvBackend, ResettableKvBackendRef, TxnService}; use common_meta::rpc::store::{ BatchDeleteRequest, BatchDeleteResponse, BatchGetRequest, BatchGetResponse, BatchPutRequest, @@ -37,7 +38,6 @@ use snafu::{ensure, OptionExt, ResultExt}; use crate::error; use crate::error::{match_for_io_error, Result}; -use crate::key::{DatanodeStatKey, DatanodeStatValue}; use crate::metasrv::ElectionRef; pub type MetaPeerClientRef = Arc; @@ -217,7 +217,11 @@ impl MetaPeerClient { pub async fn get_node_cnt(&self) -> Result { let kvs = self.get_dn_key_value(true).await?; kvs.into_iter() - .map(|kv| kv.key.try_into()) + .map(|kv| { + kv.key + .try_into() + .context(error::InvalidDatanodeStatFormatSnafu {}) + }) .collect::>>() .map(|hash_set| hash_set.len() as i32) } @@ -319,7 +323,14 @@ impl MetaPeerClient { fn to_stat_kv_map(kvs: Vec) -> Result> { let mut map = HashMap::with_capacity(kvs.len()); for kv in kvs { - let _ = map.insert(kv.key.try_into()?, kv.value.try_into()?); + let _ = map.insert( + kv.key + .try_into() + .context(error::InvalidDatanodeStatFormatSnafu {})?, + kv.value + .try_into() + .context(error::InvalidDatanodeStatFormatSnafu {})?, + ); } Ok(map) } @@ -356,12 +367,11 @@ fn need_retry(error: &error::Error) -> bool { #[cfg(test)] mod tests { use api::v1::meta::{Error, ErrorCode, ResponseHeader}; + use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue, Stat}; use common_meta::rpc::KeyValue; use super::{check_resp_header, to_stat_kv_map, Context}; use crate::error; - use crate::handler::node_stat::Stat; - use crate::key::{DatanodeStatKey, DatanodeStatValue}; #[test] fn test_to_stat_kv_map() { diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index 728f326871..7c8d17cbe0 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -285,22 +285,6 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to parse stat key from utf8"))] - StatKeyFromUtf8 { - #[snafu(source)] - error: std::string::FromUtf8Error, - #[snafu(implicit)] - location: Location, - }, - - #[snafu(display("Failed to parse stat value from utf8"))] - StatValueFromUtf8 { - #[snafu(source)] - error: std::string::FromUtf8Error, - #[snafu(implicit)] - location: Location, - }, - #[snafu(display("Failed to parse invalid region key from utf8"))] InvalidRegionKeyFromUtf8 { #[snafu(source)] @@ -719,6 +703,13 @@ pub enum Error { source: common_meta::error::Error, }, + #[snafu(display("Invalid datanode stat format"))] + InvalidDatanodeStatFormat { + #[snafu(implicit)] + location: Location, + source: common_meta::error::Error, + }, + #[snafu(display("Failed to serialize options to TOML"))] TomlFormat { #[snafu(implicit)] @@ -815,8 +806,6 @@ impl ErrorExt for Error { | Error::TomlFormat { .. } => StatusCode::InvalidArguments, Error::LeaseKeyFromUtf8 { .. } | Error::LeaseValueFromUtf8 { .. } - | Error::StatKeyFromUtf8 { .. } - | Error::StatValueFromUtf8 { .. } | Error::InvalidRegionKeyFromUtf8 { .. } | Error::TableRouteNotFound { .. } | Error::TableInfoNotFound { .. } @@ -830,7 +819,8 @@ impl ErrorExt for Error { | Error::MigrationRunning { .. } => StatusCode::Unexpected, Error::TableNotFound { .. } => StatusCode::TableNotFound, Error::SaveClusterInfo { source, .. } - | Error::InvalidClusterInfoFormat { source, .. } => source.status_code(), + | Error::InvalidClusterInfoFormat { source, .. } + | Error::InvalidDatanodeStatFormat { source, .. } => source.status_code(), Error::InvalidateTableCache { source, .. } => source.status_code(), Error::SubmitProcedure { source, .. } | Error::WaitProcedure { source, .. } diff --git a/src/meta-srv/src/handler.rs b/src/meta-srv/src/handler.rs index d74d49027d..063d3939c1 100644 --- a/src/meta-srv/src/handler.rs +++ b/src/meta-srv/src/handler.rs @@ -22,6 +22,7 @@ use api::v1::meta::{ HeartbeatRequest, HeartbeatResponse, MailboxMessage, RegionLease, RequestHeader, ResponseHeader, Role, PROTOCOL_VERSION, }; +use common_meta::datanode::Stat; use common_meta::instruction::{Instruction, InstructionReply}; use common_meta::sequence::Sequence; use common_telemetry::{debug, info, warn}; @@ -32,7 +33,6 @@ use store_api::storage::RegionId; use tokio::sync::mpsc::Sender; use tokio::sync::{oneshot, Notify, RwLock}; -use self::node_stat::Stat; use crate::error::{self, DeserializeFromJsonSnafu, Result, UnexpectedInstructionReplySnafu}; use crate::metasrv::Context; use crate::metrics::{METRIC_META_HANDLER_EXECUTE, METRIC_META_HEARTBEAT_CONNECTION_NUM}; @@ -48,7 +48,6 @@ pub mod failure_handler; pub mod filter_inactive_region_stats; pub mod keep_lease_handler; pub mod mailbox_handler; -pub mod node_stat; pub mod on_leader_start_handler; pub mod publish_heartbeat_handler; pub mod region_lease_handler; diff --git a/src/meta-srv/src/handler/collect_stats_handler.rs b/src/meta-srv/src/handler/collect_stats_handler.rs index ec9fa231e1..b399b961ee 100644 --- a/src/meta-srv/src/handler/collect_stats_handler.rs +++ b/src/meta-srv/src/handler/collect_stats_handler.rs @@ -15,6 +15,7 @@ use std::cmp::Ordering; use api::v1::meta::{HeartbeatRequest, Role}; +use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue, Stat}; use common_meta::instruction::CacheIdent; use common_meta::key::node_address::{NodeAddressKey, NodeAddressValue}; use common_meta::key::{MetadataKey, MetadataValue}; @@ -25,9 +26,7 @@ use dashmap::DashMap; use snafu::ResultExt; use crate::error::{self, Result}; -use crate::handler::node_stat::Stat; use crate::handler::{HandleControl, HeartbeatAccumulator, HeartbeatHandler}; -use crate::key::{DatanodeStatKey, DatanodeStatValue}; use crate::metasrv::Context; const MAX_CACHED_STATS_PER_KEY: usize = 10; @@ -138,7 +137,8 @@ impl HeartbeatHandler for CollectStatsHandler { let value: Vec = DatanodeStatValue { stats: epoch_stats.drain_all(), } - .try_into()?; + .try_into() + .context(error::InvalidDatanodeStatFormatSnafu {})?; let put = PutRequest { key, value, @@ -198,6 +198,7 @@ mod tests { use std::sync::Arc; use common_meta::cache_invalidator::DummyCacheInvalidator; + use common_meta::datanode::DatanodeStatKey; use common_meta::key::TableMetadataManager; use common_meta::kv_backend::memory::MemoryKvBackend; use common_meta::sequence::SequenceBuilder; @@ -205,7 +206,6 @@ mod tests { use super::*; use crate::cluster::MetaPeerClientBuilder; use crate::handler::{HeartbeatMailbox, Pushers}; - use crate::key::DatanodeStatKey; use crate::service::store::cached_kv::LeaderCachedKvBackend; #[tokio::test] diff --git a/src/meta-srv/src/handler/extract_stat_handler.rs b/src/meta-srv/src/handler/extract_stat_handler.rs index c23e78314f..b31f41e4ef 100644 --- a/src/meta-srv/src/handler/extract_stat_handler.rs +++ b/src/meta-srv/src/handler/extract_stat_handler.rs @@ -13,9 +13,9 @@ // limitations under the License. use api::v1::meta::{HeartbeatRequest, Role}; +use common_meta::datanode::Stat; use common_telemetry::{info, warn}; -use super::node_stat::Stat; use crate::error::Result; use crate::handler::{HandleControl, HeartbeatAccumulator, HeartbeatHandler}; use crate::metasrv::Context; diff --git a/src/meta-srv/src/handler/failure_handler.rs b/src/meta-srv/src/handler/failure_handler.rs index ebeeaf6b7f..02f423c4b4 100644 --- a/src/meta-srv/src/handler/failure_handler.rs +++ b/src/meta-srv/src/handler/failure_handler.rs @@ -64,12 +64,12 @@ impl HeartbeatHandler for RegionFailureHandler { mod tests { use api::v1::meta::HeartbeatRequest; use common_catalog::consts::default_engine; + use common_meta::datanode::{RegionStat, Stat}; use store_api::region_engine::RegionRole; use store_api::storage::RegionId; use tokio::sync::oneshot; use crate::handler::failure_handler::RegionFailureHandler; - use crate::handler::node_stat::{RegionStat, Stat}; use crate::handler::{HeartbeatAccumulator, HeartbeatHandler}; use crate::metasrv::builder::MetasrvBuilder; use crate::region::supervisor::tests::new_test_supervisor; @@ -93,7 +93,9 @@ mod tests { approximate_bytes: 0, engine: default_engine().to_string(), role: RegionRole::Follower, - extensions: Default::default(), + memtable_size: 0, + manifest_size: 0, + sst_size: 0, } } acc.stat = Some(Stat { diff --git a/src/meta-srv/src/handler/node_stat.rs b/src/meta-srv/src/handler/node_stat.rs deleted file mode 100644 index 3a6c6355cd..0000000000 --- a/src/meta-srv/src/handler/node_stat.rs +++ /dev/null @@ -1,170 +0,0 @@ -// Copyright 2023 Greptime Team -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::collections::{HashMap, HashSet}; - -use api::v1::meta::{HeartbeatRequest, RequestHeader}; -use common_meta::ClusterId; -use common_time::util as time_util; -use serde::{Deserialize, Serialize}; -use store_api::region_engine::RegionRole; -use store_api::storage::RegionId; -use table::metadata::TableId; - -use crate::key::DatanodeStatKey; - -#[derive(Debug, Clone, Default, Serialize, Deserialize)] -pub struct Stat { - pub timestamp_millis: i64, - pub cluster_id: ClusterId, - // The datanode Id. - pub id: u64, - // The datanode address. - pub addr: String, - /// The read capacity units during this period - pub rcus: i64, - /// The write capacity units during this period - pub wcus: i64, - /// How many regions on this node - pub region_num: u64, - pub region_stats: Vec, - // The node epoch is used to check whether the node has restarted or redeployed. - pub node_epoch: u64, -} - -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct RegionStat { - /// The region_id. - pub id: RegionId, - /// The read capacity units during this period - pub rcus: i64, - /// The write capacity units during this period - pub wcus: i64, - /// Approximate bytes of this region - pub approximate_bytes: i64, - /// The engine name. - pub engine: String, - /// The region role. - pub role: RegionRole, - /// The extension info of this region - pub extensions: HashMap>, -} - -impl Stat { - #[inline] - pub fn is_empty(&self) -> bool { - self.region_stats.is_empty() - } - - pub fn stat_key(&self) -> DatanodeStatKey { - DatanodeStatKey { - cluster_id: self.cluster_id, - node_id: self.id, - } - } - - /// Returns a tuple array containing [RegionId] and [RegionRole]. - pub fn regions(&self) -> Vec<(RegionId, RegionRole)> { - self.region_stats.iter().map(|s| (s.id, s.role)).collect() - } - - /// Returns all table ids in the region stats. - pub fn table_ids(&self) -> HashSet { - self.region_stats.iter().map(|s| s.id.table_id()).collect() - } - - pub fn retain_active_region_stats(&mut self, inactive_region_ids: &HashSet) { - if inactive_region_ids.is_empty() { - return; - } - - self.region_stats - .retain(|r| !inactive_region_ids.contains(&r.id)); - self.rcus = self.region_stats.iter().map(|s| s.rcus).sum(); - self.wcus = self.region_stats.iter().map(|s| s.wcus).sum(); - self.region_num = self.region_stats.len() as u64; - } -} - -impl TryFrom<&HeartbeatRequest> for Stat { - type Error = Option; - - fn try_from(value: &HeartbeatRequest) -> Result { - let HeartbeatRequest { - header, - peer, - region_stats, - node_epoch, - .. - } = value; - - match (header, peer) { - (Some(header), Some(peer)) => { - let region_stats = region_stats - .iter() - .map(RegionStat::from) - .collect::>(); - - Ok(Self { - timestamp_millis: time_util::current_time_millis(), - cluster_id: header.cluster_id, - // datanode id - id: peer.id, - // datanode address - addr: peer.addr.clone(), - rcus: region_stats.iter().map(|s| s.rcus).sum(), - wcus: region_stats.iter().map(|s| s.wcus).sum(), - region_num: region_stats.len() as u64, - region_stats, - node_epoch: *node_epoch, - }) - } - (header, _) => Err(header.clone()), - } - } -} - -impl From<&api::v1::meta::RegionStat> for RegionStat { - fn from(value: &api::v1::meta::RegionStat) -> Self { - Self { - id: RegionId::from_u64(value.region_id), - rcus: value.rcus, - wcus: value.wcus, - approximate_bytes: value.approximate_bytes, - engine: value.engine.to_string(), - role: RegionRole::from(value.role()), - extensions: value.extensions.clone(), - } - } -} - -#[cfg(test)] -mod tests { - use crate::handler::node_stat::Stat; - - #[test] - fn test_stat_key() { - let stat = Stat { - cluster_id: 3, - id: 101, - region_num: 10, - ..Default::default() - }; - - let stat_key = stat.stat_key(); - - assert_eq!(3, stat_key.cluster_id); - assert_eq!(101, stat_key.node_id); - } -} diff --git a/src/meta-srv/src/handler/region_lease_handler.rs b/src/meta-srv/src/handler/region_lease_handler.rs index 28ddb436e0..06cf818d23 100644 --- a/src/meta-srv/src/handler/region_lease_handler.rs +++ b/src/meta-srv/src/handler/region_lease_handler.rs @@ -103,6 +103,7 @@ mod test { use std::collections::{HashMap, HashSet}; use std::sync::Arc; + use common_meta::datanode::{RegionStat, Stat}; use common_meta::distributed_time_constants; use common_meta::key::table_route::TableRouteValue; use common_meta::key::test_utils::new_test_table_info; @@ -115,7 +116,6 @@ mod test { use store_api::storage::RegionId; use super::*; - use crate::handler::node_stat::{RegionStat, Stat}; use crate::metasrv::builder::MetasrvBuilder; fn new_test_keeper() -> RegionLeaseKeeper { @@ -135,7 +135,9 @@ mod test { wcus: 0, approximate_bytes: 0, engine: String::new(), - extensions: Default::default(), + memtable_size: 0, + manifest_size: 0, + sst_size: 0, } } diff --git a/src/meta-srv/src/key/datanode.rs b/src/meta-srv/src/key/datanode.rs index 4fe55685b2..1c4583c233 100644 --- a/src/meta-srv/src/key/datanode.rs +++ b/src/meta-srv/src/key/datanode.rs @@ -14,6 +14,7 @@ use std::str::FromStr; +use common_meta::datanode::DatanodeStatKey; use common_meta::ClusterId; use lazy_static::lazy_static; use regex::Regex; @@ -22,7 +23,6 @@ use snafu::{ensure, OptionExt, ResultExt}; use crate::error; use crate::error::Result; -use crate::handler::node_stat::Stat; pub(crate) const DATANODE_LEASE_PREFIX: &str = "__meta_datanode_lease"; const INACTIVE_REGION_PREFIX: &str = "__meta_inactive_region"; @@ -52,18 +52,6 @@ impl DatanodeLeaseKey { } } -#[derive(Debug, Clone, Copy, Eq, PartialEq, Hash)] -pub struct DatanodeStatKey { - pub cluster_id: ClusterId, - pub node_id: u64, -} - -impl DatanodeStatKey { - pub fn prefix_key() -> Vec { - format!("{DATANODE_STAT_PREFIX}-").into_bytes() - } -} - impl From<&DatanodeLeaseKey> for DatanodeStatKey { fn from(lease_key: &DatanodeLeaseKey) -> Self { DatanodeStatKey { @@ -73,100 +61,6 @@ impl From<&DatanodeLeaseKey> for DatanodeStatKey { } } -impl From for Vec { - fn from(value: DatanodeStatKey) -> Self { - format!( - "{}-{}-{}", - DATANODE_STAT_PREFIX, value.cluster_id, value.node_id - ) - .into_bytes() - } -} - -impl FromStr for DatanodeStatKey { - type Err = error::Error; - - fn from_str(key: &str) -> Result { - let caps = DATANODE_STAT_KEY_PATTERN - .captures(key) - .context(error::InvalidStatKeySnafu { key })?; - - ensure!(caps.len() == 3, error::InvalidStatKeySnafu { key }); - - let cluster_id = caps[1].to_string(); - let node_id = caps[2].to_string(); - let cluster_id: u64 = cluster_id.parse().context(error::ParseNumSnafu { - err_msg: format!("invalid cluster_id: {cluster_id}"), - })?; - let node_id: u64 = node_id.parse().context(error::ParseNumSnafu { - err_msg: format!("invalid node_id: {node_id}"), - })?; - - Ok(Self { - cluster_id, - node_id, - }) - } -} - -impl TryFrom> for DatanodeStatKey { - type Error = error::Error; - - fn try_from(bytes: Vec) -> Result { - String::from_utf8(bytes) - .context(error::StatKeyFromUtf8Snafu {}) - .map(|x| x.parse())? - } -} - -#[derive(Debug, Clone, Serialize, Deserialize)] -#[serde(transparent)] -pub struct DatanodeStatValue { - pub stats: Vec, -} - -impl DatanodeStatValue { - /// Get the latest number of regions. - pub fn region_num(&self) -> Option { - self.stats.last().map(|x| x.region_num) - } - - /// Get the latest node addr. - pub fn node_addr(&self) -> Option { - self.stats.last().map(|x| x.addr.clone()) - } -} - -impl TryFrom for Vec { - type Error = error::Error; - - fn try_from(stats: DatanodeStatValue) -> Result { - Ok(serde_json::to_string(&stats) - .context(error::SerializeToJsonSnafu { - input: format!("{stats:?}"), - })? - .into_bytes()) - } -} - -impl FromStr for DatanodeStatValue { - type Err = error::Error; - - fn from_str(value: &str) -> Result { - serde_json::from_str(value).context(error::DeserializeFromJsonSnafu { input: value }) - } -} - -impl TryFrom> for DatanodeStatValue { - type Error = error::Error; - - fn try_from(value: Vec) -> Result { - String::from_utf8(value) - .context(error::StatValueFromUtf8Snafu {}) - .map(|x| x.parse())? - } -} - #[derive(Debug, Clone, Copy, Eq, PartialEq, Hash, Serialize, Deserialize)] pub struct InactiveRegionKey { pub cluster_id: ClusterId, @@ -253,29 +147,6 @@ mod tests { assert_eq!(1, new_key.node_id); } - #[test] - fn test_stat_val_round_trip() { - let stat = Stat { - cluster_id: 0, - id: 101, - region_num: 100, - ..Default::default() - }; - - let stat_val = DatanodeStatValue { stats: vec![stat] }; - - let bytes: Vec = stat_val.try_into().unwrap(); - let stat_val: DatanodeStatValue = bytes.try_into().unwrap(); - let stats = stat_val.stats; - - assert_eq!(1, stats.len()); - - let stat = stats.first().unwrap(); - assert_eq!(0, stat.cluster_id); - assert_eq!(101, stat.id); - assert_eq!(100, stat.region_num); - } - #[test] fn test_lease_key_round_trip() { let key = DatanodeLeaseKey { @@ -289,67 +160,6 @@ mod tests { assert_eq!(new_key, key); } - #[test] - fn test_get_addr_from_stat_val() { - let empty = DatanodeStatValue { stats: vec![] }; - let addr = empty.node_addr(); - assert!(addr.is_none()); - - let stat_val = DatanodeStatValue { - stats: vec![ - Stat { - addr: "1".to_string(), - ..Default::default() - }, - Stat { - addr: "2".to_string(), - ..Default::default() - }, - Stat { - addr: "3".to_string(), - ..Default::default() - }, - ], - }; - let addr = stat_val.node_addr().unwrap(); - assert_eq!("3", addr); - } - - #[test] - fn test_get_region_num_from_stat_val() { - let empty = DatanodeStatValue { stats: vec![] }; - let region_num = empty.region_num(); - assert!(region_num.is_none()); - - let wrong = DatanodeStatValue { - stats: vec![Stat { - region_num: 0, - ..Default::default() - }], - }; - let right = wrong.region_num(); - assert_eq!(Some(0), right); - - let stat_val = DatanodeStatValue { - stats: vec![ - Stat { - region_num: 1, - ..Default::default() - }, - Stat { - region_num: 0, - ..Default::default() - }, - Stat { - region_num: 2, - ..Default::default() - }, - ], - }; - let region_num = stat_val.region_num().unwrap(); - assert_eq!(2, region_num); - } - #[test] fn test_lease_key_to_stat_key() { let lease_key = DatanodeLeaseKey { diff --git a/src/meta-srv/src/region/supervisor.rs b/src/meta-srv/src/region/supervisor.rs index 79b305bb80..8367acbb26 100644 --- a/src/meta-srv/src/region/supervisor.rs +++ b/src/meta-srv/src/region/supervisor.rs @@ -16,6 +16,7 @@ use std::fmt::Debug; use std::sync::{Arc, Mutex}; use std::time::Duration; +use common_meta::datanode::Stat; use common_meta::ddl::{DetectingRegion, RegionFailureDetectorController}; use common_meta::key::MAINTENANCE_KEY; use common_meta::kv_backend::KvBackendRef; @@ -32,7 +33,6 @@ use tokio::time::{interval, MissedTickBehavior}; use crate::error::{self, Result}; use crate::failure_detector::PhiAccrualFailureDetectorOptions; -use crate::handler::node_stat::Stat; use crate::metasrv::{SelectorContext, SelectorRef}; use crate::procedure::region_migration::manager::RegionMigrationManagerRef; use crate::procedure::region_migration::RegionMigrationProcedureTask; diff --git a/src/meta-srv/src/selector/load_based.rs b/src/meta-srv/src/selector/load_based.rs index 12dea5854b..d7b650deda 100644 --- a/src/meta-srv/src/selector/load_based.rs +++ b/src/meta-srv/src/selector/load_based.rs @@ -14,6 +14,7 @@ use std::collections::HashMap; +use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue}; use common_meta::key::TableMetadataManager; use common_meta::peer::Peer; use common_meta::rpc::router::find_leaders; @@ -23,7 +24,7 @@ use snafu::ResultExt; use table::metadata::TableId; use crate::error::{self, Result}; -use crate::key::{DatanodeLeaseKey, DatanodeStatKey, DatanodeStatValue, LeaseValue}; +use crate::key::{DatanodeLeaseKey, LeaseValue}; use crate::lease; use crate::metasrv::SelectorContext; use crate::selector::common::choose_peers; @@ -162,7 +163,9 @@ async fn get_leader_peer_ids( mod tests { use std::collections::HashMap; - use crate::key::{DatanodeLeaseKey, DatanodeStatKey, DatanodeStatValue, LeaseValue}; + use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue}; + + use crate::key::{DatanodeLeaseKey, LeaseValue}; use crate::selector::load_based::filter_out_expired_datanode; #[test] diff --git a/src/meta-srv/src/selector/weight_compute.rs b/src/meta-srv/src/selector/weight_compute.rs index c8c555d204..09d8833e2e 100644 --- a/src/meta-srv/src/selector/weight_compute.rs +++ b/src/meta-srv/src/selector/weight_compute.rs @@ -14,10 +14,10 @@ use std::collections::HashMap; +use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue}; use common_meta::peer::Peer; use itertools::{Itertools, MinMaxResult}; -use crate::key::{DatanodeStatKey, DatanodeStatValue}; use crate::selector::weighted_choose::WeightedItem; /// The [`WeightCompute`] trait is used to compute the weight array by heartbeats. @@ -95,13 +95,12 @@ impl WeightCompute for RegionNumsBasedWeightCompute { mod tests { use std::collections::HashMap; + use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue, RegionStat, Stat}; use common_meta::peer::Peer; use store_api::region_engine::RegionRole; use store_api::storage::RegionId; use super::{RegionNumsBasedWeightCompute, WeightCompute}; - use crate::handler::node_stat::{RegionStat, Stat}; - use crate::key::{DatanodeStatKey, DatanodeStatValue}; #[test] fn test_weight_compute() { @@ -199,7 +198,9 @@ mod tests { approximate_bytes: 1, engine: "mito2".to_string(), role: RegionRole::Leader, - extensions: Default::default(), + memtable_size: 0, + manifest_size: 0, + sst_size: 0, }], ..Default::default() } @@ -216,7 +217,9 @@ mod tests { approximate_bytes: 1, engine: "mito2".to_string(), role: RegionRole::Leader, - extensions: Default::default(), + memtable_size: 0, + manifest_size: 0, + sst_size: 0, }], ..Default::default() } @@ -233,7 +236,9 @@ mod tests { approximate_bytes: 1, engine: "mito2".to_string(), role: RegionRole::Leader, - extensions: Default::default(), + memtable_size: 0, + manifest_size: 0, + sst_size: 0, }], ..Default::default() } diff --git a/src/meta-srv/src/service/admin/heartbeat.rs b/src/meta-srv/src/service/admin/heartbeat.rs index aa4b7cc96d..078d6cdc77 100644 --- a/src/meta-srv/src/service/admin/heartbeat.rs +++ b/src/meta-srv/src/service/admin/heartbeat.rs @@ -14,13 +14,13 @@ use std::collections::HashMap; +use common_meta::datanode::DatanodeStatValue; use serde::{Deserialize, Serialize}; use snafu::ResultExt; use tonic::codegen::http; use crate::cluster::MetaPeerClientRef; use crate::error::{self, Result}; -use crate::key::DatanodeStatValue; use crate::service::admin::{util, HttpHandler}; #[derive(Clone)] @@ -85,8 +85,8 @@ fn filter_by_addr(stat_vals: Vec, addr: &str) -> Vec Option { + fn region_statistic(&self, region_id: RegionId) -> Option { if self.inner.is_physical_region(region_id) { - self.inner.mito.region_disk_usage(region_id) + self.inner.mito.region_statistic(region_id) } else { None } @@ -377,7 +379,7 @@ mod test { let logical_region_id = env.default_logical_region_id(); let physical_region_id = env.default_physical_region_id(); - assert!(env.metric().region_disk_usage(logical_region_id).is_none()); - assert!(env.metric().region_disk_usage(physical_region_id).is_some()); + assert!(env.metric().region_statistic(logical_region_id).is_none()); + assert!(env.metric().region_statistic(physical_region_id).is_some()); } } diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index c0655eb690..e9177d40bf 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -76,7 +76,8 @@ use store_api::logstore::provider::Provider; use store_api::logstore::LogStore; use store_api::metadata::RegionMetadataRef; use store_api::region_engine::{ - BatchResponses, RegionEngine, RegionRole, RegionScannerRef, SetReadonlyResponse, + BatchResponses, RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, + SetReadonlyResponse, }; use store_api::region_request::{AffectedRows, RegionOpenRequest, RegionRequest}; use store_api::storage::{RegionId, ScanRequest}; @@ -89,7 +90,6 @@ use crate::error::{ use crate::manifest::action::RegionEdit; use crate::metrics::HANDLE_REQUEST_ELAPSED; use crate::read::scan_region::{ScanParallism, ScanRegion, Scanner}; -use crate::region::RegionUsage; use crate::request::{RegionEditRequest, WorkerRequest}; use crate::wal::entry_distributor::{ build_wal_entry_distributor_and_receivers, DEFAULT_ENTRY_RECEIVER_BUFFER_SIZE, @@ -133,15 +133,12 @@ impl MitoEngine { self.inner.workers.is_region_opening(region_id) } - /// Returns the region disk/memory usage information. - pub fn get_region_usage(&self, region_id: RegionId) -> Result { - let region = self - .inner + /// Returns the region disk/memory statistic. + pub fn get_region_statistic(&self, region_id: RegionId) -> Option { + self.inner .workers .get_region(region_id) - .context(RegionNotFoundSnafu { region_id })?; - - Ok(region.region_usage()) + .map(|region| region.region_statistic()) } /// Handle substrait query and return a stream of record batches @@ -546,12 +543,8 @@ impl RegionEngine for MitoEngine { self.inner.stop().await.map_err(BoxedError::new) } - fn region_disk_usage(&self, region_id: RegionId) -> Option { - let size = self - .get_region_usage(region_id) - .map(|usage| usage.disk_usage()) - .ok()?; - size.try_into().ok() + fn region_statistic(&self, region_id: RegionId) -> Option { + self.get_region_statistic(region_id) } fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<(), BoxedError> { diff --git a/src/mito2/src/engine/basic_test.rs b/src/mito2/src/engine/basic_test.rs index 9179d8a074..533b6a2ea1 100644 --- a/src/mito2/src/engine/basic_test.rs +++ b/src/mito2/src/engine/basic_test.rs @@ -552,8 +552,8 @@ async fn test_region_usage() { .unwrap(); // region is empty now, check manifest size let region = engine.get_region(region_id).unwrap(); - let region_stat = region.region_usage(); - assert_eq!(region_stat.manifest_usage, 686); + let region_stat = region.region_statistic(); + assert_eq!(region_stat.manifest_size, 686); // put some rows let rows = Rows { @@ -563,8 +563,8 @@ async fn test_region_usage() { put_rows(&engine, region_id, rows).await; - let region_stat = region.region_usage(); - assert!(region_stat.wal_usage > 0); + let region_stat = region.region_statistic(); + assert!(region_stat.wal_size > 0); // delete some rows let rows = Rows { @@ -573,18 +573,18 @@ async fn test_region_usage() { }; delete_rows(&engine, region_id, rows).await; - let region_stat = region.region_usage(); - assert!(region_stat.wal_usage > 0); + let region_stat = region.region_statistic(); + assert!(region_stat.wal_size > 0); // flush region flush_region(&engine, region_id, None).await; - let region_stat = region.region_usage(); - assert_eq!(region_stat.sst_usage, 3010); + let region_stat = region.region_statistic(); + assert_eq!(region_stat.sst_size, 3010); // region total usage // Some memtables may share items. - assert!(region_stat.disk_usage() >= 4028); + assert!(region_stat.estimated_disk_size() >= 4028); } #[tokio::test] diff --git a/src/mito2/src/region.rs b/src/mito2/src/region.rs index e2a4801643..8fc9095ae5 100644 --- a/src/mito2/src/region.rs +++ b/src/mito2/src/region.rs @@ -28,6 +28,7 @@ use crossbeam_utils::atomic::AtomicCell; use snafu::{ensure, OptionExt}; use store_api::logstore::provider::Provider; use store_api::metadata::RegionMetadataRef; +use store_api::region_engine::RegionStatistic; use store_api::storage::RegionId; use crate::access_layer::AccessLayerRef; @@ -252,10 +253,8 @@ impl MitoRegion { } } - /// Returns the region usage in bytes. - pub(crate) fn region_usage(&self) -> RegionUsage { - let region_id = self.region_id; - + /// Returns the region statistic. + pub(crate) fn region_statistic(&self) -> RegionStatistic { let version = self.version(); let memtables = &version.memtables; let memtable_usage = (memtables.mutable_usage() + memtables.immutables_usage()) as u64; @@ -265,11 +264,11 @@ impl MitoRegion { let wal_usage = self.estimated_wal_usage(memtable_usage); let manifest_usage = self.stats.total_manifest_size(); - RegionUsage { - region_id, - wal_usage, - sst_usage, - manifest_usage, + RegionStatistic { + memtable_size: memtable_usage, + wal_size: wal_usage, + manifest_size: manifest_usage, + sst_size: sst_usage, } } diff --git a/src/query/src/optimizer/test_util.rs b/src/query/src/optimizer/test_util.rs index 773270351f..6b14a5af52 100644 --- a/src/query/src/optimizer/test_util.rs +++ b/src/query/src/optimizer/test_util.rs @@ -27,7 +27,9 @@ use datatypes::schema::ColumnSchema; use store_api::metadata::{ ColumnMetadata, RegionMetadata, RegionMetadataBuilder, RegionMetadataRef, }; -use store_api::region_engine::{RegionEngine, RegionRole, RegionScannerRef, SetReadonlyResponse}; +use store_api::region_engine::{ + RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetReadonlyResponse, +}; use store_api::region_request::RegionRequest; use store_api::storage::{ConcreteDataType, RegionId, ScanRequest}; @@ -79,7 +81,7 @@ impl RegionEngine for MetaRegionEngine { }) } - fn region_disk_usage(&self, _region_id: RegionId) -> Option { + fn region_statistic(&self, _region_id: RegionId) -> Option { None } diff --git a/src/store-api/src/region_engine.rs b/src/store-api/src/region_engine.rs index 84555a595b..483d3cc1ad 100644 --- a/src/store-api/src/region_engine.rs +++ b/src/store-api/src/region_engine.rs @@ -242,6 +242,42 @@ pub type RegionScannerRef = Box; pub type BatchResponses = Vec<(RegionId, Result)>; +/// Represents the statistics of a region. +#[derive(Debug, Deserialize, Serialize, Default)] +pub struct RegionStatistic { + /// The size of memtable in bytes. + pub memtable_size: u64, + /// The size of WAL in bytes. + pub wal_size: u64, + /// The size of manifest in bytes. + pub manifest_size: u64, + /// The size of SST files in bytes. + pub sst_size: u64, +} + +impl RegionStatistic { + /// Deserializes the region statistic to a byte array. + /// + /// Returns None if the deserialization fails. + pub fn deserialize_from_slice(value: &[u8]) -> Option { + serde_json::from_slice(value).ok() + } + + /// Serializes the region statistic to a byte array. + /// + /// Returns None if the serialization fails. + pub fn serialize_to_vec(&self) -> Option> { + serde_json::to_vec(self).ok() + } +} + +impl RegionStatistic { + /// Returns the estimated disk size of the region. + pub fn estimated_disk_size(&self) -> u64 { + self.wal_size + self.sst_size + self.manifest_size + } +} + #[async_trait] pub trait RegionEngine: Send + Sync { /// Name of this engine @@ -289,8 +325,8 @@ pub trait RegionEngine: Send + Sync { /// Retrieves region's metadata. async fn get_metadata(&self, region_id: RegionId) -> Result; - /// Retrieves region's disk usage. - fn region_disk_usage(&self, region_id: RegionId) -> Option; + /// Retrieves region's statistic. + fn region_statistic(&self, region_id: RegionId) -> Option; /// Stops the engine async fn stop(&self) -> Result<(), BoxedError>; diff --git a/tests/cases/standalone/common/show/show_databases_tables.result b/tests/cases/standalone/common/show/show_databases_tables.result index c92a5b33c8..6dc267b61b 100644 --- a/tests/cases/standalone/common/show/show_databases_tables.result +++ b/tests/cases/standalone/common/show/show_databases_tables.result @@ -49,6 +49,7 @@ SHOW TABLES; | profiling | | referential_constraints | | region_peers | +| region_statistics | | routines | | runtime_metrics | | schema_privileges | @@ -96,6 +97,7 @@ SHOW FULL TABLES; | profiling | LOCAL TEMPORARY | | referential_constraints | LOCAL TEMPORARY | | region_peers | LOCAL TEMPORARY | +| region_statistics | LOCAL TEMPORARY | | routines | LOCAL TEMPORARY | | runtime_metrics | LOCAL TEMPORARY | | schema_privileges | LOCAL TEMPORARY | @@ -137,6 +139,7 @@ SHOW TABLE STATUS; |profiling||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |referential_constraints||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |region_peers||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| +|region_statistics||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |routines||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |runtime_metrics||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| |schema_privileges||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0||| diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index 03b65cf390..882607f842 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -36,6 +36,7 @@ order by table_schema, table_name; |greptime|information_schema|profiling|LOCALTEMPORARY|19|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|referential_constraints|LOCALTEMPORARY|20|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|region_peers|LOCALTEMPORARY|29|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| +|greptime|information_schema|region_statistics|LOCALTEMPORARY|35|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|routines|LOCALTEMPORARY|21|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|runtime_metrics|LOCALTEMPORARY|27|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| |greptime|information_schema|schema_privileges|LOCALTEMPORARY|22|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y| @@ -298,6 +299,14 @@ select * from information_schema.columns order by table_schema, table_name, colu | greptime | information_schema | region_peers | peer_id | 2 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | | | greptime | information_schema | region_peers | region_id | 1 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | | | greptime | information_schema | region_peers | status | 5 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | region_statistics | engine | 7 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | region_statistics | manifest_size | 5 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | | +| greptime | information_schema | region_statistics | memtable_size | 4 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | | +| greptime | information_schema | region_statistics | region_id | 1 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | | +| greptime | information_schema | region_statistics | region_number | 3 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | | +| greptime | information_schema | region_statistics | region_role | 8 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | | +| greptime | information_schema | region_statistics | sst_size | 6 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | | +| greptime | information_schema | region_statistics | table_id | 2 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | | | greptime | information_schema | routines | character_maximum_length | 7 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | | | greptime | information_schema | routines | character_octet_length | 8 | | | 19 | 0 | | | | | | select,insert | | Int64 | bigint | FIELD | | No | bigint | | | | greptime | information_schema | routines | character_set_client | 29 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | | diff --git a/tests/cases/standalone/common/view/create.result b/tests/cases/standalone/common/view/create.result index 4fd2f59356..06d892d189 100644 --- a/tests/cases/standalone/common/view/create.result +++ b/tests/cases/standalone/common/view/create.result @@ -110,6 +110,7 @@ SELECT * FROM INFORMATION_SCHEMA.TABLES ORDER BY TABLE_NAME, TABLE_TYPE; |greptime|information_schema|profiling|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|referential_constraints|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|region_peers|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| +|greptime|information_schema|region_statistics|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|routines|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|runtime_metrics|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| |greptime|information_schema|schema_privileges|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y| From 934bc139677be572f816ee28e80708cf7ac85eab Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Fri, 27 Sep 2024 19:17:36 +0800 Subject: [PATCH 040/128] feat(mito): limit compaction output file size (#4754) * Commit Message Clarify documentation for CompactionOutput struct Updated the documentation for the `CompactionOutput` struct to specify that the output time range is only relevant for windowed compaction. * Add max_output_file_size to TwcsPicker and TwcsOptions - Introduced `max_output_file_size` to `TwcsPicker` struct and its logic to enforce output file size limits during compaction. - Updated `TwcsOptions` to include `max_output_file_size` and adjusted related tests. - Modified `new_picker` function to initialize `TwcsPicker` with the new `max_output_file_size` field. * feat/limit-compaction-output-size: Refactor compaction picker and TWCS to support append mode and improve options handling - Update compaction picker to accept a reference to options and append mode flag - Modify TWCS picker logic to consider append mode when filtering deleted rows - Remove VersionControl usage in compactor and simplify return type - Adjust enforce_max_output_size logic in TWCS picker to handle max output file size - Add append mode flag to TwcsPicker struct - Fix incorrect condition in TWCS picker for enforcing max output size - Update region options tests to reflect new max output file size format (1GB and 7MB) - Simplify InvalidTableOptionSnafu error handling in create_parser - Add `compaction.twcs.max_output_file_size` to mito engine option keys * resolve some comments --- src/mito2/src/compaction.rs | 8 +- src/mito2/src/compaction/compactor.rs | 8 +- src/mito2/src/compaction/picker.rs | 21 +-- src/mito2/src/compaction/twcs.rs | 198 +++++++++++++++++------ src/mito2/src/region/options.rs | 8 + src/sql/src/parsers/create_parser.rs | 7 +- src/store-api/src/mito_engine_options.rs | 1 + 7 files changed, 185 insertions(+), 66 deletions(-) diff --git a/src/mito2/src/compaction.rs b/src/mito2/src/compaction.rs index 0f33471b21..3cea492071 100644 --- a/src/mito2/src/compaction.rs +++ b/src/mito2/src/compaction.rs @@ -240,7 +240,11 @@ impl CompactionScheduler { request: CompactionRequest, options: compact_request::Options, ) -> Result<()> { - let picker = new_picker(options.clone(), &request.current_version.options.compaction); + let picker = new_picker( + &options, + &request.current_version.options.compaction, + request.current_version.options.append_mode, + ); let region_id = request.region_id(); let CompactionRequest { engine_config, @@ -500,7 +504,7 @@ pub struct CompactionOutput { pub inputs: Vec, /// Whether to remove deletion markers. pub filter_deleted: bool, - /// Compaction output time range. + /// Compaction output time range. Only windowed compaction specifies output time range. pub output_time_range: Option, } diff --git a/src/mito2/src/compaction/compactor.rs b/src/mito2/src/compaction/compactor.rs index 9c8c0e02bd..12b9dd5fef 100644 --- a/src/mito2/src/compaction/compactor.rs +++ b/src/mito2/src/compaction/compactor.rs @@ -38,7 +38,7 @@ use crate::memtable::MemtableBuilderProvider; use crate::read::Source; use crate::region::opener::new_manifest_dir; use crate::region::options::RegionOptions; -use crate::region::version::{VersionBuilder, VersionControl, VersionRef}; +use crate::region::version::{VersionBuilder, VersionRef}; use crate::region::ManifestContext; use crate::region::RegionState::Writable; use crate::schedule::scheduler::LocalScheduler; @@ -164,8 +164,7 @@ pub async fn open_compaction_region( .compaction_time_window(manifest.compaction_time_window) .options(req.region_options.clone()) .build(); - let version_control = Arc::new(VersionControl::new(version)); - version_control.current().version + Arc::new(version) }; Ok(CompactionRegion { @@ -395,8 +394,9 @@ impl Compactor for DefaultCompactor { ) -> Result<()> { let picker_output = { let picker_output = new_picker( - compact_request_options, + &compact_request_options, &compaction_region.region_options.compaction, + compaction_region.region_options.append_mode, ) .pick(compaction_region); diff --git a/src/mito2/src/compaction/picker.rs b/src/mito2/src/compaction/picker.rs index 30c8d28446..9397c2bf64 100644 --- a/src/mito2/src/compaction/picker.rs +++ b/src/mito2/src/compaction/picker.rs @@ -119,10 +119,11 @@ impl PickerOutput { /// Create a new picker based on the compaction request options and compaction options. pub fn new_picker( - compact_request_options: compact_request::Options, + compact_request_options: &compact_request::Options, compaction_options: &CompactionOptions, + append_mode: bool, ) -> Arc { - if let compact_request::Options::StrictWindow(window) = &compact_request_options { + if let compact_request::Options::StrictWindow(window) = compact_request_options { let window = if window.window_seconds == 0 { None } else { @@ -131,13 +132,15 @@ pub fn new_picker( Arc::new(WindowedCompactionPicker::new(window)) as Arc<_> } else { match compaction_options { - CompactionOptions::Twcs(twcs_opts) => Arc::new(TwcsPicker::new( - twcs_opts.max_active_window_runs, - twcs_opts.max_active_window_files, - twcs_opts.max_inactive_window_runs, - twcs_opts.max_inactive_window_files, - twcs_opts.time_window_seconds(), - )) as Arc<_>, + CompactionOptions::Twcs(twcs_opts) => Arc::new(TwcsPicker { + max_active_window_runs: twcs_opts.max_active_window_runs, + max_active_window_files: twcs_opts.max_active_window_files, + max_inactive_window_runs: twcs_opts.max_inactive_window_runs, + max_inactive_window_files: twcs_opts.max_inactive_window_files, + time_window_seconds: twcs_opts.time_window_seconds(), + max_output_file_size: twcs_opts.max_output_file_size.map(|r| r.as_bytes()), + append_mode, + }) as Arc<_>, } } } diff --git a/src/mito2/src/compaction/twcs.rs b/src/mito2/src/compaction/twcs.rs index 4bbad692f0..c6d2a112aa 100644 --- a/src/mito2/src/compaction/twcs.rs +++ b/src/mito2/src/compaction/twcs.rs @@ -35,30 +35,23 @@ const LEVEL_COMPACTED: Level = 1; /// candidates. #[derive(Debug)] pub struct TwcsPicker { - max_active_window_runs: usize, - max_active_window_files: usize, - max_inactive_window_runs: usize, - max_inactive_window_files: usize, - time_window_seconds: Option, + /// Max allowed sorted runs in active window. + pub max_active_window_runs: usize, + /// Max allowed files in active window. + pub max_active_window_files: usize, + /// Max allowed sorted runs in inactive windows. + pub max_inactive_window_runs: usize, + /// Max allowed files in inactive windows. + pub max_inactive_window_files: usize, + /// Compaction time window in seconds. + pub time_window_seconds: Option, + /// Max allowed compaction output file size. + pub max_output_file_size: Option, + /// Whether the target region is in append mode. + pub append_mode: bool, } impl TwcsPicker { - pub fn new( - max_active_window_runs: usize, - max_active_window_files: usize, - max_inactive_window_runs: usize, - max_inactive_window_files: usize, - time_window_seconds: Option, - ) -> Self { - Self { - max_inactive_window_runs, - max_active_window_runs, - time_window_seconds, - max_active_window_files, - max_inactive_window_files, - } - } - /// Builds compaction output from files. /// For active writing window, we allow for at most `max_active_window_runs` files to alleviate /// fragmentation. For other windows, we allow at most 1 file at each window. @@ -82,47 +75,114 @@ impl TwcsPicker { ) }; - // we only remove deletion markers once no file in current window overlaps with any other window. let found_runs = sorted_runs.len(); - let filter_deleted = !files.overlapping && (found_runs == 1 || max_runs == 1); + // We only remove deletion markers once no file in current window overlaps with any other window + // and region is not in append mode. + let filter_deleted = + !files.overlapping && (found_runs == 1 || max_runs == 1) && !self.append_mode; - if found_runs > max_runs { + let inputs = if found_runs > max_runs { let files_to_compact = reduce_runs(sorted_runs, max_runs); - info!("Building compaction output, active window: {:?}, current window: {}, max runs: {}, found runs: {}, output size: {}, remove deletion markers: {}", active_window, *window,max_runs, found_runs, files_to_compact.len(), filter_deleted); - for inputs in files_to_compact { - output.push(CompactionOutput { - output_file_id: FileId::random(), - output_level: LEVEL_COMPACTED, // always compact to l1 - inputs, - filter_deleted, - output_time_range: None, // we do not enforce output time range in twcs compactions. - }); - } + let files_to_compact_len = files_to_compact.len(); + info!( + "Building compaction output, active window: {:?}, \ + current window: {}, \ + max runs: {}, \ + found runs: {}, \ + output size: {}, \ + max output size: {:?}, \ + remove deletion markers: {}", + active_window, + *window, + max_runs, + found_runs, + files_to_compact_len, + self.max_output_file_size, + filter_deleted + ); + files_to_compact } else if files.files.len() > max_files { - debug!( - "Enforcing max file num in window: {}, active: {:?}, max: {}, current: {}", + info!( + "Enforcing max file num in window: {}, active: {:?}, max: {}, current: {}, max output size: {:?}, filter delete: {}", *window, active_window, max_files, - files.files.len() + files.files.len(), + self.max_output_file_size, + filter_deleted, ); // Files in window exceeds file num limit - let to_merge = enforce_file_num(&files.files, max_files); + vec![enforce_file_num(&files.files, max_files)] + } else { + debug!("Skip building compaction output, active window: {:?}, current window: {}, max runs: {}, found runs: {}, ", active_window, *window, max_runs, found_runs); + continue; + }; + + let split_inputs = if !filter_deleted + && let Some(max_output_file_size) = self.max_output_file_size + { + let len_before_split = inputs.len(); + let maybe_split = enforce_max_output_size(inputs, max_output_file_size); + if maybe_split.len() != len_before_split { + info!("Compaction output file size exceeds threshold {}, split compaction inputs to: {:?}", max_output_file_size, maybe_split); + } + maybe_split + } else { + inputs + }; + + for input in split_inputs { + debug_assert!(input.len() > 1); output.push(CompactionOutput { output_file_id: FileId::random(), output_level: LEVEL_COMPACTED, // always compact to l1 - inputs: to_merge, + inputs: input, filter_deleted, - output_time_range: None, + output_time_range: None, // we do not enforce output time range in twcs compactions. }); - } else { - debug!("Skip building compaction output, active window: {:?}, current window: {}, max runs: {}, found runs: {}, ", active_window, *window, max_runs, found_runs); } } output } } +/// Limits the size of compaction output in a naive manner. +/// todo(hl): we can find the output file size more precisely by checking the time range +/// of each row group and adding the sizes of those non-overlapping row groups. But now +/// we'd better not to expose the SST details in this level. +fn enforce_max_output_size( + inputs: Vec>, + max_output_file_size: u64, +) -> Vec> { + inputs + .into_iter() + .flat_map(|input| { + debug_assert!(input.len() > 1); + let estimated_output_size = input.iter().map(|f| f.size()).sum::(); + if estimated_output_size < max_output_file_size { + // total file size does not exceed the threshold, just return the original input. + return vec![input]; + } + let mut splits = vec![]; + let mut new_input = vec![]; + let mut new_input_size = 0; + for f in input { + if new_input_size + f.size() > max_output_file_size { + splits.push(std::mem::take(&mut new_input)); + new_input_size = 0; + } + new_input_size += f.size(); + new_input.push(f); + } + if !new_input.is_empty() { + splits.push(new_input); + } + splits + }) + .filter(|p| p.len() > 1) + .collect() +} + /// Merges consecutive files so that file num does not exceed `max_file_num`, and chooses /// the solution with minimum overhead according to files sizes to be merged. /// `enforce_file_num` only merges consecutive files so that it won't create overlapping outputs. @@ -305,10 +365,12 @@ fn find_latest_window_in_seconds<'a>( #[cfg(test)] mod tests { use std::collections::HashSet; + use std::sync::Arc; use super::*; use crate::compaction::test_util::{new_file_handle, new_file_handles}; - use crate::sst::file::Level; + use crate::sst::file::{FileMeta, Level}; + use crate::test_util::NoopFilePurger; #[test] fn test_get_latest_window_in_seconds() { @@ -525,8 +587,16 @@ mod tests { let mut windows = assign_to_windows(self.input_files.iter(), self.window_size); let active_window = find_latest_window_in_seconds(self.input_files.iter(), self.window_size); - let output = TwcsPicker::new(4, usize::MAX, 1, usize::MAX, None) - .build_output(&mut windows, active_window); + let output = TwcsPicker { + max_active_window_runs: 4, + max_active_window_files: usize::MAX, + max_inactive_window_runs: 1, + max_inactive_window_files: usize::MAX, + time_window_seconds: None, + max_output_file_size: None, + append_mode: false, + } + .build_output(&mut windows, active_window); let output = output .iter() @@ -641,5 +711,43 @@ mod tests { .check(); } + fn make_file_handles(inputs: &[(i64, i64, u64)]) -> Vec { + inputs + .iter() + .map(|(start, end, size)| { + FileHandle::new( + FileMeta { + region_id: Default::default(), + file_id: Default::default(), + time_range: ( + Timestamp::new_millisecond(*start), + Timestamp::new_millisecond(*end), + ), + level: 0, + file_size: *size, + available_indexes: Default::default(), + index_file_size: 0, + num_rows: 0, + num_row_groups: 0, + }, + Arc::new(NoopFilePurger), + ) + }) + .collect() + } + + #[test] + fn test_limit_output_size() { + let mut files = make_file_handles(&[(1, 1, 1)].repeat(6)); + let runs = find_sorted_runs(&mut files); + assert_eq!(6, runs.len()); + let files_to_merge = reduce_runs(runs, 2); + + let enforced = enforce_max_output_size(files_to_merge, 2); + assert_eq!(2, enforced.len()); + assert_eq!(2, enforced[0].len()); + assert_eq!(2, enforced[1].len()); + } + // TODO(hl): TTL tester that checks if get_expired_ssts function works as expected. } diff --git a/src/mito2/src/region/options.rs b/src/mito2/src/region/options.rs index cc866e5502..4abc5925b7 100644 --- a/src/mito2/src/region/options.rs +++ b/src/mito2/src/region/options.rs @@ -204,6 +204,8 @@ pub struct TwcsOptions { /// Compaction time window defined when creating tables. #[serde(with = "humantime_serde")] pub time_window: Option, + /// Compaction time window defined when creating tables. + pub max_output_file_size: Option, /// Whether to use remote compaction. #[serde_as(as = "DisplayFromStr")] pub remote_compaction: bool, @@ -236,6 +238,7 @@ impl Default for TwcsOptions { max_inactive_window_runs: 1, max_inactive_window_files: 1, time_window: None, + max_output_file_size: None, remote_compaction: false, fallback_to_local: true, } @@ -597,6 +600,7 @@ mod tests { ("compaction.twcs.max_active_window_files", "11"), ("compaction.twcs.max_inactive_window_runs", "2"), ("compaction.twcs.max_inactive_window_files", "3"), + ("compaction.twcs.max_output_file_size", "1GB"), ("compaction.twcs.time_window", "2h"), ("compaction.type", "twcs"), ("compaction.twcs.remote_compaction", "false"), @@ -624,6 +628,7 @@ mod tests { max_inactive_window_runs: 2, max_inactive_window_files: 3, time_window: Some(Duration::from_secs(3600 * 2)), + max_output_file_size: Some(ReadableSize::gb(1)), remote_compaction: false, fallback_to_local: true, }), @@ -656,6 +661,7 @@ mod tests { max_inactive_window_runs: 2, max_inactive_window_files: usize::MAX, time_window: Some(Duration::from_secs(3600 * 2)), + max_output_file_size: None, remote_compaction: false, fallback_to_local: true, }), @@ -693,6 +699,7 @@ mod tests { "compaction.twcs.max_active_window_files": "11", "compaction.twcs.max_inactive_window_runs": "2", "compaction.twcs.max_inactive_window_files": "7", + "compaction.twcs.max_output_file_size": "7MB", "compaction.twcs.time_window": "2h" }, "storage": "S3", @@ -722,6 +729,7 @@ mod tests { max_inactive_window_runs: 2, max_inactive_window_files: 7, time_window: Some(Duration::from_secs(3600 * 2)), + max_output_file_size: Some(ReadableSize::mb(7)), remote_compaction: false, fallback_to_local: true, }), diff --git a/src/sql/src/parsers/create_parser.rs b/src/sql/src/parsers/create_parser.rs index de81bd1a21..8b7eb420d2 100644 --- a/src/sql/src/parsers/create_parser.rs +++ b/src/sql/src/parsers/create_parser.rs @@ -370,12 +370,7 @@ impl<'a> ParserContext<'a> { .map(parse_option_string) .collect::>>()?; for key in options.keys() { - ensure!( - validate_table_option(key), - InvalidTableOptionSnafu { - key: key.to_string() - } - ); + ensure!(validate_table_option(key), InvalidTableOptionSnafu { key }); } Ok(options.into()) } diff --git a/src/store-api/src/mito_engine_options.rs b/src/store-api/src/mito_engine_options.rs index e641a1d2fc..0e0f3fdac7 100644 --- a/src/store-api/src/mito_engine_options.rs +++ b/src/store-api/src/mito_engine_options.rs @@ -33,6 +33,7 @@ pub fn is_mito_engine_option_key(key: &str) -> bool { "compaction.twcs.max_active_window_files", "compaction.twcs.max_inactive_window_runs", "compaction.twcs.max_inactive_window_files", + "compaction.twcs.max_output_file_size", "compaction.twcs.time_window", "compaction.twcs.remote_compaction", "compaction.twcs.fallback_to_local", From 3a46c1b2352ba52955653dd0e33f597f1c02171d Mon Sep 17 00:00:00 2001 From: JohnsonLee <0xjohnsonlee@gmail.com> Date: Sat, 28 Sep 2024 07:15:48 +0800 Subject: [PATCH 041/128] fix: use information_schema returns Unknown database (#4774) * fix: use information_schema returns Unknown database 'information_schema' * test: make sure 'use information_schma' successful --- src/frontend/src/instance.rs | 1 - .../cases/standalone/common/system/information_schema.result | 4 ++++ tests/cases/standalone/common/system/information_schema.sql | 2 ++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index dcc56f7581..fc46ae9e59 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -359,7 +359,6 @@ impl SqlQueryHandler for Instance { .schema_exists(catalog, schema, None) .await .context(error::CatalogSnafu) - .map(|b| b && !self.catalog_manager.is_reserved_schema_name(schema)) } } diff --git a/tests/cases/standalone/common/system/information_schema.result b/tests/cases/standalone/common/system/information_schema.result index 882607f842..4264bd8df6 100644 --- a/tests/cases/standalone/common/system/information_schema.result +++ b/tests/cases/standalone/common/system/information_schema.result @@ -434,6 +434,10 @@ database my_db; Affected Rows: 1 +use information_schema; + +Affected Rows: 0 + use my_db; Affected Rows: 0 diff --git a/tests/cases/standalone/common/system/information_schema.sql b/tests/cases/standalone/common/system/information_schema.sql index 49007cb44b..b913050e70 100644 --- a/tests/cases/standalone/common/system/information_schema.sql +++ b/tests/cases/standalone/common/system/information_schema.sql @@ -14,6 +14,8 @@ select * from information_schema.columns order by table_schema, table_name, colu create database my_db; +use information_schema; + use my_db; create table foo From d6be44bc7f57c0ba6ea72fa00846d4c765fd166c Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Sat, 28 Sep 2024 09:39:29 +0800 Subject: [PATCH 042/128] fix: dead loop on detecting postgres ssl handshake (#4778) --- Cargo.lock | 4 ++-- src/servers/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3d5783dcd4..25ba1eb378 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7765,9 +7765,9 @@ dependencies = [ [[package]] name = "pgwire" -version = "0.24.2" +version = "0.24.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed4ca46dd335b3a030d977be54dfe121b1b9fe22aa8bbd69161ac2434524fc68" +checksum = "4895c8e98cbe81496692ae3d2262a9fb0d26302af7c9ea483cf708000106aa0a" dependencies = [ "async-trait", "bytes", diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index b94fa17d44..0de8d0904a 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -71,7 +71,7 @@ openmetrics-parser = "0.4" opensrv-mysql = { git = "https://github.com/datafuselabs/opensrv", rev = "6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb" } opentelemetry-proto.workspace = true parking_lot = "0.12" -pgwire = { version = "0.24.2", default-features = false, features = ["server-api-ring"] } +pgwire = { version = "0.24.3", default-features = false, features = ["server-api-ring"] } pin-project = "1.0" pipeline.workspace = true postgres-types = { version = "0.2", features = ["with-chrono-0_4", "with-serde_json-1"] } From cedbbcf2b8262037225218a130002abd3fb1804c Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Sun, 29 Sep 2024 03:27:31 +0800 Subject: [PATCH 043/128] fix: update pgwire for potential issue with connection establish (#4783) --- Cargo.lock | 4 ++-- src/servers/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 25ba1eb378..2ae89afc74 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7765,9 +7765,9 @@ dependencies = [ [[package]] name = "pgwire" -version = "0.24.3" +version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4895c8e98cbe81496692ae3d2262a9fb0d26302af7c9ea483cf708000106aa0a" +checksum = "5e63bc3945a17010ff93677589c656c5e8fb4183b00bc86360de8e187d2a86cb" dependencies = [ "async-trait", "bytes", diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 0de8d0904a..725ff497a4 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -71,7 +71,7 @@ openmetrics-parser = "0.4" opensrv-mysql = { git = "https://github.com/datafuselabs/opensrv", rev = "6bbc3b65e6b19212c4f7fc4f40c20daf6f452deb" } opentelemetry-proto.workspace = true parking_lot = "0.12" -pgwire = { version = "0.24.3", default-features = false, features = ["server-api-ring"] } +pgwire = { version = "0.25.0", default-features = false, features = ["server-api-ring"] } pin-project = "1.0" pipeline.workspace = true postgres-types = { version = "0.2", features = ["with-chrono-0_4", "with-serde_json-1"] } From d9f2f0ccf0dde73341a6959b36248205bd764c49 Mon Sep 17 00:00:00 2001 From: LFC <990479+MichaelScofield@users.noreply.github.com> Date: Sun, 29 Sep 2024 11:38:50 +0800 Subject: [PATCH 044/128] feat: add a new status code for "external" errors (#4775) * feat: add a new status code for "external" errors * Update src/auth/src/error.rs Co-authored-by: shuiyisong <113876041+shuiyisong@users.noreply.github.com> * support mysql cli cleartext auth * resolve PR comments --------- Co-authored-by: shuiyisong <113876041+shuiyisong@users.noreply.github.com> --- src/auth/src/common.rs | 10 +++++++ src/auth/src/error.rs | 2 +- src/auth/src/user_provider.rs | 5 ++++ src/common/error/src/status_code.rs | 11 +++++--- src/servers/src/http/error_result.rs | 3 ++- src/servers/src/mysql/handler.rs | 36 ++++++++++++++++++++++++- src/servers/src/mysql/writer.rs | 2 +- src/servers/src/postgres/types/error.rs | 1 + 8 files changed, 63 insertions(+), 7 deletions(-) diff --git a/src/auth/src/common.rs b/src/auth/src/common.rs index 3aad89920d..8a13e3fc3d 100644 --- a/src/auth/src/common.rs +++ b/src/auth/src/common.rs @@ -75,6 +75,16 @@ pub enum Password<'a> { PgMD5(HashedPassword<'a>, Salt<'a>), } +impl Password<'_> { + pub fn r#type(&self) -> &str { + match self { + Password::PlainText(_) => "plain_text", + Password::MysqlNativePassword(_, _) => "mysql_native_password", + Password::PgMD5(_, _) => "pg_md5", + } + } +} + pub fn auth_mysql( auth_data: HashedPassword, salt: Salt, diff --git a/src/auth/src/error.rs b/src/auth/src/error.rs index 281c45234d..7ed748559b 100644 --- a/src/auth/src/error.rs +++ b/src/auth/src/error.rs @@ -89,7 +89,7 @@ impl ErrorExt for Error { Error::FileWatch { .. } => StatusCode::InvalidArguments, Error::InternalState { .. } => StatusCode::Unexpected, Error::Io { .. } => StatusCode::StorageUnavailable, - Error::AuthBackend { .. } => StatusCode::Internal, + Error::AuthBackend { source, .. } => source.status_code(), Error::UserNotFound { .. } => StatusCode::UserNotFound, Error::UnsupportedPasswordType { .. } => StatusCode::UnsupportedPasswordType, diff --git a/src/auth/src/user_provider.rs b/src/auth/src/user_provider.rs index b00f3cf29d..526e72b775 100644 --- a/src/auth/src/user_provider.rs +++ b/src/auth/src/user_provider.rs @@ -57,6 +57,11 @@ pub trait UserProvider: Send + Sync { self.authorize(catalog, schema, &user_info).await?; Ok(user_info) } + + /// Returns whether this user provider implementation is backed by an external system. + fn external(&self) -> bool { + false + } } fn load_credential_from_file(filepath: &str) -> Result>>> { diff --git a/src/common/error/src/status_code.rs b/src/common/error/src/status_code.rs index 11b4c7587e..239871e1d1 100644 --- a/src/common/error/src/status_code.rs +++ b/src/common/error/src/status_code.rs @@ -38,6 +38,8 @@ pub enum StatusCode { Cancelled = 1005, /// Illegal state, can be exposed to users. IllegalState = 1006, + /// Caused by some error originated from external system. + External = 1007, // ====== End of common status code ================ // ====== Begin of SQL related status code ========= @@ -162,7 +164,8 @@ impl StatusCode { | StatusCode::InvalidAuthHeader | StatusCode::AccessDenied | StatusCode::PermissionDenied - | StatusCode::RequestOutdated => false, + | StatusCode::RequestOutdated + | StatusCode::External => false, } } @@ -177,7 +180,9 @@ impl StatusCode { | StatusCode::IllegalState | StatusCode::EngineExecuteQuery | StatusCode::StorageUnavailable - | StatusCode::RuntimeResourcesExhausted => true, + | StatusCode::RuntimeResourcesExhausted + | StatusCode::External => true, + StatusCode::Success | StatusCode::Unsupported | StatusCode::InvalidArguments @@ -256,7 +261,7 @@ macro_rules! define_into_tonic_status { pub fn status_to_tonic_code(status_code: StatusCode) -> Code { match status_code { StatusCode::Success => Code::Ok, - StatusCode::Unknown => Code::Unknown, + StatusCode::Unknown | StatusCode::External => Code::Unknown, StatusCode::Unsupported => Code::Unimplemented, StatusCode::Unexpected | StatusCode::IllegalState diff --git a/src/servers/src/http/error_result.rs b/src/servers/src/http/error_result.rs index 518f16cde0..40766d2cbf 100644 --- a/src/servers/src/http/error_result.rs +++ b/src/servers/src/http/error_result.rs @@ -122,7 +122,8 @@ pub fn status_code_to_http_status(status_code: &StatusCode) -> HttpStatusCode { StatusCode::RegionNotReady | StatusCode::TableUnavailable | StatusCode::RegionBusy - | StatusCode::StorageUnavailable => HttpStatusCode::SERVICE_UNAVAILABLE, + | StatusCode::StorageUnavailable + | StatusCode::External => HttpStatusCode::SERVICE_UNAVAILABLE, StatusCode::Internal | StatusCode::Unexpected diff --git a/src/servers/src/mysql/handler.rs b/src/servers/src/mysql/handler.rs index 3c0ac36f4c..abf417996c 100644 --- a/src/servers/src/mysql/handler.rs +++ b/src/servers/src/mysql/handler.rs @@ -54,6 +54,9 @@ use crate::mysql::writer::{create_mysql_column, handle_err}; use crate::query_handler::sql::ServerSqlQueryHandlerRef; use crate::SqlPlan; +const MYSQL_NATIVE_PASSWORD: &str = "mysql_native_password"; +const MYSQL_CLEAR_PASSWORD: &str = "mysql_clear_password"; + // An intermediate shim for executing MySQL queries. pub struct MysqlInstanceShim { query_handler: ServerSqlQueryHandlerRef, @@ -219,6 +222,19 @@ impl MysqlInstanceShim { let mut guard = self.prepared_stmts.write(); let _ = guard.remove(&stmt_key); } + + fn auth_plugin(&self) -> &str { + if self + .user_provider + .as_ref() + .map(|x| x.external()) + .unwrap_or(false) + { + MYSQL_CLEAR_PASSWORD + } else { + MYSQL_NATIVE_PASSWORD + } + } } #[async_trait] @@ -229,6 +245,14 @@ impl AsyncMysqlShim for MysqlInstanceShi std::env::var("GREPTIMEDB_MYSQL_SERVER_VERSION").unwrap_or_else(|_| "8.4.2".to_string()) } + fn default_auth_plugin(&self) -> &str { + self.auth_plugin() + } + + async fn auth_plugin_for_username(&self, _user: &[u8]) -> &str { + self.auth_plugin() + } + fn salt(&self) -> [u8; 20] { self.salt } @@ -253,7 +277,17 @@ impl AsyncMysqlShim for MysqlInstanceShi let user_id = Identity::UserId(&username, addr.as_deref()); let password = match auth_plugin { - "mysql_native_password" => Password::MysqlNativePassword(auth_data, salt), + MYSQL_NATIVE_PASSWORD => Password::MysqlNativePassword(auth_data, salt), + MYSQL_CLEAR_PASSWORD => { + // The raw bytes received could be represented in C-like string, ended in '\0'. + // We must "trim" it to get the real password string. + let password = if let &[password @ .., 0] = &auth_data { + password + } else { + auth_data + }; + Password::PlainText(String::from_utf8_lossy(password).to_string().into()) + } other => { error!("Unsupported mysql auth plugin: {}", other); return false; diff --git a/src/servers/src/mysql/writer.rs b/src/servers/src/mysql/writer.rs index d957edaa55..8a5814d8fe 100644 --- a/src/servers/src/mysql/writer.rs +++ b/src/servers/src/mysql/writer.rs @@ -328,7 +328,7 @@ pub fn create_mysql_column_def(schema: &SchemaRef) -> Result> { fn mysql_error_kind(status_code: &StatusCode) -> ErrorKind { match status_code { StatusCode::Success => ErrorKind::ER_YES, - StatusCode::Unknown => ErrorKind::ER_UNKNOWN_ERROR, + StatusCode::Unknown | StatusCode::External => ErrorKind::ER_UNKNOWN_ERROR, StatusCode::Unsupported => ErrorKind::ER_NOT_SUPPORTED_YET, StatusCode::Cancelled => ErrorKind::ER_QUERY_INTERRUPTED, StatusCode::RuntimeResourcesExhausted => ErrorKind::ER_OUT_OF_RESOURCES, diff --git a/src/servers/src/postgres/types/error.rs b/src/servers/src/postgres/types/error.rs index 9e6f570f26..033f4f6b87 100644 --- a/src/servers/src/postgres/types/error.rs +++ b/src/servers/src/postgres/types/error.rs @@ -374,6 +374,7 @@ impl From for PgErrorCode { StatusCode::Unsupported => PgErrorCode::Ec0A000, StatusCode::InvalidArguments => PgErrorCode::Ec22023, StatusCode::Cancelled => PgErrorCode::Ec57000, + StatusCode::External => PgErrorCode::Ec58000, StatusCode::Unknown | StatusCode::Unexpected From 0a82b12d087d29e4a75a70612b72cb4a79e71340 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Sun, 29 Sep 2024 13:54:01 +0800 Subject: [PATCH 045/128] fix(sqlness): sqlness isolation (#4780) * fix: isolate logs * fix: copy cases * fix: clippy --- .../copy/copy_database_from_fs_parquet.result | 10 ++--- .../copy/copy_database_from_fs_parquet.sql | 10 ++--- .../common/copy/copy_from_fs_csv.result | 8 ++-- .../common/copy/copy_from_fs_csv.sql | 8 ++-- .../common/copy/copy_from_fs_json.result | 8 ++-- .../common/copy/copy_from_fs_json.sql | 8 ++-- .../common/copy/copy_from_fs_parquet.result | 18 ++++---- .../common/copy/copy_from_fs_parquet.sql | 16 +++---- .../standalone/common/copy/copy_to_fs.result | 6 +-- .../standalone/common/copy/copy_to_fs.sql | 6 +-- tests/runner/src/env.rs | 44 ++++++++++++++----- tests/runner/src/main.rs | 8 ++-- 12 files changed, 86 insertions(+), 64 deletions(-) diff --git a/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.result b/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.result index 18badf93f1..999271da8d 100644 --- a/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.result +++ b/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.result @@ -6,11 +6,11 @@ INSERT INTO demo(host, cpu, memory, ts) values ('host1', 66.6, 1024, 16552765570 Affected Rows: 2 -COPY DATABASE public TO '/tmp/demo/export/parquet/' WITH (FORMAT="parquet"); +COPY DATABASE public TO '${SQLNESS_HOME}/demo/export/parquet/' WITH (FORMAT="parquet"); Affected Rows: 2 -COPY DATABASE public TO '/tmp/demo/export/parquet_range/' WITH (FORMAT="parquet", start_time='2022-06-15 07:02:37.000Z', end_time='2022-06-15 07:02:37.1Z'); +COPY DATABASE public TO '${SQLNESS_HOME}/demo/export/parquet_range/' WITH (FORMAT="parquet", start_time='2022-06-15 07:02:37.000Z', end_time='2022-06-15 07:02:37.1Z'); Affected Rows: 1 @@ -23,7 +23,7 @@ SELECT * FROM demo ORDER BY ts; ++ ++ -COPY DATABASE public FROM '/tmp/demo/export/parquet/'; +COPY DATABASE public FROM '${SQLNESS_HOME}/demo/export/parquet/'; Affected Rows: 2 @@ -40,7 +40,7 @@ DELETE FROM demo; Affected Rows: 2 -COPY DATABASE public FROM '/tmp/demo/export/parquet_range/'; +COPY DATABASE public FROM '${SQLNESS_HOME}/demo/export/parquet_range/'; Affected Rows: 1 @@ -56,7 +56,7 @@ DELETE FROM demo; Affected Rows: 1 -COPY DATABASE public FROM '/tmp/demo/export/parquet_range/' LIMIT 2; +COPY DATABASE public FROM '${SQLNESS_HOME}/demo/export/parquet_range/' LIMIT 2; Error: 2000(InvalidSyntax), Invalid SQL, error: limit is not supported diff --git a/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.sql b/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.sql index 5d3716dc22..671070e07b 100644 --- a/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.sql +++ b/tests/cases/standalone/common/copy/copy_database_from_fs_parquet.sql @@ -2,26 +2,26 @@ CREATE TABLE demo(host string, cpu double, memory double, ts TIMESTAMP time inde INSERT INTO demo(host, cpu, memory, ts) values ('host1', 66.6, 1024, 1655276557000), ('host2', 88.8, 333.3, 1655276558000); -COPY DATABASE public TO '/tmp/demo/export/parquet/' WITH (FORMAT="parquet"); +COPY DATABASE public TO '${SQLNESS_HOME}/demo/export/parquet/' WITH (FORMAT="parquet"); -COPY DATABASE public TO '/tmp/demo/export/parquet_range/' WITH (FORMAT="parquet", start_time='2022-06-15 07:02:37.000Z', end_time='2022-06-15 07:02:37.1Z'); +COPY DATABASE public TO '${SQLNESS_HOME}/demo/export/parquet_range/' WITH (FORMAT="parquet", start_time='2022-06-15 07:02:37.000Z', end_time='2022-06-15 07:02:37.1Z'); DELETE FROM demo; SELECT * FROM demo ORDER BY ts; -COPY DATABASE public FROM '/tmp/demo/export/parquet/'; +COPY DATABASE public FROM '${SQLNESS_HOME}/demo/export/parquet/'; SELECT * FROM demo ORDER BY ts; DELETE FROM demo; -COPY DATABASE public FROM '/tmp/demo/export/parquet_range/'; +COPY DATABASE public FROM '${SQLNESS_HOME}/demo/export/parquet_range/'; SELECT * FROM demo ORDER BY ts; DELETE FROM demo; -COPY DATABASE public FROM '/tmp/demo/export/parquet_range/' LIMIT 2; +COPY DATABASE public FROM '${SQLNESS_HOME}/demo/export/parquet_range/' LIMIT 2; DROP TABLE demo; diff --git a/tests/cases/standalone/common/copy/copy_from_fs_csv.result b/tests/cases/standalone/common/copy/copy_from_fs_csv.result index 19b3fda3f9..f412677a32 100644 --- a/tests/cases/standalone/common/copy/copy_from_fs_csv.result +++ b/tests/cases/standalone/common/copy/copy_from_fs_csv.result @@ -11,7 +11,7 @@ values Affected Rows: 3 -Copy demo TO '/tmp/demo/export/csv/demo.csv' with (format='csv'); +Copy demo TO '${SQLNESS_HOME}/demo/export/csv/demo.csv' with (format='csv'); Affected Rows: 3 @@ -19,7 +19,7 @@ CREATE TABLE with_filename(host string, cpu double, memory double, ts timestamp Affected Rows: 0 -Copy with_filename FROM '/tmp/demo/export/csv/demo.csv' with (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_filename FROM '${SQLNESS_HOME}/demo/export/csv/demo.csv' with (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); Affected Rows: 2 @@ -36,7 +36,7 @@ CREATE TABLE with_path(host string, cpu double, memory double, ts timestamp time Affected Rows: 0 -Copy with_path FROM '/tmp/demo/export/csv/' with (format='csv', start_time='2023-06-15 07:02:37'); +Copy with_path FROM '${SQLNESS_HOME}/demo/export/csv/' with (format='csv', start_time='2023-06-15 07:02:37'); Affected Rows: 1 @@ -52,7 +52,7 @@ CREATE TABLE with_pattern(host string, cpu double, memory double, ts timestamp t Affected Rows: 0 -Copy with_pattern FROM '/tmp/demo/export/csv/' WITH (pattern = 'demo.*', format='csv', end_time='2025-06-15 07:02:39'); +Copy with_pattern FROM '${SQLNESS_HOME}/demo/export/csv/' WITH (pattern = 'demo.*', format='csv', end_time='2025-06-15 07:02:39'); Affected Rows: 3 diff --git a/tests/cases/standalone/common/copy/copy_from_fs_csv.sql b/tests/cases/standalone/common/copy/copy_from_fs_csv.sql index cd6b91f4f8..f2c6ccf5bd 100644 --- a/tests/cases/standalone/common/copy/copy_from_fs_csv.sql +++ b/tests/cases/standalone/common/copy/copy_from_fs_csv.sql @@ -7,23 +7,23 @@ values ('host2', 88.8, 333.3, 1655276558000), ('host3', 99.9, 444.4, 1722077263000); -Copy demo TO '/tmp/demo/export/csv/demo.csv' with (format='csv'); +Copy demo TO '${SQLNESS_HOME}/demo/export/csv/demo.csv' with (format='csv'); CREATE TABLE with_filename(host string, cpu double, memory double, ts timestamp time index); -Copy with_filename FROM '/tmp/demo/export/csv/demo.csv' with (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_filename FROM '${SQLNESS_HOME}/demo/export/csv/demo.csv' with (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); select * from with_filename order by ts; CREATE TABLE with_path(host string, cpu double, memory double, ts timestamp time index); -Copy with_path FROM '/tmp/demo/export/csv/' with (format='csv', start_time='2023-06-15 07:02:37'); +Copy with_path FROM '${SQLNESS_HOME}/demo/export/csv/' with (format='csv', start_time='2023-06-15 07:02:37'); select * from with_path order by ts; CREATE TABLE with_pattern(host string, cpu double, memory double, ts timestamp time index); -Copy with_pattern FROM '/tmp/demo/export/csv/' WITH (pattern = 'demo.*', format='csv', end_time='2025-06-15 07:02:39'); +Copy with_pattern FROM '${SQLNESS_HOME}/demo/export/csv/' WITH (pattern = 'demo.*', format='csv', end_time='2025-06-15 07:02:39'); select * from with_pattern order by ts; diff --git a/tests/cases/standalone/common/copy/copy_from_fs_json.result b/tests/cases/standalone/common/copy/copy_from_fs_json.result index bd71b5d624..e1e3a810d5 100644 --- a/tests/cases/standalone/common/copy/copy_from_fs_json.result +++ b/tests/cases/standalone/common/copy/copy_from_fs_json.result @@ -11,7 +11,7 @@ values Affected Rows: 3 -Copy demo TO '/tmp/demo/export/json/demo.json' with (format='json'); +Copy demo TO '${SQLNESS_HOME}/demo/export/json/demo.json' with (format='json'); Affected Rows: 3 @@ -19,7 +19,7 @@ CREATE TABLE with_filename(host string, cpu double, memory double, ts timestamp Affected Rows: 0 -Copy with_filename FROM '/tmp/demo/export/json/demo.json' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_filename FROM '${SQLNESS_HOME}/demo/export/json/demo.json' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); Affected Rows: 2 @@ -36,7 +36,7 @@ CREATE TABLE with_path(host string, cpu double, memory double, ts timestamp time Affected Rows: 0 -Copy with_path FROM '/tmp/demo/export/json/' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_path FROM '${SQLNESS_HOME}/demo/export/json/' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); Affected Rows: 2 @@ -53,7 +53,7 @@ CREATE TABLE with_pattern(host string, cpu double, memory double, ts timestamp t Affected Rows: 0 -Copy with_pattern FROM '/tmp/demo/export/json/' WITH (pattern = 'demo.*',format='json', end_time='2022-06-15 07:02:39'); +Copy with_pattern FROM '${SQLNESS_HOME}/demo/export/json/' WITH (pattern = 'demo.*',format='json', end_time='2022-06-15 07:02:39'); Affected Rows: 2 diff --git a/tests/cases/standalone/common/copy/copy_from_fs_json.sql b/tests/cases/standalone/common/copy/copy_from_fs_json.sql index c182bb82dc..55e8a55ebf 100644 --- a/tests/cases/standalone/common/copy/copy_from_fs_json.sql +++ b/tests/cases/standalone/common/copy/copy_from_fs_json.sql @@ -7,23 +7,23 @@ values ('host2', 88.8, 333.3, 1655276558000), ('host3', 99.9, 444.4, 1722077263000); -Copy demo TO '/tmp/demo/export/json/demo.json' with (format='json'); +Copy demo TO '${SQLNESS_HOME}/demo/export/json/demo.json' with (format='json'); CREATE TABLE with_filename(host string, cpu double, memory double, ts timestamp time index); -Copy with_filename FROM '/tmp/demo/export/json/demo.json' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_filename FROM '${SQLNESS_HOME}/demo/export/json/demo.json' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); select * from with_filename order by ts; CREATE TABLE with_path(host string, cpu double, memory double, ts timestamp time index); -Copy with_path FROM '/tmp/demo/export/json/' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_path FROM '${SQLNESS_HOME}/demo/export/json/' with (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); select * from with_path order by ts; CREATE TABLE with_pattern(host string, cpu double, memory double, ts timestamp time index); -Copy with_pattern FROM '/tmp/demo/export/json/' WITH (pattern = 'demo.*',format='json', end_time='2022-06-15 07:02:39'); +Copy with_pattern FROM '${SQLNESS_HOME}/demo/export/json/' WITH (pattern = 'demo.*',format='json', end_time='2022-06-15 07:02:39'); select * from with_pattern order by ts; diff --git a/tests/cases/standalone/common/copy/copy_from_fs_parquet.result b/tests/cases/standalone/common/copy/copy_from_fs_parquet.result index 82299d48b3..44e1422d5b 100644 --- a/tests/cases/standalone/common/copy/copy_from_fs_parquet.result +++ b/tests/cases/standalone/common/copy/copy_from_fs_parquet.result @@ -24,11 +24,11 @@ values Affected Rows: 3 -Copy demo TO '/tmp/demo/export/parquet_files/demo.parquet'; +Copy demo TO '${SQLNESS_HOME}/demo/export/parquet_files/demo.parquet'; Affected Rows: 3 -Copy demo_2 TO '/tmp/demo/export/parquet_files/demo_2.parquet'; +Copy demo_2 TO '${SQLNESS_HOME}/demo/export/parquet_files/demo_2.parquet'; Affected Rows: 3 @@ -36,7 +36,7 @@ CREATE TABLE with_filename(host string, cpu double, memory double, ts timestamp Affected Rows: 0 -Copy with_filename FROM '/tmp/demo/export/parquet_files/demo.parquet' with (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_filename FROM '${SQLNESS_HOME}/demo/export/parquet_files/demo.parquet' with (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); Affected Rows: 2 @@ -53,7 +53,7 @@ CREATE TABLE with_path(host string, cpu double, memory double, ts timestamp time Affected Rows: 0 -Copy with_path FROM '/tmp/demo/export/parquet_files/'; +Copy with_path FROM '${SQLNESS_HOME}/demo/export/parquet_files/'; Affected Rows: 6 @@ -74,7 +74,7 @@ CREATE TABLE with_pattern(host string, cpu double, memory double, ts timestamp t Affected Rows: 0 -Copy with_pattern FROM '/tmp/demo/export/parquet_files/' WITH (PATTERN = 'demo.*', start_time='2022-06-15 07:02:39'); +Copy with_pattern FROM '${SQLNESS_HOME}/demo/export/parquet_files/' WITH (PATTERN = 'demo.*', start_time='2022-06-15 07:02:39'); Affected Rows: 2 @@ -91,7 +91,7 @@ CREATE TABLE without_limit_rows(host string, cpu double, memory double, ts times Affected Rows: 0 -Copy without_limit_rows FROM '/tmp/demo/export/parquet_files/'; +Copy without_limit_rows FROM '${SQLNESS_HOME}/demo/export/parquet_files/'; Affected Rows: 6 @@ -107,7 +107,7 @@ CREATE TABLE with_limit_rows_segment(host string, cpu double, memory double, ts Affected Rows: 0 -Copy with_limit_rows_segment FROM '/tmp/demo/export/parquet_files/' LIMIT 3; +Copy with_limit_rows_segment FROM '${SQLNESS_HOME}/demo/export/parquet_files/' LIMIT 3; Affected Rows: 3 @@ -119,9 +119,9 @@ select count(*) from with_limit_rows_segment; | 3 | +----------+ -Copy with_limit_rows_segment FROM '/tmp/demo/export/parquet_files/' LIMIT hello; +Copy with_limit_rows_segment FROM '${SQLNESS_HOME}/demo/export/parquet_files/' LIMIT hello; -Error: 2000(InvalidSyntax), Unexpected token while parsing SQL statement, expected: 'the number of maximum rows', found: ;: sql parser error: Expected literal int, found: hello at Line: 1, Column 75 +Error: 2000(InvalidSyntax), Unexpected token while parsing SQL statement, expected: 'the number of maximum rows', found: ;: sql parser error: Expected literal int, found: hello at Line: 1, Column 86 drop table demo; diff --git a/tests/cases/standalone/common/copy/copy_from_fs_parquet.sql b/tests/cases/standalone/common/copy/copy_from_fs_parquet.sql index 10319e1281..0db5e119d8 100644 --- a/tests/cases/standalone/common/copy/copy_from_fs_parquet.sql +++ b/tests/cases/standalone/common/copy/copy_from_fs_parquet.sql @@ -16,41 +16,41 @@ values ('host5', 99.9, 444.4, 1655276556000), ('host6', 222.2, 555.5, 1722077264000); -Copy demo TO '/tmp/demo/export/parquet_files/demo.parquet'; +Copy demo TO '${SQLNESS_HOME}/demo/export/parquet_files/demo.parquet'; -Copy demo_2 TO '/tmp/demo/export/parquet_files/demo_2.parquet'; +Copy demo_2 TO '${SQLNESS_HOME}/demo/export/parquet_files/demo_2.parquet'; CREATE TABLE with_filename(host string, cpu double, memory double, ts timestamp time index); -Copy with_filename FROM '/tmp/demo/export/parquet_files/demo.parquet' with (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); +Copy with_filename FROM '${SQLNESS_HOME}/demo/export/parquet_files/demo.parquet' with (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:39'); select * from with_filename order by ts; CREATE TABLE with_path(host string, cpu double, memory double, ts timestamp time index); -Copy with_path FROM '/tmp/demo/export/parquet_files/'; +Copy with_path FROM '${SQLNESS_HOME}/demo/export/parquet_files/'; select * from with_path order by ts; CREATE TABLE with_pattern(host string, cpu double, memory double, ts timestamp time index); -Copy with_pattern FROM '/tmp/demo/export/parquet_files/' WITH (PATTERN = 'demo.*', start_time='2022-06-15 07:02:39'); +Copy with_pattern FROM '${SQLNESS_HOME}/demo/export/parquet_files/' WITH (PATTERN = 'demo.*', start_time='2022-06-15 07:02:39'); select * from with_pattern order by ts; CREATE TABLE without_limit_rows(host string, cpu double, memory double, ts timestamp time index); -Copy without_limit_rows FROM '/tmp/demo/export/parquet_files/'; +Copy without_limit_rows FROM '${SQLNESS_HOME}/demo/export/parquet_files/'; select count(*) from without_limit_rows; CREATE TABLE with_limit_rows_segment(host string, cpu double, memory double, ts timestamp time index); -Copy with_limit_rows_segment FROM '/tmp/demo/export/parquet_files/' LIMIT 3; +Copy with_limit_rows_segment FROM '${SQLNESS_HOME}/demo/export/parquet_files/' LIMIT 3; select count(*) from with_limit_rows_segment; -Copy with_limit_rows_segment FROM '/tmp/demo/export/parquet_files/' LIMIT hello; +Copy with_limit_rows_segment FROM '${SQLNESS_HOME}/demo/export/parquet_files/' LIMIT hello; drop table demo; diff --git a/tests/cases/standalone/common/copy/copy_to_fs.result b/tests/cases/standalone/common/copy/copy_to_fs.result index f6bdc22a20..a3b73ab4ad 100644 --- a/tests/cases/standalone/common/copy/copy_to_fs.result +++ b/tests/cases/standalone/common/copy/copy_to_fs.result @@ -6,15 +6,15 @@ insert into demo(host, cpu, memory, ts) values ('host1', 66.6, 1024, 16552765570 Affected Rows: 2 -COPY demo TO '/tmp/export/demo.parquet' WITH (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); +COPY demo TO '${SQLNESS_HOME}/export/demo.parquet' WITH (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); Affected Rows: 1 -COPY demo TO '/tmp/export/demo.csv' WITH (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); +COPY demo TO '${SQLNESS_HOME}/export/demo.csv' WITH (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); Affected Rows: 1 -COPY demo TO '/tmp/export/demo.json' WITH (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); +COPY demo TO '${SQLNESS_HOME}/export/demo.json' WITH (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); Affected Rows: 1 diff --git a/tests/cases/standalone/common/copy/copy_to_fs.sql b/tests/cases/standalone/common/copy/copy_to_fs.sql index d095c2d795..82e3ae1fb3 100644 --- a/tests/cases/standalone/common/copy/copy_to_fs.sql +++ b/tests/cases/standalone/common/copy/copy_to_fs.sql @@ -2,10 +2,10 @@ CREATE TABLE demo(host string, cpu DOUBLE, memory DOUBLE, ts TIMESTAMP TIME INDE insert into demo(host, cpu, memory, ts) values ('host1', 66.6, 1024, 1655276557000), ('host2', 88.8, 333.3, 1655276558000); -COPY demo TO '/tmp/export/demo.parquet' WITH (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); +COPY demo TO '${SQLNESS_HOME}/export/demo.parquet' WITH (start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); -COPY demo TO '/tmp/export/demo.csv' WITH (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); +COPY demo TO '${SQLNESS_HOME}/export/demo.csv' WITH (format='csv', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); -COPY demo TO '/tmp/export/demo.json' WITH (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); +COPY demo TO '${SQLNESS_HOME}/export/demo.json' WITH (format='json', start_time='2022-06-15 07:02:37', end_time='2022-06-15 07:02:38'); drop table demo; diff --git a/tests/runner/src/env.rs b/tests/runner/src/env.rs index d8ed5713f5..9d71489f8c 100644 --- a/tests/runner/src/env.rs +++ b/tests/runner/src/env.rs @@ -54,7 +54,7 @@ pub enum WalConfig { #[derive(Clone)] pub struct Env { - data_home: PathBuf, + sqlness_home: PathBuf, server_addr: Option, wal: WalConfig, @@ -69,6 +69,7 @@ impl EnvController for Env { type DB = GreptimeDB; async fn start(&self, mode: &str, _config: Option<&Path>) -> Self::DB { + std::env::set_var("SQLNESS_HOME", self.sqlness_home.display().to_string()); match mode { "standalone" => self.start_standalone().await, "distributed" => self.start_distributed().await, @@ -90,7 +91,7 @@ impl Env { bins_dir: Option, ) -> Self { Self { - data_home, + sqlness_home: data_home, server_addr, wal, bins_dir: Arc::new(Mutex::new(bins_dir)), @@ -204,26 +205,30 @@ impl Env { "standalone" => "greptime-sqlness-standalone.log".to_string(), _ => panic!("Unexpected subcommand: {subcommand}"), }; - let log_file_name = self.data_home.join(log_file_name).display().to_string(); + let stdout_file_name = self.sqlness_home.join(log_file_name).display().to_string(); - println!("{subcommand} log file at {log_file_name}"); + println!("{subcommand} log file at {stdout_file_name}"); - let log_file = OpenOptions::new() + let stdout_file = OpenOptions::new() .create(true) .write(true) .truncate(truncate_log) .append(!truncate_log) - .open(log_file_name) + .open(stdout_file_name) .unwrap(); let (args, check_ip_addr) = match subcommand { "datanode" => self.datanode_start_args(db_ctx), - "flownode" => self.flownode_start_args(db_ctx), + "flownode" => self.flownode_start_args(db_ctx, &self.sqlness_home), "standalone" => { let args = vec![ DEFAULT_LOG_LEVEL.to_string(), subcommand.to_string(), "start".to_string(), + format!( + "--log-dir={}/greptimedb-flownode/logs", + self.sqlness_home.display() + ), "-c".to_string(), self.generate_config_file(subcommand, db_ctx), "--http-addr=127.0.0.1:5002".to_string(), @@ -237,6 +242,10 @@ impl Env { "start".to_string(), "--metasrv-addrs=127.0.0.1:3002".to_string(), "--http-addr=127.0.0.1:5003".to_string(), + format!( + "--log-dir={}/greptimedb-frontend/logs", + self.sqlness_home.display() + ), ]; (args, SERVER_ADDR.to_string()) } @@ -250,6 +259,10 @@ impl Env { "--enable-region-failover".to_string(), "false".to_string(), "--http-addr=127.0.0.1:5002".to_string(), + format!( + "--log-dir={}/greptimedb-metasrv/logs", + self.sqlness_home.display() + ), "-c".to_string(), self.generate_config_file(subcommand, db_ctx), ]; @@ -278,7 +291,7 @@ impl Env { .current_dir(bins_dir) .env("TZ", "UTC") .args(args) - .stdout(log_file) + .stdout(stdout_file) .spawn() .unwrap_or_else(|error| { panic!("Failed to start the DB with subcommand {subcommand},Error: {error}") @@ -296,7 +309,7 @@ impl Env { let id = db_ctx.datanode_id(); let data_home = self - .data_home + .sqlness_home .join(format!("greptimedb_datanode_{}_{id}", db_ctx.time)); let subcommand = "datanode"; @@ -308,6 +321,7 @@ impl Env { args.push(format!("--rpc-addr=127.0.0.1:410{id}")); args.push(format!("--http-addr=127.0.0.1:430{id}")); args.push(format!("--data-home={}", data_home.display())); + args.push(format!("--log-dir={}/logs", data_home.display())); args.push(format!("--node-id={id}")); args.push("-c".to_string()); args.push(self.generate_config_file(subcommand, db_ctx)); @@ -315,7 +329,11 @@ impl Env { (args, format!("127.0.0.1:410{id}")) } - fn flownode_start_args(&self, _db_ctx: &GreptimeDBContext) -> (Vec, String) { + fn flownode_start_args( + &self, + _db_ctx: &GreptimeDBContext, + sqlness_home: &Path, + ) -> (Vec, String) { let id = 0; let subcommand = "flownode"; @@ -326,6 +344,10 @@ impl Env { ]; args.push(format!("--rpc-addr=127.0.0.1:680{id}")); args.push(format!("--node-id={id}")); + args.push(format!( + "--log-dir={}/greptimedb-flownode/logs", + sqlness_home.display() + )); args.push("--metasrv-addrs=127.0.0.1:3002".to_string()); (args, format!("127.0.0.1:680{id}")) } @@ -387,7 +409,7 @@ impl Env { kafka_wal_broker_endpoints: String, } - let data_home = self.data_home.join(format!("greptimedb-{subcommand}")); + let data_home = self.sqlness_home.join(format!("greptimedb-{subcommand}")); std::fs::create_dir_all(data_home.as_path()).unwrap(); let wal_dir = data_home.join("wal").display().to_string(); diff --git a/tests/runner/src/main.rs b/tests/runner/src/main.rs index 1684962645..4474676802 100644 --- a/tests/runner/src/main.rs +++ b/tests/runner/src/main.rs @@ -82,7 +82,7 @@ async fn main() { .prefix("sqlness") .tempdir() .unwrap(); - let data_home = temp_dir.into_path(); + let sqlness_home = temp_dir.into_path(); let config = ConfigBuilder::default() .case_dir(util::get_case_dir(args.case_dir)) @@ -107,13 +107,13 @@ async fn main() { let runner = Runner::new( config, - Env::new(data_home.clone(), args.server_addr, wal, args.bins_dir), + Env::new(sqlness_home.clone(), args.server_addr, wal, args.bins_dir), ); runner.run().await.unwrap(); // clean up and exit if !args.preserve_state { - println!("Removing state in {:?}", data_home); - tokio::fs::remove_dir_all(data_home).await.unwrap(); + println!("Removing state in {:?}", sqlness_home); + tokio::fs::remove_dir_all(sqlness_home).await.unwrap(); } } From 50cb59587db67bb0248ce51d2f93eac860060b14 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Sun, 29 Sep 2024 14:27:35 +0800 Subject: [PATCH 046/128] chore: replace anymap with anymap2 (#4781) --- Cargo.lock | 2 +- src/common/base/Cargo.toml | 2 +- src/common/base/src/plugins.rs | 13 +++++++------ 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2ae89afc74..4533257660 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1868,7 +1868,7 @@ checksum = "55b672471b4e9f9e95499ea597ff64941a309b2cdbffcc46f2cc5e2d971fd335" name = "common-base" version = "0.9.3" dependencies = [ - "anymap", + "anymap2", "async-trait", "bitvec", "bytes", diff --git a/src/common/base/Cargo.toml b/src/common/base/Cargo.toml index 5afbc3b88c..5dfde47bf2 100644 --- a/src/common/base/Cargo.toml +++ b/src/common/base/Cargo.toml @@ -8,7 +8,7 @@ license.workspace = true workspace = true [dependencies] -anymap = "1.0.0-beta.2" +anymap2 = "0.13" async-trait.workspace = true bitvec = "1.0" bytes.workspace = true diff --git a/src/common/base/src/plugins.rs b/src/common/base/src/plugins.rs index 84d78b0c91..98b40cd059 100644 --- a/src/common/base/src/plugins.rs +++ b/src/common/base/src/plugins.rs @@ -12,20 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::any::Any; use std::sync::{Arc, RwLock, RwLockReadGuard, RwLockWriteGuard}; -/// [`Plugins`] is a wrapper of [AnyMap](https://github.com/chris-morgan/anymap) and provides a thread-safe way to store and retrieve plugins. +use anymap2::SendSyncAnyMap; + +/// [`Plugins`] is a wrapper of [anymap2](https://github.com/azriel91/anymap2) and provides a thread-safe way to store and retrieve plugins. /// Make it Cloneable and we can treat it like an Arc struct. #[derive(Default, Clone)] pub struct Plugins { - inner: Arc>>, + inner: Arc>, } impl Plugins { pub fn new() -> Self { Self { - inner: Arc::new(RwLock::new(anymap::Map::new())), + inner: Arc::new(RwLock::new(SendSyncAnyMap::new())), } } @@ -61,11 +62,11 @@ impl Plugins { self.read().is_empty() } - fn read(&self) -> RwLockReadGuard> { + fn read(&self) -> RwLockReadGuard { self.inner.read().unwrap() } - fn write(&self) -> RwLockWriteGuard> { + fn write(&self) -> RwLockWriteGuard { self.inner.write().unwrap() } } From cd552021362c45f2ef0292ac6e85fb775d2d8449 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Sun, 29 Sep 2024 15:14:48 +0800 Subject: [PATCH 047/128] feat: unordered scanner scans data by time ranges (#4757) * feat: define range meta * feat: group ranges * feat: split range * feat: build ranges from the scan input * feat: get partition range from range meta * feat: build file range * feat: unordered scan read by ranges * feat: wip for mem ranges * feat: build ranges * feat: remove unused codes * chore: update comments * feat: update metrics * chore: address review comments * chore: debug assertion --- src/mito2/src/memtable.rs | 12 +- src/mito2/src/memtable/bulk.rs | 3 +- src/mito2/src/memtable/partition_tree.rs | 7 +- src/mito2/src/memtable/time_series.rs | 6 +- src/mito2/src/read.rs | 5 + src/mito2/src/read/dedup.rs | 1 - src/mito2/src/read/range.rs | 266 ++++++++++++++++ src/mito2/src/read/scan_region.rs | 298 +++++++++++++++--- src/mito2/src/read/seq_scan.rs | 7 +- src/mito2/src/read/unordered_scan.rs | 374 ++++++++++------------- src/mito2/src/sst/parquet.rs | 2 +- src/mito2/src/sst/parquet/reader.rs | 1 + src/mito2/src/test_util/memtable_util.rs | 5 +- 13 files changed, 711 insertions(+), 276 deletions(-) create mode 100644 src/mito2/src/read/range.rs diff --git a/src/mito2/src/memtable.rs b/src/mito2/src/memtable.rs index e79a3290be..668eb51023 100644 --- a/src/mito2/src/memtable.rs +++ b/src/mito2/src/memtable.rs @@ -14,6 +14,7 @@ //! Memtables are write buffers for regions. +use std::collections::BTreeMap; use std::fmt; use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use std::sync::Arc; @@ -71,6 +72,8 @@ pub struct MemtableStats { time_range: Option<(Timestamp, Timestamp)>, /// Total rows in memtable num_rows: usize, + /// Total number of ranges in the memtable. + num_ranges: usize, } impl MemtableStats { @@ -95,6 +98,11 @@ impl MemtableStats { pub fn num_rows(&self) -> usize { self.num_rows } + + /// Returns the number of ranges in the memtable. + pub fn num_ranges(&self) -> usize { + self.num_ranges + } } pub type BoxedBatchIterator = Box> + Send>; @@ -123,11 +131,12 @@ pub trait Memtable: Send + Sync + fmt::Debug { ) -> Result; /// Returns the ranges in the memtable. + /// The returned map contains the range id and the range after applying the predicate. fn ranges( &self, projection: Option<&[ColumnId]>, predicate: Option, - ) -> Vec; + ) -> BTreeMap; /// Returns true if the memtable is empty. fn is_empty(&self) -> bool; @@ -332,7 +341,6 @@ impl MemtableRangeContext { pub struct MemtableRange { /// Shared context. context: MemtableRangeContextRef, - // TODO(yingwen): Id to identify the range in the memtable. } impl MemtableRange { diff --git a/src/mito2/src/memtable/bulk.rs b/src/mito2/src/memtable/bulk.rs index 3ed2ed1347..46e757f3df 100644 --- a/src/mito2/src/memtable/bulk.rs +++ b/src/mito2/src/memtable/bulk.rs @@ -14,6 +14,7 @@ //! Memtable implementation for bulk load +use std::collections::BTreeMap; use std::sync::{Arc, RwLock}; use store_api::metadata::RegionMetadataRef; @@ -67,7 +68,7 @@ impl Memtable for BulkMemtable { &self, _projection: Option<&[ColumnId]>, _predicate: Option, - ) -> Vec { + ) -> BTreeMap { todo!() } diff --git a/src/mito2/src/memtable/partition_tree.rs b/src/mito2/src/memtable/partition_tree.rs index e320503886..3309e66b7e 100644 --- a/src/mito2/src/memtable/partition_tree.rs +++ b/src/mito2/src/memtable/partition_tree.rs @@ -24,6 +24,7 @@ mod shard; mod shard_builder; mod tree; +use std::collections::BTreeMap; use std::fmt; use std::sync::atomic::{AtomicI64, AtomicUsize, Ordering}; use std::sync::Arc; @@ -176,7 +177,7 @@ impl Memtable for PartitionTreeMemtable { &self, projection: Option<&[ColumnId]>, predicate: Option, - ) -> Vec { + ) -> BTreeMap { let projection = projection.map(|ids| ids.to_vec()); let builder = Box::new(PartitionTreeIterBuilder { tree: self.tree.clone(), @@ -185,7 +186,7 @@ impl Memtable for PartitionTreeMemtable { }); let context = Arc::new(MemtableRangeContext::new(self.id, builder)); - vec![MemtableRange::new(context)] + [(0, MemtableRange::new(context))].into() } fn is_empty(&self) -> bool { @@ -207,6 +208,7 @@ impl Memtable for PartitionTreeMemtable { estimated_bytes, time_range: None, num_rows: 0, + num_ranges: 0, }; } @@ -225,6 +227,7 @@ impl Memtable for PartitionTreeMemtable { estimated_bytes, time_range: Some((min_timestamp, max_timestamp)), num_rows: self.num_rows.load(Ordering::Relaxed), + num_ranges: 1, } } diff --git a/src/mito2/src/memtable/time_series.rs b/src/mito2/src/memtable/time_series.rs index 6d5fbb33a0..d82db55730 100644 --- a/src/mito2/src/memtable/time_series.rs +++ b/src/mito2/src/memtable/time_series.rs @@ -287,7 +287,7 @@ impl Memtable for TimeSeriesMemtable { &self, projection: Option<&[ColumnId]>, predicate: Option, - ) -> Vec { + ) -> BTreeMap { let projection = if let Some(projection) = projection { projection.iter().copied().collect() } else { @@ -305,7 +305,7 @@ impl Memtable for TimeSeriesMemtable { }); let context = Arc::new(MemtableRangeContext::new(self.id, builder)); - vec![MemtableRange::new(context)] + [(0, MemtableRange::new(context))].into() } fn is_empty(&self) -> bool { @@ -327,6 +327,7 @@ impl Memtable for TimeSeriesMemtable { estimated_bytes, time_range: None, num_rows: 0, + num_ranges: 0, }; } let ts_type = self @@ -343,6 +344,7 @@ impl Memtable for TimeSeriesMemtable { estimated_bytes, time_range: Some((min_timestamp, max_timestamp)), num_rows: self.num_rows.load(Ordering::Relaxed), + num_ranges: 1, } } diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index edde28dab8..d9a6fae746 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -20,6 +20,7 @@ pub mod last_row; pub mod merge; pub mod projection; pub(crate) mod prune; +pub(crate) mod range; pub(crate) mod scan_region; pub(crate) mod seq_scan; pub(crate) mod unordered_scan; @@ -753,6 +754,10 @@ pub(crate) struct ScannerMetrics { num_batches: usize, /// Number of rows returned. num_rows: usize, + /// Number of mem ranges scanned. + num_mem_ranges: usize, + /// Number of file ranges scanned. + num_file_ranges: usize, /// Filter related metrics for readers. filter_metrics: ReaderFilterMetrics, } diff --git a/src/mito2/src/read/dedup.rs b/src/mito2/src/read/dedup.rs index ddc96049e7..c77d0c3fab 100644 --- a/src/mito2/src/read/dedup.rs +++ b/src/mito2/src/read/dedup.rs @@ -400,7 +400,6 @@ pub(crate) struct LastNonNull { impl LastNonNull { /// Creates a new strategy with the given `filter_deleted` flag. - #[allow(dead_code)] pub(crate) fn new(filter_deleted: bool) -> Self { Self { buffer: None, diff --git a/src/mito2/src/read/range.rs b/src/mito2/src/read/range.rs new file mode 100644 index 0000000000..8408aad7b2 --- /dev/null +++ b/src/mito2/src/read/range.rs @@ -0,0 +1,266 @@ +// 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. + +//! Structs for partition ranges. + +use smallvec::{smallvec, SmallVec}; + +use crate::memtable::MemtableRef; +use crate::read::scan_region::ScanInput; +use crate::sst::file::{overlaps, FileHandle, FileTimeRange}; +use crate::sst::parquet::DEFAULT_ROW_GROUP_SIZE; + +const ALL_ROW_GROUPS: i64 = -1; + +/// Index to access a row group. +#[derive(Clone, Copy, PartialEq)] +pub(crate) struct RowGroupIndex { + /// Index to the memtable/file. + pub(crate) index: usize, + /// Row group index in the file. + /// Negative index indicates all row groups. + pub(crate) row_group_index: i64, +} + +/// Meta data of a partition range. +/// If the scanner is [UnorderedScan], each meta only has one row group or memtable. +/// If the scanner is [SeqScan], each meta may have multiple row groups and memtables. +pub(crate) struct RangeMeta { + /// The time range of the range. + pub(crate) time_range: FileTimeRange, + /// Indices to memtables or files. + indices: SmallVec<[usize; 2]>, + /// Indices to memtable/file row groups that this range scans. + pub(crate) row_group_indices: SmallVec<[RowGroupIndex; 2]>, + /// Estimated number of rows in the range. This can be 0 if the statistics are not available. + pub(crate) num_rows: usize, +} + +impl RangeMeta { + /// Creates a list of ranges from the `input` for seq scan. + pub(crate) fn seq_scan_ranges(input: &ScanInput) -> Vec { + let mut ranges = Vec::with_capacity(input.memtables.len() + input.files.len()); + Self::push_seq_mem_ranges(&input.memtables, &mut ranges); + Self::push_seq_file_ranges(input.memtables.len(), &input.files, &mut ranges); + + let ranges = group_ranges_for_seq_scan(ranges); + maybe_split_ranges_for_seq_scan(ranges) + } + + /// Creates a list of ranges from the `input` for unordered scan. + pub(crate) fn unordered_scan_ranges(input: &ScanInput) -> Vec { + let mut ranges = Vec::with_capacity(input.memtables.len() + input.files.len()); + Self::push_unordered_mem_ranges(&input.memtables, &mut ranges); + Self::push_unordered_file_ranges(input.memtables.len(), &input.files, &mut ranges); + + ranges + } + + /// Returns true if the time range of given `meta` overlaps with the time range of this meta. + pub(crate) fn overlaps(&self, meta: &RangeMeta) -> bool { + overlaps(&self.time_range, &meta.time_range) + } + + /// Merges given `meta` to this meta. + /// It assumes that the time ranges overlap and they don't have the same file or memtable index. + pub(crate) fn merge(&mut self, mut other: RangeMeta) { + debug_assert!(self.overlaps(&other)); + debug_assert!(self.indices.iter().all(|idx| !other.indices.contains(idx))); + debug_assert!(self + .row_group_indices + .iter() + .all(|idx| !other.row_group_indices.contains(idx))); + + self.time_range = ( + self.time_range.0.min(other.time_range.0), + self.time_range.1.max(other.time_range.1), + ); + self.indices.append(&mut other.indices); + self.row_group_indices.append(&mut other.row_group_indices); + self.num_rows += other.num_rows; + } + + /// Returns true if we can split the range into multiple smaller ranges and + /// still preserve the order for [SeqScan]. + pub(crate) fn can_split_preserve_order(&self) -> bool { + // Only one source and multiple row groups. + self.indices.len() == 1 && self.row_group_indices.len() > 1 + } + + /// Splits the range if it can preserve the order. + pub(crate) fn maybe_split(self, output: &mut Vec) { + if self.can_split_preserve_order() { + output.reserve(self.row_group_indices.len()); + let num_rows = self.num_rows / self.row_group_indices.len(); + // Splits by row group. + for index in self.row_group_indices { + output.push(RangeMeta { + time_range: self.time_range, + indices: self.indices.clone(), + row_group_indices: smallvec![index], + num_rows, + }); + } + } else { + output.push(self); + } + } + + fn push_unordered_mem_ranges(memtables: &[MemtableRef], ranges: &mut Vec) { + // For append mode, we can parallelize reading memtables. + for (memtable_index, memtable) in memtables.iter().enumerate() { + let stats = memtable.stats(); + let Some(time_range) = stats.time_range() else { + continue; + }; + for row_group_index in 0..stats.num_ranges() { + let num_rows = stats.num_rows() / stats.num_ranges(); + ranges.push(RangeMeta { + time_range, + indices: smallvec![memtable_index], + row_group_indices: smallvec![RowGroupIndex { + index: memtable_index, + row_group_index: row_group_index as i64, + }], + num_rows, + }); + } + } + } + + fn push_unordered_file_ranges( + num_memtables: usize, + files: &[FileHandle], + ranges: &mut Vec, + ) { + // For append mode, we can parallelize reading row groups. + for (i, file) in files.iter().enumerate() { + let file_index = num_memtables + i; + if file.meta_ref().num_row_groups > 0 { + // Scans each row group. + for row_group_index in 0..file.meta_ref().num_row_groups { + ranges.push(RangeMeta { + time_range: file.time_range(), + indices: smallvec![file_index], + row_group_indices: smallvec![RowGroupIndex { + index: file_index, + row_group_index: row_group_index as i64, + }], + num_rows: DEFAULT_ROW_GROUP_SIZE, + }); + } + } else { + // If we don't known the number of row groups in advance, scan all row groups. + ranges.push(RangeMeta { + time_range: file.time_range(), + indices: smallvec![file_index], + row_group_indices: smallvec![RowGroupIndex { + index: file_index, + row_group_index: ALL_ROW_GROUPS, + }], + // This may be 0. + num_rows: file.meta_ref().num_rows as usize, + }); + } + } + } + + fn push_seq_mem_ranges(memtables: &[MemtableRef], ranges: &mut Vec) { + // For non append-only mode, each range only contains one memtable. + for (i, memtable) in memtables.iter().enumerate() { + let stats = memtable.stats(); + let Some(time_range) = stats.time_range() else { + continue; + }; + ranges.push(RangeMeta { + time_range, + indices: smallvec![i], + row_group_indices: smallvec![RowGroupIndex { + index: i, + row_group_index: ALL_ROW_GROUPS, + }], + num_rows: stats.num_rows(), + }); + } + } + + fn push_seq_file_ranges( + num_memtables: usize, + files: &[FileHandle], + ranges: &mut Vec, + ) { + // For non append-only mode, each range only contains one file. + for (i, file) in files.iter().enumerate() { + let file_index = num_memtables + i; + ranges.push(RangeMeta { + time_range: file.time_range(), + indices: smallvec![file_index], + row_group_indices: smallvec![RowGroupIndex { + index: file_index, + row_group_index: ALL_ROW_GROUPS, + }], + num_rows: file.meta_ref().num_rows as usize, + }); + } + } +} + +/// Groups ranges by time range. +/// It assumes each input range only contains a file or a memtable. +fn group_ranges_for_seq_scan(mut ranges: Vec) -> Vec { + if ranges.is_empty() { + return ranges; + } + + // Sorts ranges by time range (start asc, end desc). + ranges.sort_unstable_by(|a, b| { + let l = a.time_range; + let r = b.time_range; + l.0.cmp(&r.0).then_with(|| r.1.cmp(&l.1)) + }); + let mut range_in_progress = None; + // Parts with exclusive time ranges. + let mut exclusive_ranges = Vec::with_capacity(ranges.len()); + for range in ranges { + let Some(mut prev_range) = range_in_progress.take() else { + // This is the new range to process. + range_in_progress = Some(range); + continue; + }; + + if prev_range.overlaps(&range) { + prev_range.merge(range); + range_in_progress = Some(prev_range); + } else { + exclusive_ranges.push(prev_range); + range_in_progress = Some(range); + } + } + if let Some(range) = range_in_progress { + exclusive_ranges.push(range); + } + + exclusive_ranges +} + +/// Splits the range into multiple smaller ranges. +/// It assumes the input `ranges` list is created by [group_ranges_for_seq_scan()]. +fn maybe_split_ranges_for_seq_scan(ranges: Vec) -> Vec { + let mut new_ranges = Vec::with_capacity(ranges.len()); + for range in ranges { + range.maybe_split(&mut new_ranges); + } + + new_ranges +} diff --git a/src/mito2/src/read/scan_region.rs b/src/mito2/src/read/scan_region.rs index d28562c554..28deb1186e 100644 --- a/src/mito2/src/read/scan_region.rs +++ b/src/mito2/src/read/scan_region.rs @@ -14,9 +14,9 @@ //! Scans a region according to the scan request. -use std::collections::HashSet; +use std::collections::{BTreeMap, HashSet}; use std::fmt; -use std::sync::Arc; +use std::sync::{Arc, Mutex as StdMutex}; use std::time::{Duration, Instant}; use common_error::ext::BoxedError; @@ -26,6 +26,7 @@ use common_time::range::TimestampRange; use common_time::Timestamp; use datafusion::physical_plan::DisplayFormatType; use datafusion_expr::utils::expr_to_columns; +use parquet::arrow::arrow_reader::RowSelection; use smallvec::SmallVec; use store_api::region_engine::{PartitionRange, RegionScannerRef}; use store_api::storage::{ScanRequest, TimeSeriesRowSelector}; @@ -41,6 +42,7 @@ use crate::memtable::{MemtableRange, MemtableRef}; use crate::metrics::READ_SST_COUNT; use crate::read::compat::{self, CompatBatch}; use crate::read::projection::ProjectionMapper; +use crate::read::range::{RangeMeta, RowGroupIndex}; use crate::read::seq_scan::SeqScan; use crate::read::unordered_scan::UnorderedScan; use crate::read::{Batch, Source}; @@ -51,7 +53,7 @@ use crate::sst::index::fulltext_index::applier::builder::FulltextIndexApplierBui use crate::sst::index::fulltext_index::applier::FulltextIndexApplierRef; use crate::sst::index::inverted_index::applier::builder::InvertedIndexApplierBuilder; use crate::sst::index::inverted_index::applier::InvertedIndexApplierRef; -use crate::sst::parquet::file_range::FileRange; +use crate::sst::parquet::file_range::{FileRange, FileRangeContextRef}; use crate::sst::parquet::reader::ReaderMetrics; /// A scanner scans a region and returns a [SendableRecordBatchStream]. @@ -643,6 +645,61 @@ impl ScanInput { Ok(sources) } + /// Prunes a memtable to scan and returns the builder to build readers. + fn prune_memtable(&self, mem_index: usize) -> MemRangeBuilder { + let memtable = &self.memtables[mem_index]; + let row_groups = memtable.ranges(Some(self.mapper.column_ids()), self.predicate.clone()); + MemRangeBuilder { row_groups } + } + + /// Prunes a file to scan and returns the builder to build readers. + async fn prune_file( + &self, + file_index: usize, + reader_metrics: &mut ReaderMetrics, + ) -> Result { + let file = &self.files[file_index]; + let res = self + .access_layer + .read_sst(file.clone()) + .predicate(self.predicate.clone()) + .time_range(self.time_range) + .projection(Some(self.mapper.column_ids().to_vec())) + .cache(self.cache_manager.clone()) + .inverted_index_applier(self.inverted_index_applier.clone()) + .fulltext_index_applier(self.fulltext_index_applier.clone()) + .expected_metadata(Some(self.mapper.metadata().clone())) + .build_reader_input(reader_metrics) + .await; + let (mut file_range_ctx, row_groups) = match res { + Ok(x) => x, + Err(e) => { + if e.is_object_not_found() && self.ignore_file_not_found { + error!(e; "File to scan does not exist, region_id: {}, file: {}", file.region_id(), file.file_id()); + return Ok(FileRangeBuilder::default()); + } else { + return Err(e); + } + } + }; + if !compat::has_same_columns( + self.mapper.metadata(), + file_range_ctx.read_format().metadata(), + ) { + // They have different schema. We need to adapt the batch first so the + // mapper can convert it. + let compat = CompatBatch::new( + &self.mapper, + file_range_ctx.read_format().metadata().clone(), + )?; + file_range_ctx.set_compat_batch(Some(compat)); + } + Ok(FileRangeBuilder { + context: Some(Arc::new(file_range_ctx)), + row_groups, + }) + } + /// Prunes file ranges to scan and adds them to the `collector`. pub(crate) async fn prune_file_ranges( &self, @@ -749,51 +806,6 @@ impl ScanInput { pub(crate) fn predicate(&self) -> Option { self.predicate.clone() } - - /// Retrieves [`PartitionRange`] from memtable and files - pub(crate) fn partition_ranges(&self) -> Vec { - let mut id = 0; - let mut container = Vec::with_capacity(self.memtables.len() + self.files.len()); - - for memtable in &self.memtables { - let range = PartitionRange { - // TODO(ruihang): filter out empty memtables in the future. - start: memtable.stats().time_range().unwrap().0, - end: memtable.stats().time_range().unwrap().1, - num_rows: memtable.stats().num_rows(), - identifier: id, - }; - id += 1; - container.push(range); - } - - for file in &self.files { - if self.append_mode { - // For append mode, we can parallelize reading row groups. - for _ in 0..file.meta_ref().num_row_groups { - let range = PartitionRange { - start: file.time_range().0, - end: file.time_range().1, - num_rows: file.num_rows(), - identifier: id, - }; - id += 1; - container.push(range); - } - } else { - let range = PartitionRange { - start: file.meta_ref().time_range.0, - end: file.meta_ref().time_range.1, - num_rows: file.meta_ref().num_rows as usize, - identifier: id, - }; - id += 1; - container.push(range); - } - } - - container - } } #[cfg(test)] @@ -967,6 +979,10 @@ pub(crate) struct StreamContext { /// The scanner builds parts to scan from the input lazily. /// The mutex is used to ensure the parts are only built once. pub(crate) parts: Mutex<(ScanPartList, Duration)>, + /// Metadata for partition ranges. + pub(crate) ranges: Vec, + /// Lists of range builders. + range_builders: RangeBuilderList, // Metrics: /// The start time of the query. @@ -974,17 +990,77 @@ pub(crate) struct StreamContext { } impl StreamContext { - /// Creates a new [StreamContext]. - pub(crate) fn new(input: ScanInput) -> Self { + /// Creates a new [StreamContext] for [SeqScan]. + pub(crate) fn seq_scan_ctx(input: ScanInput) -> Self { let query_start = input.query_start.unwrap_or_else(Instant::now); + let ranges = RangeMeta::seq_scan_ranges(&input); + READ_SST_COUNT.observe(input.files.len() as f64); + let range_builders = RangeBuilderList::new(input.memtables.len(), input.files.len()); Self { input, parts: Mutex::new((ScanPartList::default(), Duration::default())), + ranges, + range_builders, query_start, } } + /// Creates a new [StreamContext] for [UnorderedScan]. + pub(crate) fn unordered_scan_ctx(input: ScanInput) -> Self { + let query_start = input.query_start.unwrap_or_else(Instant::now); + let ranges = RangeMeta::unordered_scan_ranges(&input); + READ_SST_COUNT.observe(input.files.len() as f64); + let range_builders = RangeBuilderList::new(input.memtables.len(), input.files.len()); + + Self { + input, + parts: Mutex::new((ScanPartList::default(), Duration::default())), + ranges, + range_builders, + query_start, + } + } + + /// Returns true if the index refers to a memtable. + pub(crate) fn is_mem_range_index(&self, index: RowGroupIndex) -> bool { + self.input.memtables.len() > index.index + } + + /// Creates file ranges to scan. + pub(crate) async fn build_file_ranges( + &self, + index: RowGroupIndex, + ranges: &mut Vec, + reader_metrics: &mut ReaderMetrics, + ) -> Result<()> { + ranges.clear(); + self.range_builders + .build_file_ranges(&self.input, index, ranges, reader_metrics) + .await + } + + /// Creates memtable ranges to scan. + pub(crate) fn build_mem_ranges(&self, index: RowGroupIndex, ranges: &mut Vec) { + ranges.clear(); + self.range_builders + .build_mem_ranges(&self.input, index, ranges) + } + + /// Retrieves the partition ranges. + pub(crate) fn partition_ranges(&self) -> Vec { + self.ranges + .iter() + .enumerate() + .map(|(idx, range_meta)| PartitionRange { + start: range_meta.time_range.0, + end: range_meta.time_range.1, + num_rows: range_meta.num_rows, + identifier: idx, + }) + .collect() + } + /// Format the context for explain. pub(crate) fn format_for_explain( &self, @@ -1011,3 +1087,125 @@ impl StreamContext { Ok(()) } } + +/// List to manages the builders to create file ranges. +struct RangeBuilderList { + mem_builders: Vec>>, + file_builders: Vec>>, +} + +impl RangeBuilderList { + /// Creates a new [ReaderBuilderList] with the given number of memtables and files. + fn new(num_memtables: usize, num_files: usize) -> Self { + let mem_builders = (0..num_memtables).map(|_| StdMutex::new(None)).collect(); + let file_builders = (0..num_files).map(|_| Mutex::new(None)).collect(); + Self { + mem_builders, + file_builders, + } + } + + /// Builds file ranges to read the row group at `index`. + async fn build_file_ranges( + &self, + input: &ScanInput, + index: RowGroupIndex, + ranges: &mut Vec, + reader_metrics: &mut ReaderMetrics, + ) -> Result<()> { + let file_index = index.index - self.mem_builders.len(); + let mut builder_opt = self.file_builders[file_index].lock().await; + match &mut *builder_opt { + Some(builder) => builder.build_ranges(index.row_group_index, ranges), + None => { + let builder = input.prune_file(file_index, reader_metrics).await?; + builder.build_ranges(index.row_group_index, ranges); + *builder_opt = Some(builder); + } + } + Ok(()) + } + + /// Builds mem ranges to read the row group at `index`. + fn build_mem_ranges( + &self, + input: &ScanInput, + index: RowGroupIndex, + ranges: &mut Vec, + ) { + let mut builder_opt = self.mem_builders[index.index].lock().unwrap(); + match &mut *builder_opt { + Some(builder) => builder.build_ranges(index.row_group_index, ranges), + None => { + let builder = input.prune_memtable(index.index); + builder.build_ranges(index.row_group_index, ranges); + *builder_opt = Some(builder); + } + } + } +} + +/// Builder to create file ranges. +#[derive(Default)] +struct FileRangeBuilder { + /// Context for the file. + /// None indicates nothing to read. + context: Option, + /// Row selections for each row group to read. + /// It skips the row group if it is not in the map. + row_groups: BTreeMap>, +} + +impl FileRangeBuilder { + /// Builds file ranges to read. + /// Negative `row_group_index` indicates all row groups. + fn build_ranges(&self, row_group_index: i64, ranges: &mut Vec) { + let Some(context) = self.context.clone() else { + return; + }; + if row_group_index >= 0 { + let row_group_index = row_group_index as usize; + // Scans one row group. + let Some(row_selection) = self.row_groups.get(&row_group_index) else { + return; + }; + ranges.push(FileRange::new( + context, + row_group_index, + row_selection.clone(), + )); + } else { + // Scans all row groups. + ranges.extend( + self.row_groups + .iter() + .map(|(row_group_index, row_selection)| { + FileRange::new(context.clone(), *row_group_index, row_selection.clone()) + }), + ); + } + } +} + +/// Builder to create mem ranges. +struct MemRangeBuilder { + /// Ranges of a memtable. + row_groups: BTreeMap, +} + +impl MemRangeBuilder { + /// Builds mem ranges to read in the memtable. + /// Negative `row_group_index` indicates all row groups. + fn build_ranges(&self, row_group_index: i64, ranges: &mut Vec) { + if row_group_index >= 0 { + let row_group_index = row_group_index as usize; + // Scans one row group. + let Some(range) = self.row_groups.get(&row_group_index) else { + return; + }; + ranges.push(range.clone()); + } else { + ranges.extend(self.row_groups.values().cloned()); + } + } +} diff --git a/src/mito2/src/read/seq_scan.rs b/src/mito2/src/read/seq_scan.rs index 296d55250b..6e45c8ce4a 100644 --- a/src/mito2/src/read/seq_scan.rs +++ b/src/mito2/src/read/seq_scan.rs @@ -68,11 +68,10 @@ impl SeqScan { pub(crate) fn new(input: ScanInput) -> Self { let parallelism = input.parallelism.parallelism.max(1); let mut properties = ScannerProperties::default() - .with_parallelism(parallelism) .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - properties.partitions = vec![input.partition_ranges()]; - let stream_ctx = Arc::new(StreamContext::new(input)); + let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input)); + properties.partitions = vec![stream_ctx.partition_ranges()]; Self { properties, @@ -594,7 +593,7 @@ impl SeqDistributor { continue; } let part = ScanPart { - memtable_ranges: mem_ranges, + memtable_ranges: mem_ranges.into_values().collect(), file_ranges: smallvec![], time_range: stats.time_range(), }; diff --git a/src/mito2/src/read/unordered_scan.rs b/src/mito2/src/read/unordered_scan.rs index 67e87197a6..8a019132d7 100644 --- a/src/mito2/src/read/unordered_scan.rs +++ b/src/mito2/src/read/unordered_scan.rs @@ -16,7 +16,7 @@ use std::fmt; use std::sync::Arc; -use std::time::{Duration, Instant}; +use std::time::Instant; use async_stream::{stream, try_stream}; use common_error::ext::BoxedError; @@ -25,23 +25,18 @@ use common_recordbatch::{RecordBatch, RecordBatchStreamWrapper, SendableRecordBa use common_telemetry::debug; use datafusion::physical_plan::{DisplayAs, DisplayFormatType}; use datatypes::schema::SchemaRef; -use futures::StreamExt; -use smallvec::smallvec; +use futures::{Stream, StreamExt}; use snafu::ResultExt; use store_api::region_engine::{PartitionRange, RegionScanner, ScannerProperties}; -use store_api::storage::ColumnId; -use table::predicate::Predicate; use crate::cache::CacheManager; use crate::error::Result; -use crate::memtable::{MemtableRange, MemtableRef}; +use crate::memtable::MemtableRange; use crate::read::compat::CompatBatch; use crate::read::projection::ProjectionMapper; -use crate::read::scan_region::{ - FileRangeCollector, ScanInput, ScanPart, ScanPartList, StreamContext, -}; +use crate::read::range::RowGroupIndex; +use crate::read::scan_region::{ScanInput, StreamContext}; use crate::read::{ScannerMetrics, Source}; -use crate::sst::file::FileMeta; use crate::sst::parquet::file_range::FileRange; use crate::sst::parquet::reader::ReaderMetrics; @@ -58,13 +53,11 @@ pub struct UnorderedScan { impl UnorderedScan { /// Creates a new [UnorderedScan]. pub(crate) fn new(input: ScanInput) -> Self { - let parallelism = input.parallelism.parallelism.max(1); let mut properties = ScannerProperties::default() - .with_parallelism(parallelism) .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - properties.partitions = vec![input.partition_ranges()]; - let stream_ctx = Arc::new(StreamContext::new(input)); + let stream_ctx = Arc::new(StreamContext::unordered_scan_ctx(input)); + properties.partitions = vec![stream_ctx.partition_ranges()]; Self { properties, @@ -127,6 +120,161 @@ impl UnorderedScan { Ok(Some(record_batch)) } + + /// Scans a [PartitionRange] and returns a stream. + fn scan_partition_range<'a>( + stream_ctx: &'a StreamContext, + part_range: &'a PartitionRange, + mem_ranges: &'a mut Vec, + file_ranges: &'a mut Vec, + reader_metrics: &'a mut ReaderMetrics, + metrics: &'a mut ScannerMetrics, + ) -> impl Stream> + 'a { + stream! { + // Gets range meta. + let range_meta = &stream_ctx.ranges[part_range.identifier]; + for index in &range_meta.row_group_indices { + if stream_ctx.is_mem_range_index(*index) { + let stream = Self::scan_mem_ranges(stream_ctx, *index, mem_ranges, metrics); + for await batch in stream { + yield batch; + } + } else { + let stream = Self::scan_file_ranges(stream_ctx, *index, file_ranges, reader_metrics, metrics); + for await batch in stream { + yield batch; + } + } + } + } + } + + /// Scans memtable ranges at `index`. + fn scan_mem_ranges<'a>( + stream_ctx: &'a StreamContext, + index: RowGroupIndex, + ranges: &'a mut Vec, + metrics: &'a mut ScannerMetrics, + ) -> impl Stream> + 'a { + try_stream! { + let mapper = &stream_ctx.input.mapper; + let cache = stream_ctx.input.cache_manager.as_deref(); + stream_ctx.build_mem_ranges(index, ranges); + metrics.num_mem_ranges += ranges.len(); + for range in ranges { + let build_reader_start = Instant::now(); + let iter = range.build_iter().map_err(BoxedError::new).context(ExternalSnafu)?; + metrics.build_reader_cost = build_reader_start.elapsed(); + + let mut source = Source::Iter(iter); + while let Some(batch) = + Self::fetch_from_source(&mut source, mapper, cache, None, metrics).await? + { + metrics.num_batches += 1; + metrics.num_rows += batch.num_rows(); + let yield_start = Instant::now(); + yield batch; + metrics.yield_cost += yield_start.elapsed(); + } + } + } + } + + /// Scans file ranges at `index`. + fn scan_file_ranges<'a>( + stream_ctx: &'a StreamContext, + index: RowGroupIndex, + ranges: &'a mut Vec, + reader_metrics: &'a mut ReaderMetrics, + metrics: &'a mut ScannerMetrics, + ) -> impl Stream> + 'a { + try_stream! { + let mapper = &stream_ctx.input.mapper; + let cache = stream_ctx.input.cache_manager.as_deref(); + stream_ctx + .build_file_ranges(index, ranges, reader_metrics) + .await + .map_err(BoxedError::new) + .context(ExternalSnafu)?; + metrics.num_file_ranges += ranges.len(); + for range in ranges { + let build_reader_start = Instant::now(); + let reader = range + .reader(None) + .await + .map_err(BoxedError::new) + .context(ExternalSnafu)?; + metrics.build_reader_cost += build_reader_start.elapsed(); + let compat_batch = range.compat_batch(); + let mut source = Source::PruneReader(reader); + while let Some(batch) = + Self::fetch_from_source(&mut source, mapper, cache, compat_batch, metrics) + .await? + { + metrics.num_batches += 1; + metrics.num_rows += batch.num_rows(); + let yield_start = Instant::now(); + yield batch; + metrics.yield_cost += yield_start.elapsed(); + } + if let Source::PruneReader(mut reader) = source { + reader_metrics.merge_from(reader.metrics()); + } + } + } + } + + fn scan_partition_impl( + &self, + partition: usize, + ) -> Result { + let mut metrics = ScannerMetrics { + prepare_scan_cost: self.stream_ctx.query_start.elapsed(), + ..Default::default() + }; + let stream_ctx = self.stream_ctx.clone(); + let ranges_opt = self.properties.partitions.get(partition).cloned(); + + let stream = stream! { + let first_poll = stream_ctx.query_start.elapsed(); + let Some(part_ranges) = ranges_opt else { + return; + }; + + let mut mem_ranges = Vec::new(); + let mut file_ranges = Vec::new(); + let mut reader_metrics = ReaderMetrics::default(); + // Scans each part. + for part_range in part_ranges { + let stream = Self::scan_partition_range( + &stream_ctx, + &part_range, + &mut mem_ranges, + &mut file_ranges, + &mut reader_metrics, + &mut metrics, + ); + for await batch in stream { + yield batch; + } + } + + reader_metrics.observe_rows("unordered_scan_files"); + metrics.total_cost = stream_ctx.query_start.elapsed(); + metrics.observe_metrics_on_finish(); + let mapper = &stream_ctx.input.mapper; + debug!( + "Unordered scan partition {} finished, region_id: {}, metrics: {:?}, reader_metrics: {:?}, first_poll: {:?}", + partition, mapper.metadata().region_id, metrics, reader_metrics, first_poll, + ); + }; + let stream = Box::pin(RecordBatchStreamWrapper::new( + self.stream_ctx.input.mapper.output_schema(), + Box::pin(stream), + )); + + Ok(stream) + } } impl RegionScanner for UnorderedScan { @@ -144,89 +292,7 @@ impl RegionScanner for UnorderedScan { } fn scan_partition(&self, partition: usize) -> Result { - let mut metrics = ScannerMetrics { - prepare_scan_cost: self.stream_ctx.query_start.elapsed(), - ..Default::default() - }; - let stream_ctx = self.stream_ctx.clone(); - let parallelism = self.properties.num_partitions(); - let stream = try_stream! { - let first_poll = stream_ctx.query_start.elapsed(); - let part = { - let mut parts = stream_ctx.parts.lock().await; - maybe_init_parts(&stream_ctx.input, &mut parts, &mut metrics, parallelism) - .await - .map_err(BoxedError::new) - .context(ExternalSnafu)?; - // Clone the part and releases the lock. - // TODO(yingwen): We might wrap the part in an Arc in the future if cloning is too expensive. - let Some(part) = parts.0.get_part(partition).cloned() else { - return; - }; - part - }; - - let build_reader_start = Instant::now(); - let mapper = &stream_ctx.input.mapper; - let memtable_sources = part - .memtable_ranges - .iter() - .map(|mem| { - let iter = mem.build_iter()?; - Ok(Source::Iter(iter)) - }) - .collect::>>() - .map_err(BoxedError::new) - .context(ExternalSnafu)?; - metrics.build_reader_cost = build_reader_start.elapsed(); - - let query_start = stream_ctx.query_start; - let cache = stream_ctx.input.cache_manager.as_deref(); - // Scans memtables first. - for mut source in memtable_sources { - while let Some(batch) = Self::fetch_from_source(&mut source, mapper, cache, None, &mut metrics).await? { - metrics.num_batches += 1; - metrics.num_rows += batch.num_rows(); - let yield_start = Instant::now(); - yield batch; - metrics.yield_cost += yield_start.elapsed(); - } - } - // Then scans file ranges. - let mut reader_metrics = ReaderMetrics::default(); - // Safety: UnorderedDistributor::build_parts() ensures this. - for file_range in &part.file_ranges[0] { - let build_reader_start = Instant::now(); - let reader = file_range.reader(None).await.map_err(BoxedError::new).context(ExternalSnafu)?; - metrics.build_reader_cost += build_reader_start.elapsed(); - let compat_batch = file_range.compat_batch(); - let mut source = Source::PruneReader(reader); - while let Some(batch) = Self::fetch_from_source(&mut source, mapper, cache, compat_batch, &mut metrics).await? { - metrics.num_batches += 1; - metrics.num_rows += batch.num_rows(); - let yield_start = Instant::now(); - yield batch; - metrics.yield_cost += yield_start.elapsed(); - } - if let Source::PruneReader(mut reader) = source { - reader_metrics.merge_from(reader.metrics()); - } - } - - reader_metrics.observe_rows("unordered_scan_files"); - metrics.total_cost = query_start.elapsed(); - metrics.observe_metrics_on_finish(); - debug!( - "Unordered scan partition {} finished, region_id: {}, metrics: {:?}, reader_metrics: {:?}, first_poll: {:?}, ranges: {}", - partition, mapper.metadata().region_id, metrics, reader_metrics, first_poll, part.file_ranges[0].len(), - ); - }; - let stream = Box::pin(RecordBatchStreamWrapper::new( - self.stream_ctx.input.mapper.output_schema(), - Box::pin(stream), - )); - - Ok(stream) + self.scan_partition_impl(partition) } fn has_predicate(&self) -> bool { @@ -261,117 +327,3 @@ impl UnorderedScan { &self.stream_ctx.input } } - -/// Initializes parts if they are not built yet. -async fn maybe_init_parts( - input: &ScanInput, - part_list: &mut (ScanPartList, Duration), - metrics: &mut ScannerMetrics, - parallelism: usize, -) -> Result<()> { - if part_list.0.is_none() { - let now = Instant::now(); - let mut distributor = UnorderedDistributor::default(); - let reader_metrics = input.prune_file_ranges(&mut distributor).await?; - distributor.append_mem_ranges( - &input.memtables, - Some(input.mapper.column_ids()), - input.predicate.clone(), - ); - part_list.0.set_parts(distributor.build_parts(parallelism)); - let build_part_cost = now.elapsed(); - part_list.1 = build_part_cost; - - metrics.observe_init_part(build_part_cost, &reader_metrics); - } else { - // Updates the cost of building parts. - metrics.build_parts_cost = part_list.1; - } - Ok(()) -} - -/// Builds [ScanPart]s without preserving order. It distributes file ranges and memtables -/// across partitions. Each partition scans a subset of memtables and file ranges. There -/// is no output ordering guarantee of each partition. -#[derive(Default)] -struct UnorderedDistributor { - mem_ranges: Vec, - file_ranges: Vec, -} - -impl FileRangeCollector for UnorderedDistributor { - fn append_file_ranges( - &mut self, - _file_meta: &FileMeta, - file_ranges: impl Iterator, - ) { - self.file_ranges.extend(file_ranges); - } -} - -impl UnorderedDistributor { - /// Appends memtable ranges to the distributor. - fn append_mem_ranges( - &mut self, - memtables: &[MemtableRef], - projection: Option<&[ColumnId]>, - predicate: Option, - ) { - for mem in memtables { - let mut mem_ranges = mem.ranges(projection, predicate.clone()); - if mem_ranges.is_empty() { - continue; - } - self.mem_ranges.append(&mut mem_ranges); - } - } - - /// Distributes file ranges and memtables across partitions according to the `parallelism`. - /// The output number of parts may be `<= parallelism`. - /// - /// [ScanPart] created by this distributor only contains one group of file ranges. - fn build_parts(self, parallelism: usize) -> Vec { - if parallelism <= 1 { - // Returns a single part. - let part = ScanPart { - memtable_ranges: self.mem_ranges.clone(), - file_ranges: smallvec![self.file_ranges], - time_range: None, - }; - return vec![part]; - } - - let mems_per_part = ((self.mem_ranges.len() + parallelism - 1) / parallelism).max(1); - let ranges_per_part = ((self.file_ranges.len() + parallelism - 1) / parallelism).max(1); - debug!( - "Parallel scan is enabled, parallelism: {}, {} mem_ranges, {} file_ranges, mems_per_part: {}, ranges_per_part: {}", - parallelism, - self.mem_ranges.len(), - self.file_ranges.len(), - mems_per_part, - ranges_per_part - ); - let mut scan_parts = self - .mem_ranges - .chunks(mems_per_part) - .map(|mems| ScanPart { - memtable_ranges: mems.to_vec(), - file_ranges: smallvec![Vec::new()], // Ensures there is always one group. - time_range: None, - }) - .collect::>(); - for (i, ranges) in self.file_ranges.chunks(ranges_per_part).enumerate() { - if i == scan_parts.len() { - scan_parts.push(ScanPart { - memtable_ranges: Vec::new(), - file_ranges: smallvec![ranges.to_vec()], - time_range: None, - }); - } else { - scan_parts[i].file_ranges = smallvec![ranges.to_vec()]; - } - } - - scan_parts - } -} diff --git a/src/mito2/src/sst/parquet.rs b/src/mito2/src/sst/parquet.rs index cce9965539..cc0cff605b 100644 --- a/src/mito2/src/sst/parquet.rs +++ b/src/mito2/src/sst/parquet.rs @@ -40,7 +40,7 @@ pub const PARQUET_METADATA_KEY: &str = "greptime:metadata"; /// Default batch size to read parquet files. pub(crate) const DEFAULT_READ_BATCH_SIZE: usize = 1024; /// Default row group size for parquet files. -const DEFAULT_ROW_GROUP_SIZE: usize = 100 * DEFAULT_READ_BATCH_SIZE; +pub(crate) const DEFAULT_ROW_GROUP_SIZE: usize = 100 * DEFAULT_READ_BATCH_SIZE; /// Parquet write options. #[derive(Debug)] diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index 766dab0429..ff7fe77e7a 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -238,6 +238,7 @@ impl ParquetReaderBuilder { cache_manager: self.cache_manager.clone(), }; + // TODO(yingwen): count the cost of the method. metrics.build_cost = start.elapsed(); let mut filters = if let Some(predicate) = &self.predicate { diff --git a/src/mito2/src/test_util/memtable_util.rs b/src/mito2/src/test_util/memtable_util.rs index 235e9694c7..70dec3cad4 100644 --- a/src/mito2/src/test_util/memtable_util.rs +++ b/src/mito2/src/test_util/memtable_util.rs @@ -14,6 +14,7 @@ //! Memtable test utilities. +use std::collections::BTreeMap; use std::sync::Arc; use api::helper::ColumnDataTypeWrapper; @@ -92,8 +93,8 @@ impl Memtable for EmptyMemtable { &self, _projection: Option<&[ColumnId]>, _predicate: Option, - ) -> Vec { - vec![] + ) -> BTreeMap { + BTreeMap::new() } fn is_empty(&self) -> bool { From 77af4fd981f2816ca05e2a94f2d8632e6f7aeedc Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Mon, 30 Sep 2024 10:56:53 +0800 Subject: [PATCH 048/128] refactor: introduce `HeartbeatHandlerGroupBuilder` (#4785) --- src/meta-srv/src/handler.rs | 154 +++++++++++++++++++++----- src/meta-srv/src/metasrv.rs | 6 +- src/meta-srv/src/metasrv/builder.rs | 53 ++------- src/meta-srv/src/service/heartbeat.rs | 4 +- 4 files changed, 142 insertions(+), 75 deletions(-) diff --git a/src/meta-srv/src/handler.rs b/src/meta-srv/src/handler.rs index 063d3939c1..5363b6c548 100644 --- a/src/meta-srv/src/handler.rs +++ b/src/meta-srv/src/handler.rs @@ -22,12 +22,28 @@ use api::v1::meta::{ HeartbeatRequest, HeartbeatResponse, MailboxMessage, RegionLease, RequestHeader, ResponseHeader, Role, PROTOCOL_VERSION, }; +use check_leader_handler::CheckLeaderHandler; +use collect_cluster_info_handler::{ + CollectDatanodeClusterInfoHandler, CollectFlownodeClusterInfoHandler, + CollectFrontendClusterInfoHandler, +}; +use collect_stats_handler::CollectStatsHandler; +use common_base::Plugins; use common_meta::datanode::Stat; use common_meta::instruction::{Instruction, InstructionReply}; use common_meta::sequence::Sequence; use common_telemetry::{debug, info, warn}; use dashmap::DashMap; +use extract_stat_handler::ExtractStatHandler; +use failure_handler::RegionFailureHandler; +use filter_inactive_region_stats::FilterInactiveRegionStatsHandler; use futures::future::join_all; +use keep_lease_handler::{DatanodeKeepLeaseHandler, FlownodeKeepLeaseHandler}; +use mailbox_handler::MailboxHandler; +use on_leader_start_handler::OnLeaderStartHandler; +use publish_heartbeat_handler::PublishHeartbeatHandler; +use region_lease_handler::RegionLeaseHandler; +use response_header_handler::ResponseHeaderHandler; use snafu::{OptionExt, ResultExt}; use store_api::storage::RegionId; use tokio::sync::mpsc::Sender; @@ -36,6 +52,7 @@ use tokio::sync::{oneshot, Notify, RwLock}; use crate::error::{self, DeserializeFromJsonSnafu, Result, UnexpectedInstructionReplySnafu}; use crate::metasrv::Context; use crate::metrics::{METRIC_META_HANDLER_EXECUTE, METRIC_META_HEARTBEAT_CONNECTION_NUM}; +use crate::pubsub::PublisherRef; use crate::service::mailbox::{ BroadcastChannel, Channel, Mailbox, MailboxReceiver, MailboxRef, MessageId, }; @@ -96,6 +113,7 @@ impl HeartbeatAccumulator { } } +/// The pusher of the heartbeat response. pub struct Pusher { sender: Sender>, res_header: ResponseHeader, @@ -131,6 +149,7 @@ impl Pusher { } } +/// The group of heartbeat pushers. #[derive(Clone, Default)] pub struct Pushers(Arc>>); @@ -203,50 +222,57 @@ impl NameCachedHandler { } } -#[derive(Clone, Default)] +pub type HeartbeatHandlerGroupRef = Arc; + +/// The group of heartbeat handlers. +#[derive(Default)] pub struct HeartbeatHandlerGroup { - handlers: Arc>>, + handlers: Vec, pushers: Pushers, } impl HeartbeatHandlerGroup { pub(crate) fn new(pushers: Pushers) -> Self { Self { - handlers: Arc::new(RwLock::new(vec![])), + handlers: vec![], pushers, } } - pub async fn add_handler(&self, handler: impl HeartbeatHandler + 'static) { - let mut handlers = self.handlers.write().await; - handlers.push(NameCachedHandler::new(handler)); + fn add_handler(&mut self, handler: impl HeartbeatHandler + 'static) { + self.handlers.push(NameCachedHandler::new(handler)); } - pub async fn register(&self, key: impl AsRef, pusher: Pusher) { + /// Registers the heartbeat response [`Pusher`] with the given key to the group. + pub async fn register_pusher(&self, key: impl AsRef, pusher: Pusher) { let key = key.as_ref(); METRIC_META_HEARTBEAT_CONNECTION_NUM.inc(); info!("Pusher register: {}", key); let _ = self.pushers.insert(key.to_string(), pusher).await; } - pub async fn deregister(&self, key: impl AsRef) -> Option { + /// Deregisters the heartbeat response [`Pusher`] with the given key from the group. + /// + /// Returns the [`Pusher`] if it exists. + pub async fn deregister_push(&self, key: impl AsRef) -> Option { let key = key.as_ref(); METRIC_META_HEARTBEAT_CONNECTION_NUM.dec(); info!("Pusher unregister: {}", key); self.pushers.remove(key).await } + /// Returns the [`Pushers`] of the group. pub fn pushers(&self) -> Pushers { self.pushers.clone() } + /// Handles the heartbeat request. pub async fn handle( &self, req: HeartbeatRequest, mut ctx: Context, ) -> Result { let mut acc = HeartbeatAccumulator::default(); - let handlers = self.handlers.read().await; let role = req .header .as_ref() @@ -255,7 +281,7 @@ impl HeartbeatHandlerGroup { err_msg: format!("invalid role: {:?}", req.header), })?; - for NameCachedHandler { name, handler } in handlers.iter() { + for NameCachedHandler { name, handler } in self.handlers.iter() { if !handler.is_acceptable(role) { continue; } @@ -426,6 +452,84 @@ impl Mailbox for HeartbeatMailbox { } } +/// The builder to build the group of heartbeat handlers. +pub struct HeartbeatHandlerGroupBuilder { + /// The handler to handle region failure. + region_failure_handler: Option, + + /// The handler to handle region lease. + region_lease_handler: RegionLeaseHandler, + + /// The plugins. + plugins: Option, + + /// The heartbeat response pushers. + pushers: Pushers, +} + +impl HeartbeatHandlerGroupBuilder { + pub fn new(pushers: Pushers, region_lease_handler: RegionLeaseHandler) -> Self { + Self { + region_failure_handler: None, + region_lease_handler, + plugins: None, + pushers, + } + } + + /// Sets the [`RegionFailureHandler`]. + pub fn with_region_failure_handler(mut self, handler: Option) -> Self { + self.region_failure_handler = handler; + self + } + + /// Sets the [`Plugins`]. + pub fn with_plugins(mut self, plugins: Option) -> Self { + self.plugins = plugins; + self + } + + /// Builds the group of heartbeat handlers. + pub fn build(self) -> HeartbeatHandlerGroup { + // Extract the `PublishHeartbeatHandler` from the plugins. + let publish_heartbeat_handler = if let Some(plugins) = self.plugins { + plugins + .get::() + .map(|publish| PublishHeartbeatHandler::new(publish.clone())) + } else { + None + }; + + // TODO(weny): Considers classifying handlers + // to make it easier for upper layers to customize handler groups. + let mut group = HeartbeatHandlerGroup::new(self.pushers); + group.add_handler(ResponseHeaderHandler); + // `KeepLeaseHandler` should preferably be in front of `CheckLeaderHandler`, + // because even if the current meta-server node is no longer the leader it can + // still help the datanode to keep lease. + group.add_handler(DatanodeKeepLeaseHandler); + group.add_handler(FlownodeKeepLeaseHandler); + group.add_handler(CheckLeaderHandler); + group.add_handler(OnLeaderStartHandler); + group.add_handler(ExtractStatHandler); + group.add_handler(CollectDatanodeClusterInfoHandler); + group.add_handler(CollectFrontendClusterInfoHandler); + group.add_handler(CollectFlownodeClusterInfoHandler); + group.add_handler(MailboxHandler); + group.add_handler(self.region_lease_handler); + group.add_handler(FilterInactiveRegionStatsHandler); + if let Some(region_failure_handler) = self.region_failure_handler { + group.add_handler(region_failure_handler); + } + if let Some(publish_heartbeat_handler) = publish_heartbeat_handler { + group.add_handler(publish_heartbeat_handler); + } + group.add_handler(CollectStatsHandler::default()); + + group + } +} + #[cfg(test)] mod tests { @@ -489,7 +593,7 @@ mod tests { let pusher: Pusher = Pusher::new(pusher_tx, &res_header); let handler_group = HeartbeatHandlerGroup::default(); handler_group - .register(format!("{}-{}", Role::Datanode as i32, datanode_id), pusher) + .register_pusher(format!("{}-{}", Role::Datanode as i32, datanode_id), pusher) .await; let kv_backend = Arc::new(MemoryKvBackend::new()); @@ -519,21 +623,21 @@ mod tests { #[tokio::test] async fn test_handler_name() { - let group = HeartbeatHandlerGroup::default(); - group.add_handler(ResponseHeaderHandler).await; - group.add_handler(DatanodeKeepLeaseHandler).await; - group.add_handler(FlownodeKeepLeaseHandler).await; - group.add_handler(CheckLeaderHandler).await; - group.add_handler(OnLeaderStartHandler).await; - group.add_handler(ExtractStatHandler).await; - group.add_handler(CollectDatanodeClusterInfoHandler).await; - group.add_handler(CollectFrontendClusterInfoHandler).await; - group.add_handler(CollectFlownodeClusterInfoHandler).await; - group.add_handler(MailboxHandler).await; - group.add_handler(FilterInactiveRegionStatsHandler).await; - group.add_handler(CollectStatsHandler::default()).await; + let mut group = HeartbeatHandlerGroup::default(); + group.add_handler(ResponseHeaderHandler); + group.add_handler(DatanodeKeepLeaseHandler); + group.add_handler(FlownodeKeepLeaseHandler); + group.add_handler(CheckLeaderHandler); + group.add_handler(OnLeaderStartHandler); + group.add_handler(ExtractStatHandler); + group.add_handler(CollectDatanodeClusterInfoHandler); + group.add_handler(CollectFrontendClusterInfoHandler); + group.add_handler(CollectFlownodeClusterInfoHandler); + group.add_handler(MailboxHandler); + group.add_handler(FilterInactiveRegionStatsHandler); + group.add_handler(CollectStatsHandler::default()); - let handlers = group.handlers.read().await; + let handlers = group.handlers; assert_eq!(12, handlers.len()); diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index 2beb09859b..de7d54aa65 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -52,7 +52,7 @@ use crate::error::{ StopProcedureManagerSnafu, }; use crate::failure_detector::PhiAccrualFailureDetectorOptions; -use crate::handler::HeartbeatHandlerGroup; +use crate::handler::HeartbeatHandlerGroupRef; use crate::lease::lookup_datanode_peer; use crate::lock::DistLockRef; use crate::procedure::region_migration::manager::RegionMigrationManagerRef; @@ -366,7 +366,7 @@ pub struct Metasrv { selector: SelectorRef, // The flow selector is used to select a target flownode. flow_selector: SelectorRef, - handler_group: HeartbeatHandlerGroup, + handler_group: HeartbeatHandlerGroupRef, election: Option, lock: DistLockRef, procedure_manager: ProcedureManagerRef, @@ -562,7 +562,7 @@ impl Metasrv { &self.flow_selector } - pub fn handler_group(&self) -> &HeartbeatHandlerGroup { + pub fn handler_group(&self) -> &HeartbeatHandlerGroupRef { &self.handler_group } diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 6b06bab867..662de433ab 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -46,22 +46,11 @@ use crate::cluster::{MetaPeerClientBuilder, MetaPeerClientRef}; use crate::error::{self, Result}; use crate::flow_meta_alloc::FlowPeerAllocator; use crate::greptimedb_telemetry::get_greptimedb_telemetry_task; -use crate::handler::check_leader_handler::CheckLeaderHandler; -use crate::handler::collect_cluster_info_handler::{ - CollectDatanodeClusterInfoHandler, CollectFlownodeClusterInfoHandler, - CollectFrontendClusterInfoHandler, -}; -use crate::handler::collect_stats_handler::CollectStatsHandler; -use crate::handler::extract_stat_handler::ExtractStatHandler; use crate::handler::failure_handler::RegionFailureHandler; -use crate::handler::filter_inactive_region_stats::FilterInactiveRegionStatsHandler; -use crate::handler::keep_lease_handler::{DatanodeKeepLeaseHandler, FlownodeKeepLeaseHandler}; -use crate::handler::mailbox_handler::MailboxHandler; -use crate::handler::on_leader_start_handler::OnLeaderStartHandler; -use crate::handler::publish_heartbeat_handler::PublishHeartbeatHandler; use crate::handler::region_lease_handler::RegionLeaseHandler; -use crate::handler::response_header_handler::ResponseHeaderHandler; -use crate::handler::{HeartbeatHandlerGroup, HeartbeatMailbox, Pushers}; +use crate::handler::{ + HeartbeatHandlerGroup, HeartbeatHandlerGroupBuilder, HeartbeatMailbox, Pushers, +}; use crate::lease::MetaPeerLookupService; use crate::lock::memory::MemLock; use crate::lock::DistLockRef; @@ -70,7 +59,6 @@ use crate::metasrv::{ }; use crate::procedure::region_migration::manager::RegionMigrationManager; use crate::procedure::region_migration::DefaultContextFactory; -use crate::pubsub::PublisherRef; use crate::region::supervisor::{ HeartbeatAcceptor, RegionFailureDetectorControl, RegionSupervisor, RegionSupervisorTicker, DEFAULT_TICK_INTERVAL, @@ -364,41 +352,16 @@ impl MetasrvBuilder { let handler_group = match handler_group { Some(handler_group) => handler_group, None => { - let publish_heartbeat_handler = plugins - .clone() - .and_then(|plugins| plugins.get::()) - .map(|publish| PublishHeartbeatHandler::new(publish.clone())); - let region_lease_handler = RegionLeaseHandler::new( distributed_time_constants::REGION_LEASE_SECS, table_metadata_manager.clone(), memory_region_keeper.clone(), ); - let group = HeartbeatHandlerGroup::new(pushers); - group.add_handler(ResponseHeaderHandler).await; - // `KeepLeaseHandler` should preferably be in front of `CheckLeaderHandler`, - // because even if the current meta-server node is no longer the leader it can - // still help the datanode to keep lease. - group.add_handler(DatanodeKeepLeaseHandler).await; - group.add_handler(FlownodeKeepLeaseHandler).await; - group.add_handler(CheckLeaderHandler).await; - group.add_handler(OnLeaderStartHandler).await; - group.add_handler(ExtractStatHandler).await; - group.add_handler(CollectDatanodeClusterInfoHandler).await; - group.add_handler(CollectFrontendClusterInfoHandler).await; - group.add_handler(CollectFlownodeClusterInfoHandler).await; - group.add_handler(MailboxHandler).await; - group.add_handler(region_lease_handler).await; - group.add_handler(FilterInactiveRegionStatsHandler).await; - if let Some(region_failover_handler) = region_failover_handler { - group.add_handler(region_failover_handler).await; - } - if let Some(publish_heartbeat_handler) = publish_heartbeat_handler { - group.add_handler(publish_heartbeat_handler).await; - } - group.add_handler(CollectStatsHandler::default()).await; - group + HeartbeatHandlerGroupBuilder::new(pushers, region_lease_handler) + .with_plugins(plugins.clone()) + .with_region_failure_handler(region_failover_handler) + .build() } }; @@ -417,7 +380,7 @@ impl MetasrvBuilder { selector, // TODO(jeremy): We do not allow configuring the flow selector. flow_selector: Arc::new(RoundRobinSelector::new(SelectTarget::Flownode)), - handler_group, + handler_group: Arc::new(handler_group), election, lock, procedure_manager, diff --git a/src/meta-srv/src/service/heartbeat.rs b/src/meta-srv/src/service/heartbeat.rs index f5ac74a4b5..569d6a8089 100644 --- a/src/meta-srv/src/service/heartbeat.rs +++ b/src/meta-srv/src/service/heartbeat.rs @@ -113,7 +113,7 @@ impl heartbeat_server::Heartbeat for Metasrv { ); if let Some(key) = pusher_key { - let _ = handler_group.deregister(&key).await; + let _ = handler_group.deregister_push(&key).await; } }); @@ -177,7 +177,7 @@ async fn register_pusher( let node_id = get_node_id(header); let key = format!("{}-{}", role, node_id); let pusher = Pusher::new(sender, header); - handler_group.register(&key, pusher).await; + handler_group.register_pusher(&key, pusher).await; Some(key) } From e39a9e6feb30dcd7f58742e1bc296856a9b27b04 Mon Sep 17 00:00:00 2001 From: zyy17 Date: Mon, 30 Sep 2024 11:26:50 +0800 Subject: [PATCH 049/128] feat: add `StatementStatistics` for slow query logging implementation (#4719) * feat: log slow query * feat: log slow query for sql * refactor: add slow query logging options * ci: fix errors * feat: add StatementStatistics * chore: revert modification of servers crate * docs: update config docs * fix: clippy errors --- Cargo.lock | 1 + config/config.md | 20 ++++++ config/datanode.example.toml | 13 ++++ config/flownode.example.toml | 13 ++++ config/frontend.example.toml | 13 ++++ config/metasrv.example.toml | 13 ++++ config/standalone.example.toml | 13 ++++ src/cmd/src/frontend.rs | 2 + src/cmd/src/standalone.rs | 2 + src/common/telemetry/Cargo.toml | 1 + src/common/telemetry/src/logging.rs | 62 +++++++++++++++++- src/common/telemetry/src/macros.rs | 11 ++++ src/flow/src/server.rs | 2 + src/frontend/src/instance/builder.rs | 5 ++ src/operator/src/statement.rs | 6 ++ src/query/Cargo.toml | 1 + src/query/src/lib.rs | 1 + src/query/src/stats.rs | 98 ++++++++++++++++++++++++++++ tests-integration/src/cluster.rs | 2 + tests-integration/src/standalone.rs | 2 + tests-integration/tests/http.rs | 3 + 21 files changed, 282 insertions(+), 2 deletions(-) create mode 100644 src/query/src/stats.rs diff --git a/Cargo.lock b/Cargo.lock index 4533257660..7aebe2e719 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2289,6 +2289,7 @@ dependencies = [ "common-error", "console-subscriber", "greptime-proto", + "humantime-serde", "lazy_static", "once_cell", "opentelemetry 0.21.0", diff --git a/config/config.md b/config/config.md index 641eee4b58..d413936ff3 100644 --- a/config/config.md +++ b/config/config.md @@ -163,6 +163,10 @@ | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | +| `logging.slow_query` | -- | -- | The slow query log options. | +| `logging.slow_query.enable` | Bool | `false` | Whether to enable slow query log. | +| `logging.slow_query.threshold` | String | Unset | The threshold of slow query. | +| `logging.slow_query.sample_ratio` | Float | Unset | The sampling ratio of slow query log. The value should be in the range of (0, 1]. | | `export_metrics` | -- | -- | The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API.
This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. | | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | @@ -249,6 +253,10 @@ | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | +| `logging.slow_query` | -- | -- | The slow query log options. | +| `logging.slow_query.enable` | Bool | `false` | Whether to enable slow query log. | +| `logging.slow_query.threshold` | String | Unset | The threshold of slow query. | +| `logging.slow_query.sample_ratio` | Float | Unset | The sampling ratio of slow query log. The value should be in the range of (0, 1]. | | `export_metrics` | -- | -- | The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API.
This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. | | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | @@ -314,6 +322,10 @@ | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | +| `logging.slow_query` | -- | -- | The slow query log options. | +| `logging.slow_query.enable` | Bool | `false` | Whether to enable slow query log. | +| `logging.slow_query.threshold` | String | Unset | The threshold of slow query. | +| `logging.slow_query.sample_ratio` | Float | Unset | The sampling ratio of slow query log. The value should be in the range of (0, 1]. | | `export_metrics` | -- | -- | The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API.
This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. | | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | @@ -466,6 +478,10 @@ | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | +| `logging.slow_query` | -- | -- | The slow query log options. | +| `logging.slow_query.enable` | Bool | `false` | Whether to enable slow query log. | +| `logging.slow_query.threshold` | String | Unset | The threshold of slow query. | +| `logging.slow_query.sample_ratio` | Float | Unset | The sampling ratio of slow query log. The value should be in the range of (0, 1]. | | `export_metrics` | -- | -- | The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API.
This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. | | `export_metrics.enable` | Bool | `false` | whether enable export metrics. | | `export_metrics.write_interval` | String | `30s` | The interval of export metrics. | @@ -512,5 +528,9 @@ | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | +| `logging.slow_query` | -- | -- | The slow query log options. | +| `logging.slow_query.enable` | Bool | `false` | Whether to enable slow query log. | +| `logging.slow_query.threshold` | String | Unset | The threshold of slow query. | +| `logging.slow_query.sample_ratio` | Float | Unset | The sampling ratio of slow query log. The value should be in the range of (0, 1]. | | `tracing` | -- | -- | The tracing options. Only effect when compiled with `tokio-console` feature. | | `tracing.tokio_console_addr` | String | Unset | The tokio console address. | diff --git a/config/datanode.example.toml b/config/datanode.example.toml index e4a3dca6d3..06a59ebd6f 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -586,6 +586,19 @@ log_format = "text" [logging.tracing_sample_ratio] default_ratio = 1.0 +## The slow query log options. +[logging.slow_query] +## Whether to enable slow query log. +enable = false + +## The threshold of slow query. +## @toml2docs:none-default +threshold = "10s" + +## The sampling ratio of slow query log. The value should be in the range of (0, 1]. +## @toml2docs:none-default +sample_ratio = 1.0 + ## The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API. ## This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. [export_metrics] diff --git a/config/flownode.example.toml b/config/flownode.example.toml index a3a414fb98..9d6d2fe5aa 100644 --- a/config/flownode.example.toml +++ b/config/flownode.example.toml @@ -84,6 +84,19 @@ log_format = "text" [logging.tracing_sample_ratio] default_ratio = 1.0 +## The slow query log options. +[logging.slow_query] +## Whether to enable slow query log. +enable = false + +## The threshold of slow query. +## @toml2docs:none-default +threshold = "10s" + +## The sampling ratio of slow query log. The value should be in the range of (0, 1]. +## @toml2docs:none-default +sample_ratio = 1.0 + ## The tracing options. Only effect when compiled with `tokio-console` feature. [tracing] ## The tokio console address. diff --git a/config/frontend.example.toml b/config/frontend.example.toml index eae001a2db..cc9698f61e 100644 --- a/config/frontend.example.toml +++ b/config/frontend.example.toml @@ -191,6 +191,19 @@ log_format = "text" [logging.tracing_sample_ratio] default_ratio = 1.0 +## The slow query log options. +[logging.slow_query] +## Whether to enable slow query log. +enable = false + +## The threshold of slow query. +## @toml2docs:none-default +threshold = "10s" + +## The sampling ratio of slow query log. The value should be in the range of (0, 1]. +## @toml2docs:none-default +sample_ratio = 1.0 + ## The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API. ## This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. [export_metrics] diff --git a/config/metasrv.example.toml b/config/metasrv.example.toml index e95a9fa7f2..8431940b45 100644 --- a/config/metasrv.example.toml +++ b/config/metasrv.example.toml @@ -178,6 +178,19 @@ log_format = "text" [logging.tracing_sample_ratio] default_ratio = 1.0 +## The slow query log options. +[logging.slow_query] +## Whether to enable slow query log. +enable = false + +## The threshold of slow query. +## @toml2docs:none-default +threshold = "10s" + +## The sampling ratio of slow query log. The value should be in the range of (0, 1]. +## @toml2docs:none-default +sample_ratio = 1.0 + ## The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API. ## This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. [export_metrics] diff --git a/config/standalone.example.toml b/config/standalone.example.toml index 1cd75e6414..2d14b6550f 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -630,6 +630,19 @@ log_format = "text" [logging.tracing_sample_ratio] default_ratio = 1.0 +## The slow query log options. +[logging.slow_query] +## Whether to enable slow query log. +enable = false + +## The threshold of slow query. +## @toml2docs:none-default +threshold = "10s" + +## The sampling ratio of slow query log. The value should be in the range of (0, 1]. +## @toml2docs:none-default +sample_ratio = 1.0 + ## The datanode can export its metrics and send to Prometheus compatible service (e.g. send to `greptimedb` itself) from remote-write API. ## This is only used for `greptimedb` to export its own metrics internally. It's different from prometheus scrape. [export_metrics] diff --git a/src/cmd/src/frontend.rs b/src/cmd/src/frontend.rs index 320dc49c19..7678e90c88 100644 --- a/src/cmd/src/frontend.rs +++ b/src/cmd/src/frontend.rs @@ -36,6 +36,7 @@ use frontend::instance::builder::FrontendBuilder; use frontend::instance::{FrontendInstance, Instance as FeInstance}; use frontend::server::Services; use meta_client::{MetaClientOptions, MetaClientType}; +use query::stats::StatementStatistics; use servers::tls::{TlsMode, TlsOption}; use servers::Mode; use snafu::{OptionExt, ResultExt}; @@ -352,6 +353,7 @@ impl StartCommand { catalog_manager, Arc::new(client), meta_client, + StatementStatistics::new(opts.logging.slow_query.clone()), ) .with_plugin(plugins.clone()) .with_local_cache_invalidator(layered_cache_registry) diff --git a/src/cmd/src/standalone.rs b/src/cmd/src/standalone.rs index 80b38ebaa7..4335bd5447 100644 --- a/src/cmd/src/standalone.rs +++ b/src/cmd/src/standalone.rs @@ -55,6 +55,7 @@ use frontend::service_config::{ }; use meta_srv::metasrv::{FLOW_ID_SEQ, TABLE_ID_SEQ}; use mito2::config::MitoConfig; +use query::stats::StatementStatistics; use serde::{Deserialize, Serialize}; use servers::export_metrics::ExportMetricsOption; use servers::grpc::GrpcOptions; @@ -557,6 +558,7 @@ impl StartCommand { catalog_manager.clone(), node_manager.clone(), ddl_task_executor.clone(), + StatementStatistics::new(opts.logging.slow_query.clone()), ) .with_plugin(plugins.clone()) .try_build() diff --git a/src/common/telemetry/Cargo.toml b/src/common/telemetry/Cargo.toml index 20fc52a763..2b4023cf7c 100644 --- a/src/common/telemetry/Cargo.toml +++ b/src/common/telemetry/Cargo.toml @@ -17,6 +17,7 @@ backtrace = "0.3" common-error.workspace = true console-subscriber = { version = "0.1", optional = true } greptime-proto.workspace = true +humantime-serde.workspace = true lazy_static.workspace = true once_cell.workspace = true opentelemetry = { version = "0.21.0", default-features = false, features = [ diff --git a/src/common/telemetry/src/logging.rs b/src/common/telemetry/src/logging.rs index de018aa4b6..c21766f04d 100644 --- a/src/common/telemetry/src/logging.rs +++ b/src/common/telemetry/src/logging.rs @@ -15,6 +15,7 @@ //! logging stuffs, inspired by databend use std::env; use std::sync::{Arc, Mutex, Once}; +use std::time::Duration; use once_cell::sync::{Lazy, OnceCell}; use opentelemetry::{global, KeyValue}; @@ -26,7 +27,7 @@ use serde::{Deserialize, Serialize}; use tracing_appender::non_blocking::WorkerGuard; use tracing_appender::rolling::{RollingFileAppender, Rotation}; use tracing_log::LogTracer; -use tracing_subscriber::filter::Targets; +use tracing_subscriber::filter::{FilterFn, Targets}; use tracing_subscriber::fmt::Layer; use tracing_subscriber::layer::SubscriberExt; use tracing_subscriber::prelude::*; @@ -64,6 +65,24 @@ pub struct LoggingOptions { /// The tracing sample ratio. pub tracing_sample_ratio: Option, + + /// The logging options of slow query. + pub slow_query: SlowQueryOptions, +} + +/// The options of slow query. +#[derive(Clone, Debug, Serialize, Deserialize, Default)] +#[serde(default)] +pub struct SlowQueryOptions { + /// Whether to enable slow query log. + pub enable: bool, + + /// The threshold of slow queries. + #[serde(with = "humantime_serde")] + pub threshold: Option, + + /// The sample ratio of slow queries. + pub sample_ratio: Option, } #[derive(Clone, Debug, Copy, PartialEq, Eq, Serialize, Deserialize)] @@ -96,6 +115,7 @@ impl Default for LoggingOptions { otlp_endpoint: None, tracing_sample_ratio: None, append_stdout: true, + slow_query: SlowQueryOptions::default(), } } } @@ -235,6 +255,42 @@ pub fn init_global_logging( None }; + let slow_query_logging_layer = if !opts.dir.is_empty() && opts.slow_query.enable { + let rolling_appender = + RollingFileAppender::new(Rotation::HOURLY, &opts.dir, "greptimedb-slow-queries"); + let (writer, guard) = tracing_appender::non_blocking(rolling_appender); + guards.push(guard); + + // Only logs if the field contains "slow". + let slow_query_filter = FilterFn::new(|metadata| { + metadata + .fields() + .iter() + .any(|field| field.name().contains("slow")) + }); + + if opts.log_format == LogFormat::Json { + Some( + Layer::new() + .json() + .with_writer(writer) + .with_ansi(false) + .with_filter(slow_query_filter) + .boxed(), + ) + } else { + Some( + Layer::new() + .with_writer(writer) + .with_ansi(false) + .with_filter(slow_query_filter) + .boxed(), + ) + } + } else { + None + }; + // resolve log level settings from: // - options from command line or config files // - environment variable: RUST_LOG @@ -279,6 +335,7 @@ pub fn init_global_logging( .with(stdout_logging_layer) .with(file_logging_layer) .with(err_file_logging_layer) + .with(slow_query_logging_layer) }; // consume the `tracing_opts` to avoid "unused" warnings. @@ -289,7 +346,8 @@ pub fn init_global_logging( .with(dyn_filter) .with(stdout_logging_layer) .with(file_logging_layer) - .with(err_file_logging_layer); + .with(err_file_logging_layer) + .with(slow_query_logging_layer); if opts.enable_otlp_tracing { global::set_text_map_propagator(TraceContextPropagator::new()); diff --git a/src/common/telemetry/src/macros.rs b/src/common/telemetry/src/macros.rs index cb838db6fe..2846bd5d20 100644 --- a/src/common/telemetry/src/macros.rs +++ b/src/common/telemetry/src/macros.rs @@ -152,6 +152,17 @@ macro_rules! trace { }; } +#[macro_export] +macro_rules! slow { + (target: $target:expr, $($arg:tt)+) => { + $crate::log!(target: $target, slow = true, $crate::tracing::Level::INFO, $($arg)+) + }; + + ($($arg:tt)+) => { + $crate::log!($crate::tracing::Level::INFO, slow = true, $($arg)+) + }; +} + #[cfg(test)] mod tests { use common_error::mock::MockError; diff --git a/src/flow/src/server.rs b/src/flow/src/server.rs index d78f9219cb..4381dd06a0 100644 --- a/src/flow/src/server.rs +++ b/src/flow/src/server.rs @@ -37,6 +37,7 @@ use operator::delete::Deleter; use operator::insert::Inserter; use operator::statement::StatementExecutor; use partition::manager::PartitionRuleManager; +use query::stats::StatementStatistics; use query::{QueryEngine, QueryEngineFactory}; use servers::error::{AlreadyStartedSnafu, StartGrpcSnafu, TcpBindSnafu, TcpIncomingSnafu}; use servers::server::Server; @@ -475,6 +476,7 @@ impl FrontendInvoker { layered_cache_registry.clone(), inserter.clone(), table_route_cache, + StatementStatistics::default(), )); let invoker = FrontendInvoker::new(inserter, deleter, statement_executor); diff --git a/src/frontend/src/instance/builder.rs b/src/frontend/src/instance/builder.rs index 5450e55ce2..a9513121d8 100644 --- a/src/frontend/src/instance/builder.rs +++ b/src/frontend/src/instance/builder.rs @@ -33,6 +33,7 @@ use operator::statement::{StatementExecutor, StatementExecutorRef}; use operator::table::TableMutationOperator; use partition::manager::PartitionRuleManager; use pipeline::pipeline_operator::PipelineOperator; +use query::stats::StatementStatistics; use query::QueryEngineFactory; use servers::server::ServerHandlers; use snafu::OptionExt; @@ -55,6 +56,7 @@ pub struct FrontendBuilder { plugins: Option, procedure_executor: ProcedureExecutorRef, heartbeat_task: Option, + stats: StatementStatistics, } impl FrontendBuilder { @@ -65,6 +67,7 @@ impl FrontendBuilder { catalog_manager: CatalogManagerRef, node_manager: NodeManagerRef, procedure_executor: ProcedureExecutorRef, + stats: StatementStatistics, ) -> Self { Self { options, @@ -76,6 +79,7 @@ impl FrontendBuilder { plugins: None, procedure_executor, heartbeat_task: None, + stats, } } @@ -181,6 +185,7 @@ impl FrontendBuilder { local_cache_invalidator, inserter.clone(), table_route_cache, + self.stats, )); let pipeline_operator = Arc::new(PipelineOperator::new( diff --git a/src/operator/src/statement.rs b/src/operator/src/statement.rs index 7c76d0dcff..033bd14e9c 100644 --- a/src/operator/src/statement.rs +++ b/src/operator/src/statement.rs @@ -45,6 +45,7 @@ use common_time::Timestamp; use datafusion_expr::LogicalPlan; use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef}; use query::parser::QueryStatement; +use query::stats::StatementStatistics; use query::QueryEngineRef; use session::context::{Channel, QueryContextRef}; use session::table_name::table_idents_to_full_name; @@ -80,11 +81,13 @@ pub struct StatementExecutor { partition_manager: PartitionRuleManagerRef, cache_invalidator: CacheInvalidatorRef, inserter: InserterRef, + stats: StatementStatistics, } pub type StatementExecutorRef = Arc; impl StatementExecutor { + #[allow(clippy::too_many_arguments)] pub fn new( catalog_manager: CatalogManagerRef, query_engine: QueryEngineRef, @@ -93,6 +96,7 @@ impl StatementExecutor { cache_invalidator: CacheInvalidatorRef, inserter: InserterRef, table_route_cache: TableRouteCacheRef, + stats: StatementStatistics, ) -> Self { Self { catalog_manager, @@ -104,6 +108,7 @@ impl StatementExecutor { partition_manager: Arc::new(PartitionRuleManager::new(kv_backend, table_route_cache)), cache_invalidator, inserter, + stats, } } @@ -113,6 +118,7 @@ impl StatementExecutor { stmt: QueryStatement, query_ctx: QueryContextRef, ) -> Result { + let _slow_query_timer = self.stats.start_slow_query_timer(stmt.clone()); match stmt { QueryStatement::Sql(stmt) => self.execute_sql(stmt, query_ctx).await, QueryStatement::Promql(_) => self.plan_exec(stmt, query_ctx).await, diff --git a/src/query/Cargo.toml b/src/query/Cargo.toml index eed0d5a339..4bcda7b0a7 100644 --- a/src/query/Cargo.toml +++ b/src/query/Cargo.toml @@ -54,6 +54,7 @@ prometheus.workspace = true promql.workspace = true promql-parser.workspace = true prost.workspace = true +rand.workspace = true regex.workspace = true session.workspace = true snafu.workspace = true diff --git a/src/query/src/lib.rs b/src/query/src/lib.rs index d6dfc5e097..e2843c708e 100644 --- a/src/query/src/lib.rs +++ b/src/query/src/lib.rs @@ -35,6 +35,7 @@ pub mod query_engine; mod range_select; pub mod region_query; pub mod sql; +pub mod stats; #[cfg(test)] mod tests; diff --git a/src/query/src/stats.rs b/src/query/src/stats.rs new file mode 100644 index 0000000000..896271897d --- /dev/null +++ b/src/query/src/stats.rs @@ -0,0 +1,98 @@ +// 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::time::Duration; + +use common_telemetry::logging::SlowQueryOptions; +use common_telemetry::slow; +use rand::random; + +use crate::parser::QueryStatement; + +/// StatementStatistics is used to collect statistics for a statement. +#[derive(Default, Clone, Debug)] +pub struct StatementStatistics { + /// slow_query is used to configure slow query log. + pub slow_query: SlowQueryOptions, +} + +impl StatementStatistics { + pub fn new(slow_query_options: SlowQueryOptions) -> Self { + Self { + slow_query: slow_query_options, + } + } + + pub fn start_slow_query_timer(&self, stmt: QueryStatement) -> Option { + if self.slow_query.enable { + Some(SlowQueryTimer { + start: std::time::Instant::now(), + stmt, + threshold: self.slow_query.threshold, + sample_ratio: self.slow_query.sample_ratio, + }) + } else { + None + } + } +} + +/// SlowQueryTimer is used to log slow query when it's dropped. +pub struct SlowQueryTimer { + start: std::time::Instant, + stmt: QueryStatement, + threshold: Option, + sample_ratio: Option, +} + +impl SlowQueryTimer { + fn log_slow_query(&self, elapsed: Duration, threshold: Duration) { + match &self.stmt { + QueryStatement::Sql(stmt) => { + slow!( + cost = elapsed.as_millis() as u64, + threshold = threshold.as_millis() as u64, + sql = stmt.to_string() + ); + } + QueryStatement::Promql(stmt) => { + slow!( + cost = elapsed.as_millis() as u64, + threshold = threshold.as_millis() as u64, + // TODO(zyy17): It's better to implement Display for EvalStmt for pretty print. + promql = format!("{:?}", stmt) + ); + } + } + } +} + +impl Drop for SlowQueryTimer { + fn drop(&mut self) { + if let Some(threshold) = self.threshold { + let elapsed = self.start.elapsed(); + if elapsed > threshold { + if let Some(ratio) = self.sample_ratio { + // Generate a random number in [0, 1) and compare it with sample_ratio. + if ratio >= 1.0 || random::() <= ratio { + self.log_slow_query(elapsed, threshold); + } + } else { + // If sample_ratio is not set, log all slow queries. + self.log_slow_query(elapsed, threshold); + } + } + } + } +} diff --git a/tests-integration/src/cluster.rs b/tests-integration/src/cluster.rs index ce2803996a..ad2c3e369f 100644 --- a/tests-integration/src/cluster.rs +++ b/tests-integration/src/cluster.rs @@ -48,6 +48,7 @@ use meta_client::client::MetaClientBuilder; use meta_srv::cluster::MetaPeerClientRef; use meta_srv::metasrv::{Metasrv, MetasrvOptions, SelectorRef}; use meta_srv::mocks::MockInfo; +use query::stats::StatementStatistics; use servers::grpc::flight::FlightCraftWrapper; use servers::grpc::region_server::RegionServerRequestHandler; use servers::heartbeat_options::HeartbeatOptions; @@ -393,6 +394,7 @@ impl GreptimeDbClusterBuilder { catalog_manager, datanode_clients, meta_client, + StatementStatistics::default(), ) .with_local_cache_invalidator(cache_registry) .with_heartbeat_task(heartbeat_task) diff --git a/tests-integration/src/standalone.rs b/tests-integration/src/standalone.rs index fa6e8f2a9e..123614e436 100644 --- a/tests-integration/src/standalone.rs +++ b/tests-integration/src/standalone.rs @@ -40,6 +40,7 @@ use flow::FlownodeBuilder; use frontend::instance::builder::FrontendBuilder; use frontend::instance::{FrontendInstance, Instance, StandaloneDatanodeManager}; use meta_srv::metasrv::{FLOW_ID_SEQ, TABLE_ID_SEQ}; +use query::stats::StatementStatistics; use servers::Mode; use snafu::ResultExt; @@ -215,6 +216,7 @@ impl GreptimeDbStandaloneBuilder { catalog_manager.clone(), node_manager.clone(), ddl_task_executor.clone(), + StatementStatistics::default(), ) .with_plugin(plugins) .try_build() diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index e11060fbbd..5f3eb854bb 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -836,6 +836,9 @@ retry_delay = "500ms" append_stdout = true enable_otlp_tracing = false +[logging.slow_query] +enable = false + [[region_engine]] [region_engine.mito] From 6e776d5f988c54ce8481064c5179c387985ec6cd Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Mon, 30 Sep 2024 16:28:51 +0800 Subject: [PATCH 050/128] feat: support to reject write after flushing (#4759) * refactor: use `RegionRoleState` instead of `RegionState` * feat: introducing `RegionLeaderState::Downgrading` * refactor: introduce `set_region_role_state_gracefully` * refactor: use `set_region_role` instead of `set_writable` * feat: support to reject write after flushing * fix: fix unit tests * test: add unit test for `should_reject_write` * chore: add comments * chore: refine comments * fix: fix unit test * test: enable fuzz tests for Local WAL * chore: add logs * chore: rename `RegionStatus` to `RegionState` * feat: introduce `DowngradingLeader` * chore: rename * refactor: refactor `set_role_state` tests * test: ensure downgrading region will reject write * chore: enhance logs * chore: refine name * chore: refine comment * test: add tests for `set_role_role_state` * fix: fix unit tests * chore: apply suggestions from CR * chore: apply suggestions from CR --- .github/workflows/develop.yml | 9 +- Cargo.lock | 2 +- Cargo.toml | 2 +- .../information_schema/region_peers.rs | 6 +- src/cmd/src/cli/bench.rs | 2 +- .../src/ddl/alter_table/region_request.rs | 2 +- src/common/meta/src/ddl/tests/alter_table.rs | 12 +- src/common/meta/src/ddl/tests/drop_table.rs | 6 +- src/common/meta/src/instruction.rs | 6 +- src/common/meta/src/key.rs | 40 +-- src/common/meta/src/key/table_route.rs | 43 ++- src/common/meta/src/region_keeper.rs | 2 +- src/common/meta/src/rpc/router.rs | 90 ++--- src/datanode/src/alive_keeper.rs | 16 +- src/datanode/src/datanode.rs | 6 +- src/datanode/src/heartbeat.rs | 4 +- src/datanode/src/heartbeat/handler.rs | 8 +- .../src/heartbeat/handler/downgrade_region.rs | 163 +++++---- .../src/heartbeat/handler/upgrade_region.rs | 2 +- src/datanode/src/region_server.rs | 33 +- src/datanode/src/tests.rs | 12 +- src/file-engine/src/engine.rs | 19 +- .../src/handler/region_lease_handler.rs | 6 +- .../downgrade_leader_region.rs | 96 +++++- .../src/procedure/region_migration/manager.rs | 2 +- .../procedure/region_migration/test_util.rs | 2 +- .../downgrade_leader_region.rs | 12 +- .../rollback_downgraded_region.rs | 16 +- .../upgrade_candidate_region.rs | 24 +- src/meta-srv/src/region/lease_keeper.rs | 12 +- src/meta-srv/src/test_util.rs | 2 +- src/metric-engine/src/engine.rs | 17 +- src/mito2/src/compaction/compactor.rs | 10 +- src/mito2/src/engine.rs | 34 +- src/mito2/src/engine/alter_test.rs | 8 +- src/mito2/src/engine/catchup_test.rs | 16 +- src/mito2/src/engine/compaction_test.rs | 8 +- src/mito2/src/engine/open_test.rs | 10 +- src/mito2/src/engine/set_readonly_test.rs | 98 ------ src/mito2/src/engine/set_role_state_test.rs | 159 +++++++++ src/mito2/src/error.rs | 23 +- src/mito2/src/flush.rs | 20 +- src/mito2/src/region.rs | 321 ++++++++++++++---- src/mito2/src/region/opener.rs | 23 +- src/mito2/src/request.rs | 17 +- src/mito2/src/test_util.rs | 6 +- src/mito2/src/test_util/scheduler_util.rs | 4 +- src/mito2/src/worker.rs | 24 +- src/mito2/src/worker/handle_catchup.rs | 6 +- src/mito2/src/worker/handle_drop.rs | 4 +- src/mito2/src/worker/handle_flush.rs | 16 +- src/mito2/src/worker/handle_manifest.rs | 21 +- src/mito2/src/worker/handle_truncate.rs | 4 +- src/operator/src/tests/partition_manager.rs | 6 +- src/query/src/optimizer/test_util.rs | 10 +- src/store-api/src/region_engine.rs | 52 ++- 56 files changed, 1078 insertions(+), 496 deletions(-) delete mode 100644 src/mito2/src/engine/set_readonly_test.rs create mode 100644 src/mito2/src/engine/set_role_state_test.rs diff --git a/.github/workflows/develop.yml b/.github/workflows/develop.yml index 073c407486..13c155d617 100644 --- a/.github/workflows/develop.yml +++ b/.github/workflows/develop.yml @@ -442,6 +442,13 @@ jobs: minio: true kafka: true values: "with-remote-wal.yaml" + include: + - target: "fuzz_migrate_mito_regions" + mode: + name: "Local WAL" + minio: true + kafka: false + values: "with-minio.yaml" steps: - name: Remove unused software run: | @@ -530,7 +537,7 @@ jobs: with: image-registry: localhost:5001 values-filename: ${{ matrix.mode.values }} - enable-region-failover: true + enable-region-failover: ${{ matrix.mode.kafka }} - name: Port forward (mysql) run: | kubectl port-forward service/my-greptimedb-frontend 4002:4002 -n my-greptimedb& diff --git a/Cargo.lock b/Cargo.lock index 7aebe2e719..ab5043939a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4414,7 +4414,7 @@ dependencies = [ [[package]] name = "greptime-proto" version = "0.1.0" -source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=36334744c7020734dcb4a6b8d24d52ae7ed53fe1#36334744c7020734dcb4a6b8d24d52ae7ed53fe1" +source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=0b4f7c8ab06399f6b90e1626e8d5b9697cb33bb9#0b4f7c8ab06399f6b90e1626e8d5b9697cb33bb9" dependencies = [ "prost 0.12.6", "serde", diff --git a/Cargo.toml b/Cargo.toml index 44b2cda122..63d7ad3ba7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -120,7 +120,7 @@ etcd-client = { version = "0.13" } fst = "0.4.7" futures = "0.3" futures-util = "0.3" -greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "36334744c7020734dcb4a6b8d24d52ae7ed53fe1" } +greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "0b4f7c8ab06399f6b90e1626e8d5b9697cb33bb9" } humantime = "2.1" humantime-serde = "1.1" itertools = "0.10" diff --git a/src/catalog/src/system_schema/information_schema/region_peers.rs b/src/catalog/src/system_schema/information_schema/region_peers.rs index 5496879af0..50c2593f86 100644 --- a/src/catalog/src/system_schema/information_schema/region_peers.rs +++ b/src/catalog/src/system_schema/information_schema/region_peers.rs @@ -224,8 +224,8 @@ impl InformationSchemaRegionPeersBuilder { let region_id = RegionId::new(table_id, route.region.id.region_number()).as_u64(); let peer_id = route.leader_peer.clone().map(|p| p.id); let peer_addr = route.leader_peer.clone().map(|p| p.addr); - let status = if let Some(status) = route.leader_status { - Some(status.as_ref().to_string()) + let state = if let Some(state) = route.leader_state { + Some(state.as_ref().to_string()) } else { // Alive by default Some("ALIVE".to_string()) @@ -242,7 +242,7 @@ impl InformationSchemaRegionPeersBuilder { self.peer_ids.push(peer_id); self.peer_addrs.push(peer_addr.as_deref()); self.is_leaders.push(Some("Yes")); - self.statuses.push(status.as_deref()); + self.statuses.push(state.as_deref()); self.down_seconds .push(route.leader_down_millis().map(|m| m / 1000)); } diff --git a/src/cmd/src/cli/bench.rs b/src/cmd/src/cli/bench.rs index bf5a6825f0..f3d1d0f809 100644 --- a/src/cmd/src/cli/bench.rs +++ b/src/cmd/src/cli/bench.rs @@ -158,7 +158,7 @@ fn create_region_routes(regions: Vec) -> Vec { addr: String::new(), }), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }); } diff --git a/src/common/meta/src/ddl/alter_table/region_request.rs b/src/common/meta/src/ddl/alter_table/region_request.rs index b4223b8ea0..0756360395 100644 --- a/src/common/meta/src/ddl/alter_table/region_request.rs +++ b/src/common/meta/src/ddl/alter_table/region_request.rs @@ -187,7 +187,7 @@ mod tests { region: Region::new_test(region_id), leader_peer: Some(Peer::empty(1)), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }]), HashMap::new(), diff --git a/src/common/meta/src/ddl/tests/alter_table.rs b/src/common/meta/src/ddl/tests/alter_table.rs index 06654cfe0f..36a1ff0ece 100644 --- a/src/common/meta/src/ddl/tests/alter_table.rs +++ b/src/common/meta/src/ddl/tests/alter_table.rs @@ -107,21 +107,21 @@ async fn test_on_submit_alter_request() { region: Region::new_test(RegionId::new(table_id, 1)), leader_peer: Some(Peer::empty(1)), follower_peers: vec![Peer::empty(5)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 2)), leader_peer: Some(Peer::empty(2)), follower_peers: vec![Peer::empty(4)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 3)), leader_peer: Some(Peer::empty(3)), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, ]), @@ -193,21 +193,21 @@ async fn test_on_submit_alter_request_with_outdated_request() { region: Region::new_test(RegionId::new(table_id, 1)), leader_peer: Some(Peer::empty(1)), follower_peers: vec![Peer::empty(5)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 2)), leader_peer: Some(Peer::empty(2)), follower_peers: vec![Peer::empty(4)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 3)), leader_peer: Some(Peer::empty(3)), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, ]), diff --git a/src/common/meta/src/ddl/tests/drop_table.rs b/src/common/meta/src/ddl/tests/drop_table.rs index aff1237472..c3a5f5875c 100644 --- a/src/common/meta/src/ddl/tests/drop_table.rs +++ b/src/common/meta/src/ddl/tests/drop_table.rs @@ -119,21 +119,21 @@ async fn test_on_datanode_drop_regions() { region: Region::new_test(RegionId::new(table_id, 1)), leader_peer: Some(Peer::empty(1)), follower_peers: vec![Peer::empty(5)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 2)), leader_peer: Some(Peer::empty(2)), follower_peers: vec![Peer::empty(4)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 3)), leader_peer: Some(Peer::empty(3)), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, ]), diff --git a/src/common/meta/src/instruction.rs b/src/common/meta/src/instruction.rs index 61e2811e72..4864f7562d 100644 --- a/src/common/meta/src/instruction.rs +++ b/src/common/meta/src/instruction.rs @@ -137,14 +137,16 @@ pub struct DowngradeRegion { /// `None` stands for don't flush before downgrading the region. #[serde(default)] pub flush_timeout: Option, + /// Rejects all write requests after flushing. + pub reject_write: bool, } impl Display for DowngradeRegion { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!( f, - "DowngradeRegion(region_id={}, flush_timeout={:?})", - self.region_id, self.flush_timeout, + "DowngradeRegion(region_id={}, flush_timeout={:?}, rejct_write={})", + self.region_id, self.flush_timeout, self.reject_write ) } } diff --git a/src/common/meta/src/key.rs b/src/common/meta/src/key.rs index d864882da6..0f703b9430 100644 --- a/src/common/meta/src/key.rs +++ b/src/common/meta/src/key.rs @@ -140,7 +140,7 @@ use crate::key::table_route::TableRouteKey; use crate::key::txn_helper::TxnOpGetResponseSet; use crate::kv_backend::txn::{Txn, TxnOp}; use crate::kv_backend::KvBackendRef; -use crate::rpc::router::{region_distribution, RegionRoute, RegionStatus}; +use crate::rpc::router::{region_distribution, LeaderState, RegionRoute}; use crate::rpc::store::BatchDeleteRequest; use crate::DatanodeId; @@ -1126,14 +1126,14 @@ impl TableMetadataManager { next_region_route_status: F, ) -> Result<()> where - F: Fn(&RegionRoute) -> Option>, + F: Fn(&RegionRoute) -> Option>, { let mut new_region_routes = current_table_route_value.region_routes()?.clone(); let mut updated = 0; for route in &mut new_region_routes { - if let Some(status) = next_region_route_status(route) { - if route.set_leader_status(status) { + if let Some(state) = next_region_route_status(route) { + if route.set_leader_state(state) { updated += 1; } } @@ -1280,7 +1280,7 @@ mod tests { use crate::key::{DeserializedValueWithBytes, TableMetadataManager, ViewInfoValue}; use crate::kv_backend::memory::MemoryKvBackend; use crate::peer::Peer; - use crate::rpc::router::{region_distribution, Region, RegionRoute, RegionStatus}; + use crate::rpc::router::{region_distribution, LeaderState, Region, RegionRoute}; #[test] fn test_deserialized_value_with_bytes() { @@ -1324,7 +1324,7 @@ mod tests { }, leader_peer: Some(Peer::new(datanode, "a2")), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, } } @@ -1715,7 +1715,7 @@ mod tests { attrs: BTreeMap::new(), }, leader_peer: Some(Peer::new(datanode, "a2")), - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), follower_peers: vec![], leader_down_since: Some(current_time_millis()), }, @@ -1727,7 +1727,7 @@ mod tests { attrs: BTreeMap::new(), }, leader_peer: Some(Peer::new(datanode, "a1")), - leader_status: None, + leader_state: None, follower_peers: vec![], leader_down_since: None, }, @@ -1750,10 +1750,10 @@ mod tests { table_metadata_manager .update_leader_region_status(table_id, ¤t_table_route_value, |region_route| { - if region_route.leader_status.is_some() { + if region_route.leader_state.is_some() { None } else { - Some(Some(RegionStatus::Downgraded)) + Some(Some(LeaderState::Downgrading)) } }) .await @@ -1768,8 +1768,8 @@ mod tests { .unwrap(); assert_eq!( - updated_route_value.region_routes().unwrap()[0].leader_status, - Some(RegionStatus::Downgraded) + updated_route_value.region_routes().unwrap()[0].leader_state, + Some(LeaderState::Downgrading) ); assert!(updated_route_value.region_routes().unwrap()[0] @@ -1777,8 +1777,8 @@ mod tests { .is_some()); assert_eq!( - updated_route_value.region_routes().unwrap()[1].leader_status, - Some(RegionStatus::Downgraded) + updated_route_value.region_routes().unwrap()[1].leader_state, + Some(LeaderState::Downgrading) ); assert!(updated_route_value.region_routes().unwrap()[1] .leader_down_since @@ -1943,21 +1943,21 @@ mod tests { region: Region::new_test(RegionId::new(table_id, 1)), leader_peer: Some(Peer::empty(1)), follower_peers: vec![Peer::empty(5)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 2)), leader_peer: Some(Peer::empty(2)), follower_peers: vec![Peer::empty(4)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 3)), leader_peer: Some(Peer::empty(3)), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, ]), @@ -1996,21 +1996,21 @@ mod tests { region: Region::new_test(RegionId::new(table_id, 1)), leader_peer: Some(Peer::empty(1)), follower_peers: vec![Peer::empty(5)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 2)), leader_peer: Some(Peer::empty(2)), follower_peers: vec![Peer::empty(4)], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 3)), leader_peer: Some(Peer::empty(3)), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, ]), diff --git a/src/common/meta/src/key/table_route.rs b/src/common/meta/src/key/table_route.rs index 0be0aab3aa..c9990ab121 100644 --- a/src/common/meta/src/key/table_route.rs +++ b/src/common/meta/src/key/table_route.rs @@ -744,6 +744,7 @@ mod tests { use crate::kv_backend::memory::MemoryKvBackend; use crate::kv_backend::{KvBackend, TxnService}; use crate::peer::Peer; + use crate::rpc::router::Region; use crate::rpc::store::PutRequest; #[test] @@ -751,11 +752,43 @@ mod tests { let old_raw_v = r#"{"region_routes":[{"region":{"id":1,"name":"r1","partition":null,"attrs":{}},"leader_peer":{"id":2,"addr":"a2"},"follower_peers":[]},{"region":{"id":1,"name":"r1","partition":null,"attrs":{}},"leader_peer":{"id":2,"addr":"a2"},"follower_peers":[]}],"version":0}"#; let v = TableRouteValue::try_from_raw_value(old_raw_v.as_bytes()).unwrap(); - let new_raw_v = format!("{:?}", v); - assert_eq!( - new_raw_v, - r#"Physical(PhysicalTableRouteValue { region_routes: [RegionRoute { region: Region { id: 1(0, 1), name: "r1", partition: None, attrs: {} }, leader_peer: Some(Peer { id: 2, addr: "a2" }), follower_peers: [], leader_status: None, leader_down_since: None }, RegionRoute { region: Region { id: 1(0, 1), name: "r1", partition: None, attrs: {} }, leader_peer: Some(Peer { id: 2, addr: "a2" }), follower_peers: [], leader_status: None, leader_down_since: None }], version: 0 })"# - ); + let expected_table_route = TableRouteValue::Physical(PhysicalTableRouteValue { + region_routes: vec![ + RegionRoute { + region: Region { + id: RegionId::new(0, 1), + name: "r1".to_string(), + partition: None, + attrs: Default::default(), + }, + leader_peer: Some(Peer { + id: 2, + addr: "a2".to_string(), + }), + follower_peers: vec![], + leader_state: None, + leader_down_since: None, + }, + RegionRoute { + region: Region { + id: RegionId::new(0, 1), + name: "r1".to_string(), + partition: None, + attrs: Default::default(), + }, + leader_peer: Some(Peer { + id: 2, + addr: "a2".to_string(), + }), + follower_peers: vec![], + leader_state: None, + leader_down_since: None, + }, + ], + version: 0, + }); + + assert_eq!(v, expected_table_route); } #[test] diff --git a/src/common/meta/src/region_keeper.rs b/src/common/meta/src/region_keeper.rs index a0d53b8477..54d5d6cc11 100644 --- a/src/common/meta/src/region_keeper.rs +++ b/src/common/meta/src/region_keeper.rs @@ -58,7 +58,7 @@ impl MemoryRegionKeeper { Default::default() } - /// Returns [OpeningRegionGuard] if Region(`region_id`) on Peer(`datanode_id`) does not exist. + /// Returns [OperatingRegionGuard] if Region(`region_id`) on Peer(`datanode_id`) does not exist. pub fn register( &self, datanode_id: DatanodeId, diff --git a/src/common/meta/src/rpc/router.rs b/src/common/meta/src/rpc/router.rs index 3e609e4af4..8dc409c8be 100644 --- a/src/common/meta/src/rpc/router.rs +++ b/src/common/meta/src/rpc/router.rs @@ -108,16 +108,16 @@ pub fn convert_to_region_peer_map( .collect::>() } -/// Returns the HashMap<[RegionNumber], [RegionStatus]>; -pub fn convert_to_region_leader_status_map( +/// Returns the HashMap<[RegionNumber], [LeaderState]>; +pub fn convert_to_region_leader_state_map( region_routes: &[RegionRoute], -) -> HashMap { +) -> HashMap { region_routes .iter() .filter_map(|x| { - x.leader_status + x.leader_state .as_ref() - .map(|status| (x.region.id.region_number(), *status)) + .map(|state| (x.region.id.region_number(), *state)) }) .collect::>() } @@ -205,7 +205,7 @@ impl TableRoute { region, leader_peer, follower_peers, - leader_status: None, + leader_state: None, leader_down_since: None, }); } @@ -259,9 +259,13 @@ pub struct RegionRoute { pub follower_peers: Vec, /// `None` by default. #[builder(setter(into, strip_option), default)] - #[serde(default, skip_serializing_if = "Option::is_none")] - pub leader_status: Option, - /// The start time when the leader is in `Downgraded` status. + #[serde( + default, + alias = "leader_status", + skip_serializing_if = "Option::is_none" + )] + pub leader_state: Option, + /// The start time when the leader is in `Downgraded` state. #[serde(default)] #[builder(default = "self.default_leader_down_since()")] pub leader_down_since: Option, @@ -269,76 +273,78 @@ pub struct RegionRoute { impl RegionRouteBuilder { fn default_leader_down_since(&self) -> Option { - match self.leader_status { - Some(Some(RegionStatus::Downgraded)) => Some(current_time_millis()), + match self.leader_state { + Some(Some(LeaderState::Downgrading)) => Some(current_time_millis()), _ => None, } } } -/// The Status of the [Region]. +/// The State of the [`Region`] Leader. /// TODO(dennis): It's better to add more fine-grained statuses such as `PENDING` etc. #[derive(Debug, Clone, Copy, Deserialize, Serialize, PartialEq, AsRefStr)] #[strum(serialize_all = "UPPERCASE")] -pub enum RegionStatus { - /// The following cases in which the [Region] will be downgraded. +pub enum LeaderState { + /// The following cases in which the [`Region`] will be downgraded. /// - /// - The [Region] is unavailable(e.g., Crashed, Network disconnected). - /// - The [Region] was planned to migrate to another [Peer]. - Downgraded, + /// - The [`Region`] may be unavailable (e.g., Crashed, Network disconnected). + /// - The [`Region`] was planned to migrate to another [`Peer`]. + Downgrading, } impl RegionRoute { - /// Returns true if the Leader [Region] is downgraded. + /// Returns true if the Leader [`Region`] is downgraded. /// - /// The following cases in which the [Region] will be downgraded. + /// The following cases in which the [`Region`] will be downgraded. /// - /// - The [Region] is unavailable(e.g., Crashed, Network disconnected). - /// - The [Region] was planned to migrate to another [Peer]. + /// - The [`Region`] is unavailable(e.g., Crashed, Network disconnected). + /// - The [`Region`] was planned to migrate to another [`Peer`]. /// - pub fn is_leader_downgraded(&self) -> bool { - matches!(self.leader_status, Some(RegionStatus::Downgraded)) + pub fn is_leader_downgrading(&self) -> bool { + matches!(self.leader_state, Some(LeaderState::Downgrading)) } - /// Marks the Leader [Region] as downgraded. + /// Marks the Leader [`Region`] as [`RegionState::Downgrading`]. /// - /// We should downgrade a [Region] before deactivating it: + /// We should downgrade a [`Region`] before deactivating it: /// - /// - During the [Region] Failover Procedure. - /// - Migrating a [Region]. + /// - During the [`Region`] Failover Procedure. + /// - Migrating a [`Region`]. /// - /// **Notes:** Meta Server will stop renewing the lease for the downgraded [Region]. + /// **Notes:** Meta Server will renewing a special lease(`Downgrading`) for the downgrading [`Region`]. + /// + /// A downgrading region will reject any write requests, and only allow memetable to be flushed to object storage /// pub fn downgrade_leader(&mut self) { self.leader_down_since = Some(current_time_millis()); - self.leader_status = Some(RegionStatus::Downgraded) + self.leader_state = Some(LeaderState::Downgrading) } - /// Returns how long since the leader is in `Downgraded` status. + /// Returns how long since the leader is in `Downgraded` state. pub fn leader_down_millis(&self) -> Option { self.leader_down_since .map(|start| current_time_millis() - start) } - /// Sets the leader status. + /// Sets the leader state. /// /// Returns true if updated. - pub fn set_leader_status(&mut self, status: Option) -> bool { - let updated = self.leader_status != status; + pub fn set_leader_state(&mut self, state: Option) -> bool { + let updated = self.leader_state != state; - match (status, updated) { - (Some(RegionStatus::Downgraded), true) => { + match (state, updated) { + (Some(LeaderState::Downgrading), true) => { self.leader_down_since = Some(current_time_millis()); } - (Some(RegionStatus::Downgraded), false) => { - // Do nothing if leader is still in `Downgraded` status. + (Some(LeaderState::Downgrading), false) => { + // Do nothing if leader is still in `Downgraded` state. } _ => { self.leader_down_since = None; } } - self.leader_status = status; + self.leader_state = state; updated } } @@ -477,15 +483,15 @@ mod tests { }, leader_peer: Some(Peer::new(1, "a1")), follower_peers: vec![Peer::new(2, "a2"), Peer::new(3, "a3")], - leader_status: None, + leader_state: None, leader_down_since: None, }; - assert!(!region_route.is_leader_downgraded()); + assert!(!region_route.is_leader_downgrading()); region_route.downgrade_leader(); - assert!(region_route.is_leader_downgraded()); + assert!(region_route.is_leader_downgrading()); } #[test] @@ -499,7 +505,7 @@ mod tests { }, leader_peer: Some(Peer::new(1, "a1")), follower_peers: vec![Peer::new(2, "a2"), Peer::new(3, "a3")], - leader_status: None, + leader_state: None, leader_down_since: None, }; diff --git a/src/datanode/src/alive_keeper.rs b/src/datanode/src/alive_keeper.rs index c6ef6cb3f6..a0ea2c0188 100644 --- a/src/datanode/src/alive_keeper.rs +++ b/src/datanode/src/alive_keeper.rs @@ -129,8 +129,10 @@ impl RegionAliveKeeper { let request = RegionRequest::Close(RegionCloseRequest {}); if let Err(e) = self.region_server.handle_request(region_id, request).await { if e.status_code() != StatusCode::RegionNotFound { - let _ = self.region_server.set_writable(region_id, false); - error!(e; "Failed to close staled region {}, set region to readonly.",region_id); + let _ = self + .region_server + .set_region_role(region_id, RegionRole::Follower); + error!(e; "Failed to close staled region {}, convert region to follower.", region_id); } } } @@ -378,7 +380,7 @@ impl CountdownTask { } }, Some(CountdownCommand::Reset((role, deadline))) => { - let _ = self.region_server.set_writable(self.region_id, role.writable()); + let _ = self.region_server.set_region_role(self.region_id, role); trace!( "Reset deadline of region {region_id} to approximately {} seconds later.", (deadline - Instant::now()).as_secs_f32(), @@ -399,8 +401,8 @@ impl CountdownTask { } } () = &mut countdown => { - warn!("The region {region_id} lease is expired, set region to readonly."); - let _ = self.region_server.set_writable(self.region_id, false); + warn!("The region {region_id} lease is expired, convert region to follower."); + let _ = self.region_server.set_region_role(self.region_id, RegionRole::Follower); // resets the countdown. let far_future = Instant::now() + Duration::from_secs(86400 * 365 * 30); countdown.as_mut().reset(far_future); @@ -436,7 +438,9 @@ mod test { .handle_request(region_id, RegionRequest::Create(builder.build())) .await .unwrap(); - region_server.set_writable(region_id, true).unwrap(); + region_server + .set_region_role(region_id, RegionRole::Leader) + .unwrap(); // Register a region before starting. alive_keeper.register_region(region_id).await; diff --git a/src/datanode/src/datanode.rs b/src/datanode/src/datanode.rs index f5d7bd9fc6..128a60ab9b 100644 --- a/src/datanode/src/datanode.rs +++ b/src/datanode/src/datanode.rs @@ -47,7 +47,7 @@ use servers::server::ServerHandlers; use servers::Mode; use snafu::{ensure, OptionExt, ResultExt}; use store_api::path_utils::{region_dir, WAL_DIR}; -use store_api::region_engine::RegionEngineRef; +use store_api::region_engine::{RegionEngineRef, RegionRole}; use store_api::region_request::RegionOpenRequest; use store_api::storage::RegionId; use tokio::fs; @@ -546,9 +546,9 @@ async fn open_all_regions( for region_id in open_regions { if open_with_writable { - if let Err(e) = region_server.set_writable(region_id, true) { + if let Err(e) = region_server.set_region_role(region_id, RegionRole::Leader) { error!( - e; "failed to set writable for region {region_id}" + e; "failed to convert region {region_id} to leader" ); } } diff --git a/src/datanode/src/heartbeat.rs b/src/datanode/src/heartbeat.rs index d84552a8d2..ef9af0acdd 100644 --- a/src/datanode/src/heartbeat.rs +++ b/src/datanode/src/heartbeat.rs @@ -126,7 +126,9 @@ impl HeartbeatTask { let mut follower_region_lease_count = 0; for lease in &lease.regions { match lease.role() { - RegionRole::Leader => leader_region_lease_count += 1, + RegionRole::Leader | RegionRole::DowngradingLeader => { + leader_region_lease_count += 1 + } RegionRole::Follower => follower_region_lease_count += 1, } } diff --git a/src/datanode/src/heartbeat/handler.rs b/src/datanode/src/heartbeat/handler.rs index d23615eb13..89b6991788 100644 --- a/src/datanode/src/heartbeat/handler.rs +++ b/src/datanode/src/heartbeat/handler.rs @@ -153,6 +153,7 @@ mod tests { use mito2::engine::MITO_ENGINE_NAME; use mito2::test_util::{CreateRequestBuilder, TestEnv}; use store_api::path_utils::region_dir; + use store_api::region_engine::RegionRole; use store_api::region_request::{RegionCloseRequest, RegionRequest}; use store_api::storage::RegionId; use tokio::sync::mpsc::{self, Receiver}; @@ -213,6 +214,7 @@ mod tests { let instruction = Instruction::DowngradeRegion(DowngradeRegion { region_id: RegionId::new(2048, 1), flush_timeout: Some(Duration::from_secs(1)), + reject_write: false, }); assert!(heartbeat_handler .is_acceptable(&heartbeat_env.create_handler_ctx((meta.clone(), instruction)))); @@ -295,7 +297,9 @@ mod tests { } assert_matches!( - region_server.set_writable(region_id, true).unwrap_err(), + region_server + .set_region_role(region_id, RegionRole::Leader) + .unwrap_err(), error::Error::RegionNotFound { .. } ); } @@ -411,6 +415,7 @@ mod tests { let instruction = Instruction::DowngradeRegion(DowngradeRegion { region_id, flush_timeout: Some(Duration::from_secs(1)), + reject_write: false, }); let mut ctx = heartbeat_env.create_handler_ctx((meta, instruction)); @@ -433,6 +438,7 @@ mod tests { let instruction = Instruction::DowngradeRegion(DowngradeRegion { region_id: RegionId::new(2048, 1), flush_timeout: Some(Duration::from_secs(1)), + reject_write: false, }); let mut ctx = heartbeat_env.create_handler_ctx((meta, instruction)); let control = heartbeat_handler.handle(&mut ctx).await.unwrap(); diff --git a/src/datanode/src/heartbeat/handler/downgrade_region.rs b/src/datanode/src/heartbeat/handler/downgrade_region.rs index ac11792803..fd85c75ba2 100644 --- a/src/datanode/src/heartbeat/handler/downgrade_region.rs +++ b/src/datanode/src/heartbeat/handler/downgrade_region.rs @@ -16,7 +16,7 @@ use common_meta::instruction::{DowngradeRegion, DowngradeRegionReply, Instructio use common_telemetry::tracing::info; use common_telemetry::warn; use futures_util::future::BoxFuture; -use store_api::region_engine::SetReadonlyResponse; +use store_api::region_engine::{SetRegionRoleStateResponse, SettableRegionRoleState}; use store_api::region_request::{RegionFlushRequest, RegionRequest}; use store_api::storage::RegionId; @@ -24,16 +24,20 @@ use crate::heartbeat::handler::HandlerContext; use crate::heartbeat::task_tracker::WaitResult; impl HandlerContext { - async fn set_readonly_gracefully(&self, region_id: RegionId) -> InstructionReply { - match self.region_server.set_readonly_gracefully(region_id).await { - Ok(SetReadonlyResponse::Success { last_entry_id }) => { + async fn downgrade_to_follower_gracefully(&self, region_id: RegionId) -> InstructionReply { + match self + .region_server + .set_region_role_state_gracefully(region_id, SettableRegionRoleState::Follower) + .await + { + Ok(SetRegionRoleStateResponse::Success { last_entry_id }) => { InstructionReply::DowngradeRegion(DowngradeRegionReply { last_entry_id, exists: true, error: None, }) } - Ok(SetReadonlyResponse::NotFound) => { + Ok(SetRegionRoleStateResponse::NotFound) => { InstructionReply::DowngradeRegion(DowngradeRegionReply { last_entry_id: None, exists: false, @@ -53,10 +57,12 @@ impl HandlerContext { DowngradeRegion { region_id, flush_timeout, + reject_write, }: DowngradeRegion, ) -> BoxFuture<'static, InstructionReply> { Box::pin(async move { - let Some(writable) = self.region_server.is_writable(region_id) else { + let Some(writable) = self.region_server.is_region_leader(region_id) else { + warn!("Region: {region_id} is not found"); return InstructionReply::DowngradeRegion(DowngradeRegionReply { last_entry_id: None, exists: false, @@ -64,61 +70,89 @@ impl HandlerContext { }); }; + let region_server_moved = self.region_server.clone(); + // Ignores flush request if !writable { - return self.set_readonly_gracefully(region_id).await; + return self.downgrade_to_follower_gracefully(region_id).await; } - let region_server_moved = self.region_server.clone(); - if let Some(flush_timeout) = flush_timeout { - let register_result = self - .downgrade_tasks - .try_register( + // If flush_timeout is not set, directly convert region to follower. + let Some(flush_timeout) = flush_timeout else { + return self.downgrade_to_follower_gracefully(region_id).await; + }; + + if reject_write { + // Sets region to downgrading, the downgrading region will reject all write requests. + match self + .region_server + .set_region_role_state_gracefully( region_id, - Box::pin(async move { - info!("Flush region: {region_id} before downgrading region"); - region_server_moved - .handle_request( - region_id, - RegionRequest::Flush(RegionFlushRequest { - row_group_size: None, - }), - ) - .await?; - - Ok(()) - }), + SettableRegionRoleState::DowngradingLeader, ) - .await; - - if register_result.is_busy() { - warn!("Another flush task is running for the region: {region_id}"); - } - - let mut watcher = register_result.into_watcher(); - let result = self.catchup_tasks.wait(&mut watcher, flush_timeout).await; - - match result { - WaitResult::Timeout => { - InstructionReply::DowngradeRegion(DowngradeRegionReply { + .await + { + Ok(SetRegionRoleStateResponse::Success { .. }) => {} + Ok(SetRegionRoleStateResponse::NotFound) => { + warn!("Region: {region_id} is not found"); + return InstructionReply::DowngradeRegion(DowngradeRegionReply { last_entry_id: None, - exists: true, - error: Some(format!( - "Flush region: {region_id} before downgrading region is timeout" - )), - }) + exists: false, + error: None, + }); } - WaitResult::Finish(Ok(_)) => self.set_readonly_gracefully(region_id).await, - WaitResult::Finish(Err(err)) => { - InstructionReply::DowngradeRegion(DowngradeRegionReply { + Err(err) => { + warn!(err; "Failed to convert region to downgrading leader"); + return InstructionReply::DowngradeRegion(DowngradeRegionReply { last_entry_id: None, exists: true, error: Some(format!("{err:?}")), - }) + }); } } - } else { - self.set_readonly_gracefully(region_id).await + } + + let register_result = self + .downgrade_tasks + .try_register( + region_id, + Box::pin(async move { + info!("Flush region: {region_id} before converting region to follower"); + region_server_moved + .handle_request( + region_id, + RegionRequest::Flush(RegionFlushRequest { + row_group_size: None, + }), + ) + .await?; + + Ok(()) + }), + ) + .await; + + if register_result.is_busy() { + warn!("Another flush task is running for the region: {region_id}"); + } + + let mut watcher = register_result.into_watcher(); + let result = self.catchup_tasks.wait(&mut watcher, flush_timeout).await; + + match result { + WaitResult::Timeout => InstructionReply::DowngradeRegion(DowngradeRegionReply { + last_entry_id: None, + exists: true, + error: Some(format!("Flush region: {region_id} is timeout")), + }), + WaitResult::Finish(Ok(_)) => self.downgrade_to_follower_gracefully(region_id).await, + WaitResult::Finish(Err(err)) => { + InstructionReply::DowngradeRegion(DowngradeRegionReply { + last_entry_id: None, + exists: true, + error: Some(format!("{err:?}")), + }) + } } }) } @@ -131,7 +165,7 @@ mod tests { use common_meta::instruction::{DowngradeRegion, InstructionReply}; use mito2::engine::MITO_ENGINE_NAME; - use store_api::region_engine::{RegionRole, SetReadonlyResponse}; + use store_api::region_engine::{RegionRole, SetRegionRoleStateResponse}; use store_api::region_request::RegionRequest; use store_api::storage::RegionId; use tokio::time::Instant; @@ -155,6 +189,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout, + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -182,8 +217,9 @@ mod tests { Ok(0) })); - region_engine.handle_set_readonly_gracefully_mock_fn = - Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + region_engine.handle_set_readonly_gracefully_mock_fn = Some(Box::new(|_| { + Ok(SetRegionRoleStateResponse::success(Some(1024))) + })) }); mock_region_server.register_test_region(region_id, mock_engine); let handler_context = HandlerContext::new_for_test(mock_region_server); @@ -195,6 +231,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout, + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -215,8 +252,9 @@ mod tests { MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { region_engine.mock_role = Some(Some(RegionRole::Leader)); region_engine.handle_request_delay = Some(Duration::from_secs(100)); - region_engine.handle_set_readonly_gracefully_mock_fn = - Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + region_engine.handle_set_readonly_gracefully_mock_fn = Some(Box::new(|_| { + Ok(SetRegionRoleStateResponse::success(Some(1024))) + })) }); mock_region_server.register_test_region(region_id, mock_engine); let handler_context = HandlerContext::new_for_test(mock_region_server); @@ -227,6 +265,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout: Some(flush_timeout), + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -246,8 +285,9 @@ mod tests { MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { region_engine.mock_role = Some(Some(RegionRole::Leader)); region_engine.handle_request_delay = Some(Duration::from_millis(300)); - region_engine.handle_set_readonly_gracefully_mock_fn = - Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + region_engine.handle_set_readonly_gracefully_mock_fn = Some(Box::new(|_| { + Ok(SetRegionRoleStateResponse::success(Some(1024))) + })) }); mock_region_server.register_test_region(region_id, mock_engine); let handler_context = HandlerContext::new_for_test(mock_region_server); @@ -263,6 +303,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout, + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -277,6 +318,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout: Some(Duration::from_millis(500)), + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -304,8 +346,9 @@ mod tests { } .fail() })); - region_engine.handle_set_readonly_gracefully_mock_fn = - Some(Box::new(|_| Ok(SetReadonlyResponse::success(Some(1024))))) + region_engine.handle_set_readonly_gracefully_mock_fn = Some(Box::new(|_| { + Ok(SetRegionRoleStateResponse::success(Some(1024))) + })) }); mock_region_server.register_test_region(region_id, mock_engine); let handler_context = HandlerContext::new_for_test(mock_region_server); @@ -321,6 +364,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout, + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -335,6 +379,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout: Some(Duration::from_millis(500)), + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -356,7 +401,7 @@ mod tests { MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| { region_engine.mock_role = Some(Some(RegionRole::Leader)); region_engine.handle_set_readonly_gracefully_mock_fn = - Some(Box::new(|_| Ok(SetReadonlyResponse::NotFound))); + Some(Box::new(|_| Ok(SetRegionRoleStateResponse::NotFound))); }); mock_region_server.register_test_region(region_id, mock_engine); let handler_context = HandlerContext::new_for_test(mock_region_server); @@ -365,6 +410,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout: None, + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); @@ -396,6 +442,7 @@ mod tests { .handle_downgrade_region_instruction(DowngradeRegion { region_id, flush_timeout: None, + reject_write: false, }) .await; assert_matches!(reply, InstructionReply::DowngradeRegion(_)); diff --git a/src/datanode/src/heartbeat/handler/upgrade_region.rs b/src/datanode/src/heartbeat/handler/upgrade_region.rs index 0d1ef0476c..9acb3da9c3 100644 --- a/src/datanode/src/heartbeat/handler/upgrade_region.rs +++ b/src/datanode/src/heartbeat/handler/upgrade_region.rs @@ -31,7 +31,7 @@ impl HandlerContext { }: UpgradeRegion, ) -> BoxFuture<'static, InstructionReply> { Box::pin(async move { - let Some(writable) = self.region_server.is_writable(region_id) else { + let Some(writable) = self.region_server.is_region_leader(region_id) else { return InstructionReply::UpgradeRegion(UpgradeRegionReply { ready: false, exists: false, diff --git a/src/datanode/src/region_server.rs b/src/datanode/src/region_server.rs index aa80f52a5c..0bac53e4d6 100644 --- a/src/datanode/src/region_server.rs +++ b/src/datanode/src/region_server.rs @@ -54,7 +54,10 @@ use snafu::{ensure, OptionExt, ResultExt}; use store_api::metric_engine_consts::{ FILE_ENGINE_NAME, LOGICAL_TABLE_METADATA_KEY, METRIC_ENGINE_NAME, }; -use store_api::region_engine::{RegionEngineRef, RegionRole, RegionStatistic, SetReadonlyResponse}; +use store_api::region_engine::{ + RegionEngineRef, RegionRole, RegionStatistic, SetRegionRoleStateResponse, + SettableRegionRoleState, +}; use store_api::region_request::{ AffectedRows, RegionCloseRequest, RegionOpenRequest, RegionRequest, }; @@ -274,37 +277,47 @@ impl RegionServer { .collect() } - pub fn is_writable(&self, region_id: RegionId) -> Option { - // TODO(weny): Finds a better way. + pub fn is_region_leader(&self, region_id: RegionId) -> Option { self.inner.region_map.get(®ion_id).and_then(|engine| { engine.role(region_id).map(|role| match role { RegionRole::Follower => false, RegionRole::Leader => true, + RegionRole::DowngradingLeader => true, }) }) } - pub fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<()> { + pub fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<()> { let engine = self .inner .region_map .get(®ion_id) .with_context(|| RegionNotFoundSnafu { region_id })?; engine - .set_writable(region_id, writable) + .set_region_role(region_id, role) .with_context(|_| HandleRegionRequestSnafu { region_id }) } - pub async fn set_readonly_gracefully( + /// Set region role state gracefully. + /// + /// For [SettableRegionRoleState::Follower]: + /// After the call returns, the engine ensures that + /// no **further** write or flush operations will succeed in this region. + /// + /// For [SettableRegionRoleState::DowngradingLeader]: + /// After the call returns, the engine ensures that + /// no **further** write operations will succeed in this region. + pub async fn set_region_role_state_gracefully( &self, region_id: RegionId, - ) -> Result { + state: SettableRegionRoleState, + ) -> Result { match self.inner.region_map.get(®ion_id) { Some(engine) => Ok(engine - .set_readonly_gracefully(region_id) + .set_region_role_state_gracefully(region_id, state) .await .with_context(|_| HandleRegionRequestSnafu { region_id })?), - None => Ok(SetReadonlyResponse::NotFound), + None => Ok(SetRegionRoleStateResponse::NotFound), } } @@ -842,7 +855,7 @@ impl RegionServerInner { info!("Region {region_id} is deregistered from engine {engine_type}"); self.region_map .remove(®ion_id) - .map(|(id, engine)| engine.set_writable(id, false)); + .map(|(id, engine)| engine.set_region_role(id, RegionRole::Follower)); self.event_listener.on_region_deregistered(region_id); } RegionChange::Catchup => { diff --git a/src/datanode/src/tests.rs b/src/datanode/src/tests.rs index 35f513bc83..2acc66a592 100644 --- a/src/datanode/src/tests.rs +++ b/src/datanode/src/tests.rs @@ -32,7 +32,8 @@ use query::{QueryEngine, QueryEngineContext}; use session::context::QueryContextRef; use store_api::metadata::RegionMetadataRef; use store_api::region_engine::{ - RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetReadonlyResponse, + RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetRegionRoleStateResponse, + SettableRegionRoleState, }; use store_api::region_request::{AffectedRows, RegionRequest}; use store_api::storage::{RegionId, ScanRequest}; @@ -106,7 +107,7 @@ pub type MockRequestHandler = Box Result + Send + Sync>; pub type MockSetReadonlyGracefullyHandler = - Box Result + Send + Sync>; + Box Result + Send + Sync>; pub struct MockRegionEngine { sender: Sender<(RegionId, RegionRequest)>, @@ -220,14 +221,15 @@ impl RegionEngine for MockRegionEngine { Ok(()) } - fn set_writable(&self, _region_id: RegionId, _writable: bool) -> Result<(), BoxedError> { + fn set_region_role(&self, _region_id: RegionId, _role: RegionRole) -> Result<(), BoxedError> { Ok(()) } - async fn set_readonly_gracefully( + async fn set_region_role_state_gracefully( &self, region_id: RegionId, - ) -> Result { + _region_role_state: SettableRegionRoleState, + ) -> Result { if let Some(mock_fn) = &self.handle_set_readonly_gracefully_mock_fn { return mock_fn(region_id).map_err(BoxedError::new); }; diff --git a/src/file-engine/src/engine.rs b/src/file-engine/src/engine.rs index 32e1a1d58d..e6313f4322 100644 --- a/src/file-engine/src/engine.rs +++ b/src/file-engine/src/engine.rs @@ -26,8 +26,8 @@ use object_store::ObjectStore; use snafu::{ensure, OptionExt}; use store_api::metadata::RegionMetadataRef; use store_api::region_engine::{ - RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetReadonlyResponse, - SinglePartitionScanner, + RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetRegionRoleStateResponse, + SettableRegionRoleState, SinglePartitionScanner, }; use store_api::region_request::{ AffectedRows, RegionCloseRequest, RegionCreateRequest, RegionDropRequest, RegionOpenRequest, @@ -113,22 +113,23 @@ impl RegionEngine for FileRegionEngine { None } - fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<(), BoxedError> { + fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<(), BoxedError> { self.inner - .set_writable(region_id, writable) + .set_region_role(region_id, role) .map_err(BoxedError::new) } - async fn set_readonly_gracefully( + async fn set_region_role_state_gracefully( &self, region_id: RegionId, - ) -> Result { + _region_role_state: SettableRegionRoleState, + ) -> Result { let exists = self.inner.get_region(region_id).await.is_some(); if exists { - Ok(SetReadonlyResponse::success(None)) + Ok(SetRegionRoleStateResponse::success(None)) } else { - Ok(SetReadonlyResponse::NotFound) + Ok(SetRegionRoleStateResponse::NotFound) } } @@ -189,7 +190,7 @@ impl EngineInner { Ok(()) } - fn set_writable(&self, _region_id: RegionId, _writable: bool) -> EngineResult<()> { + fn set_region_role(&self, _region_id: RegionId, _region_role: RegionRole) -> EngineResult<()> { // TODO(zhongzc): Improve the semantics and implementation of this API. Ok(()) } diff --git a/src/meta-srv/src/handler/region_lease_handler.rs b/src/meta-srv/src/handler/region_lease_handler.rs index 06cf818d23..de491da371 100644 --- a/src/meta-srv/src/handler/region_lease_handler.rs +++ b/src/meta-srv/src/handler/region_lease_handler.rs @@ -111,7 +111,7 @@ mod test { use common_meta::kv_backend::memory::MemoryKvBackend; use common_meta::peer::Peer; use common_meta::region_keeper::MemoryRegionKeeper; - use common_meta::rpc::router::{Region, RegionRoute, RegionStatus}; + use common_meta::rpc::router::{LeaderState, Region, RegionRoute}; use store_api::region_engine::RegionRole; use store_api::storage::RegionId; @@ -297,7 +297,7 @@ mod test { region: Region::new_test(region_id), leader_peer: Some(peer.clone()), follower_peers: vec![follower_peer.clone()], - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), leader_down_since: Some(1), }, RegionRoute { @@ -352,7 +352,7 @@ mod test { assert_region_lease( acc, vec![ - GrantedRegion::new(region_id, RegionRole::Follower), + GrantedRegion::new(region_id, RegionRole::DowngradingLeader), GrantedRegion::new(another_region_id, RegionRole::Leader), ], ); diff --git a/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs b/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs index 836ca4c532..ec5114b9eb 100644 --- a/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs +++ b/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs @@ -22,8 +22,10 @@ use common_meta::instruction::{ }; use common_procedure::Status; use common_telemetry::{error, info, warn}; +use common_wal::options::WalOptions; use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; +use store_api::storage::RegionId; use tokio::time::{sleep, Instant}; use super::update_metadata::UpdateMetadata; @@ -95,15 +97,32 @@ impl DowngradeLeaderRegion { &self, ctx: &Context, flush_timeout: Duration, + reject_write: bool, ) -> Instruction { let pc = &ctx.persistent_ctx; let region_id = pc.region_id; Instruction::DowngradeRegion(DowngradeRegion { region_id, flush_timeout: Some(flush_timeout), + reject_write, }) } + async fn should_reject_write(ctx: &mut Context, region_id: RegionId) -> Result { + let datanode_table_value = ctx.get_from_peer_datanode_table_value().await?; + if let Some(wal_option) = datanode_table_value + .region_info + .region_wal_options + .get(®ion_id.region_number()) + { + let options: WalOptions = serde_json::from_str(wal_option) + .with_context(|_| error::DeserializeFromJsonSnafu { input: wal_option })?; + return Ok(matches!(options, WalOptions::RaftEngine)); + } + + Ok(true) + } + /// Tries to downgrade a leader region. /// /// Retry: @@ -118,16 +137,17 @@ impl DowngradeLeaderRegion { /// - [ExceededDeadline](error::Error::ExceededDeadline) /// - Invalid JSON. async fn downgrade_region(&self, ctx: &mut Context) -> Result<()> { - let pc = &ctx.persistent_ctx; - let region_id = pc.region_id; - let leader = &pc.from_peer; + let region_id = ctx.persistent_ctx.region_id; let operation_timeout = ctx.next_operation_timeout() .context(error::ExceededDeadlineSnafu { operation: "Downgrade region", })?; - let downgrade_instruction = self.build_downgrade_region_instruction(ctx, operation_timeout); + let reject_write = Self::should_reject_write(ctx, region_id).await?; + let downgrade_instruction = + self.build_downgrade_region_instruction(ctx, operation_timeout, reject_write); + let leader = &ctx.persistent_ctx.from_peer; let msg = MailboxMessage::json_message( &format!("Downgrade leader region: {}", region_id), &format!("Meta@{}", ctx.server_addr()), @@ -240,8 +260,13 @@ impl DowngradeLeaderRegion { #[cfg(test)] mod tests { use std::assert_matches::assert_matches; + use std::collections::HashMap; + use common_meta::key::table_route::TableRouteValue; + use common_meta::key::test_utils::new_test_table_info; use common_meta::peer::Peer; + use common_meta::rpc::router::{Region, RegionRoute}; + use common_wal::options::KafkaWalOptions; use store_api::storage::RegionId; use tokio::time::Instant; @@ -264,19 +289,73 @@ mod tests { } } + async fn prepare_table_metadata(ctx: &Context, wal_options: HashMap) { + let table_info = + new_test_table_info(ctx.persistent_ctx.region_id.table_id(), vec![1]).into(); + let region_routes = vec![RegionRoute { + region: Region::new_test(ctx.persistent_ctx.region_id), + leader_peer: Some(ctx.persistent_ctx.from_peer.clone()), + follower_peers: vec![ctx.persistent_ctx.to_peer.clone()], + ..Default::default() + }]; + ctx.table_metadata_manager + .create_table_metadata( + table_info, + TableRouteValue::physical(region_routes), + wal_options, + ) + .await + .unwrap(); + } + #[tokio::test] async fn test_datanode_is_unreachable() { let state = DowngradeLeaderRegion::default(); let persistent_context = new_persistent_context(); let env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); - + prepare_table_metadata(&ctx, HashMap::default()).await; let err = state.downgrade_region(&mut ctx).await.unwrap_err(); assert_matches!(err, Error::PusherNotFound { .. }); assert!(!err.is_retryable()); } + #[tokio::test] + async fn test_should_reject_writes() { + let persistent_context = new_persistent_context(); + let region_id = persistent_context.region_id; + let env = TestingEnv::new(); + let mut ctx = env.context_factory().new_context(persistent_context); + let wal_options = + HashMap::from([(1, serde_json::to_string(&WalOptions::RaftEngine).unwrap())]); + prepare_table_metadata(&ctx, wal_options).await; + + let reject_write = DowngradeLeaderRegion::should_reject_write(&mut ctx, region_id) + .await + .unwrap(); + assert!(reject_write); + + // Remote WAL + let persistent_context = new_persistent_context(); + let region_id = persistent_context.region_id; + let env = TestingEnv::new(); + let mut ctx = env.context_factory().new_context(persistent_context); + let wal_options = HashMap::from([( + 1, + serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions { + topic: "my_topic".to_string(), + })) + .unwrap(), + )]); + prepare_table_metadata(&ctx, wal_options).await; + + let reject_write = DowngradeLeaderRegion::should_reject_write(&mut ctx, region_id) + .await + .unwrap(); + assert!(!reject_write); + } + #[tokio::test] async fn test_pusher_dropped() { let state = DowngradeLeaderRegion::default(); @@ -285,6 +364,7 @@ mod tests { let mut env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); + prepare_table_metadata(&ctx, HashMap::default()).await; let mailbox_ctx = env.mailbox_context(); let (tx, rx) = tokio::sync::mpsc::channel(1); @@ -307,6 +387,7 @@ mod tests { let persistent_context = new_persistent_context(); let env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); + prepare_table_metadata(&ctx, HashMap::default()).await; ctx.volatile_ctx.operations_elapsed = ctx.persistent_ctx.timeout + Duration::from_secs(1); let err = state.downgrade_region(&mut ctx).await.unwrap_err(); @@ -330,6 +411,7 @@ mod tests { let mut env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); + prepare_table_metadata(&ctx, HashMap::default()).await; let mailbox_ctx = env.mailbox_context(); let mailbox = mailbox_ctx.mailbox().clone(); @@ -356,6 +438,7 @@ mod tests { let mut env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); + prepare_table_metadata(&ctx, HashMap::default()).await; let mailbox_ctx = env.mailbox_context(); let mailbox = mailbox_ctx.mailbox().clone(); @@ -383,6 +466,7 @@ mod tests { let mut env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); + prepare_table_metadata(&ctx, HashMap::default()).await; let mailbox_ctx = env.mailbox_context(); let mailbox = mailbox_ctx.mailbox().clone(); @@ -416,6 +500,7 @@ mod tests { let mut env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); + prepare_table_metadata(&ctx, HashMap::default()).await; let mailbox_ctx = env.mailbox_context(); let mailbox = mailbox_ctx.mailbox().clone(); @@ -508,6 +593,7 @@ mod tests { let mut env = TestingEnv::new(); let mut ctx = env.context_factory().new_context(persistent_context); + prepare_table_metadata(&ctx, HashMap::default()).await; let mailbox_ctx = env.mailbox_context(); let mailbox = mailbox_ctx.mailbox().clone(); diff --git a/src/meta-srv/src/procedure/region_migration/manager.rs b/src/meta-srv/src/procedure/region_migration/manager.rs index bb3eff80c0..01ea887ca9 100644 --- a/src/meta-srv/src/procedure/region_migration/manager.rs +++ b/src/meta-srv/src/procedure/region_migration/manager.rs @@ -246,7 +246,7 @@ impl RegionMigrationManager { region_route: &RegionRoute, task: &RegionMigrationProcedureTask, ) -> Result { - if region_route.is_leader_downgraded() { + if region_route.is_leader_downgrading() { return Ok(false); } diff --git a/src/meta-srv/src/procedure/region_migration/test_util.rs b/src/meta-srv/src/procedure/region_migration/test_util.rs index 65e33ab3d9..cb3b5a3dc3 100644 --- a/src/meta-srv/src/procedure/region_migration/test_util.rs +++ b/src/meta-srv/src/procedure/region_migration/test_util.rs @@ -449,7 +449,7 @@ impl ProcedureMigrationTestSuite { .find(|route| route.region.id == region_id) .unwrap(); - assert!(!region_route.is_leader_downgraded()); + assert!(!region_route.is_leader_downgrading()); assert_eq!( region_route.leader_peer.as_ref().unwrap().id, expected_leader_id diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs index 3b3f6a6c0c..d8bad44871 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs @@ -13,7 +13,7 @@ // limitations under the License. use common_error::ext::BoxedError; -use common_meta::rpc::router::RegionStatus; +use common_meta::rpc::router::LeaderState; use snafu::ResultExt; use crate::error::{self, Result}; @@ -53,7 +53,7 @@ impl UpdateMetadata { .as_ref() .is_some_and(|leader_peer| leader_peer.id == from_peer_id) { - Some(Some(RegionStatus::Downgraded)) + Some(Some(LeaderState::Downgrading)) } else { None } @@ -81,7 +81,7 @@ mod tests { use common_meta::key::test_utils::new_test_table_info; use common_meta::peer::Peer; - use common_meta::rpc::router::{Region, RegionRoute, RegionStatus}; + use common_meta::rpc::router::{LeaderState, Region, RegionRoute}; use store_api::storage::RegionId; use crate::error::Error; @@ -155,7 +155,7 @@ mod tests { table_metadata_manager .update_leader_region_status(table_id, &original_table_route, |route| { if route.region.id == RegionId::new(1024, 2) { - Some(Some(RegionStatus::Downgraded)) + Some(Some(LeaderState::Downgrading)) } else { None } @@ -210,7 +210,7 @@ mod tests { // It should remain unchanged. assert_eq!(latest_table_route.version().unwrap(), 0); - assert!(!latest_table_route.region_routes().unwrap()[0].is_leader_downgraded()); + assert!(!latest_table_route.region_routes().unwrap()[0].is_leader_downgrading()); assert!(ctx.volatile_ctx.table_route.is_none()); } @@ -251,7 +251,7 @@ mod tests { .unwrap() .unwrap(); - assert!(latest_table_route.region_routes().unwrap()[0].is_leader_downgraded()); + assert!(latest_table_route.region_routes().unwrap()[0].is_leader_downgrading()); assert!(ctx.volatile_ctx.table_route.is_none()); } } diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs index 4e6f20ef19..0d568ab7b0 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs @@ -65,7 +65,7 @@ mod tests { use common_meta::key::test_utils::new_test_table_info; use common_meta::peer::Peer; - use common_meta::rpc::router::{Region, RegionRoute, RegionStatus}; + use common_meta::rpc::router::{LeaderState, Region, RegionRoute}; use store_api::storage::RegionId; use crate::error::Error; @@ -110,13 +110,13 @@ mod tests { RegionRoute { region: Region::new_test(RegionId::new(1024, 1)), leader_peer: Some(from_peer.clone()), - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), ..Default::default() }, RegionRoute { region: Region::new_test(RegionId::new(1024, 2)), leader_peer: Some(Peer::empty(4)), - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), ..Default::default() }, RegionRoute { @@ -128,8 +128,8 @@ mod tests { let expected_region_routes = { let mut region_routes = region_routes.clone(); - region_routes[0].leader_status = None; - region_routes[1].leader_status = None; + region_routes[0].leader_state = None; + region_routes[1].leader_state = None; region_routes }; @@ -207,13 +207,13 @@ mod tests { RegionRoute { region: Region::new_test(RegionId::new(1024, 1)), leader_peer: Some(from_peer.clone()), - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), ..Default::default() }, RegionRoute { region: Region::new_test(RegionId::new(1024, 2)), leader_peer: Some(Peer::empty(4)), - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), ..Default::default() }, RegionRoute { @@ -225,7 +225,7 @@ mod tests { let expected_region_routes = { let mut region_routes = region_routes.clone(); - region_routes[0].leader_status = None; + region_routes[0].leader_state = None; region_routes }; diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs index 75f93f760e..b710a0e1f3 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs @@ -43,7 +43,7 @@ impl UpdateMetadata { .context(error::RegionRouteNotFoundSnafu { region_id })?; // Removes downgraded status. - region_route.set_leader_status(None); + region_route.set_leader_state(None); let candidate = &ctx.persistent_ctx.to_peer; let expected_old_leader = &ctx.persistent_ctx.from_peer; @@ -106,7 +106,7 @@ impl UpdateMetadata { if leader_peer.id == candidate_peer_id { ensure!( - !region_route.is_leader_downgraded(), + !region_route.is_leader_downgrading(), error::UnexpectedSnafu { violated: format!("Unexpected intermediate state is found during the update metadata for upgrading region {region_id}"), } @@ -190,7 +190,7 @@ mod tests { use common_meta::key::test_utils::new_test_table_info; use common_meta::peer::Peer; use common_meta::region_keeper::MemoryRegionKeeper; - use common_meta::rpc::router::{Region, RegionRoute, RegionStatus}; + use common_meta::rpc::router::{LeaderState, Region, RegionRoute}; use common_time::util::current_time_millis; use store_api::storage::RegionId; @@ -286,7 +286,7 @@ mod tests { region: Region::new_test(RegionId::new(1024, 1)), leader_peer: Some(Peer::empty(1)), follower_peers: vec![Peer::empty(2), Peer::empty(3)], - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), leader_down_since: Some(current_time_millis()), }]; @@ -298,7 +298,7 @@ mod tests { .await .unwrap(); - assert!(!new_region_routes[0].is_leader_downgraded()); + assert!(!new_region_routes[0].is_leader_downgrading()); assert!(new_region_routes[0].leader_down_since.is_none()); assert_eq!(new_region_routes[0].follower_peers, vec![Peer::empty(3)]); assert_eq!(new_region_routes[0].leader_peer.as_ref().unwrap().id, 2); @@ -319,13 +319,13 @@ mod tests { region: Region::new_test(RegionId::new(table_id, 1)), leader_peer: Some(Peer::empty(1)), follower_peers: vec![Peer::empty(5), Peer::empty(3)], - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), leader_down_since: Some(current_time_millis()), }, RegionRoute { region: Region::new_test(RegionId::new(table_id, 2)), leader_peer: Some(Peer::empty(4)), - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), ..Default::default() }, ]; @@ -382,7 +382,7 @@ mod tests { region: Region::new_test(RegionId::new(1024, 1)), leader_peer: Some(leader_peer), follower_peers: vec![Peer::empty(2), Peer::empty(3)], - leader_status: None, + leader_state: None, leader_down_since: None, }]; @@ -406,7 +406,7 @@ mod tests { region: Region::new_test(RegionId::new(1024, 1)), leader_peer: Some(candidate_peer), follower_peers: vec![Peer::empty(2), Peer::empty(3)], - leader_status: None, + leader_state: None, leader_down_since: None, }]; @@ -430,7 +430,7 @@ mod tests { region: Region::new_test(RegionId::new(1024, 1)), leader_peer: Some(candidate_peer), follower_peers: vec![Peer::empty(2), Peer::empty(3)], - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), leader_down_since: None, }]; @@ -455,7 +455,7 @@ mod tests { let region_routes = vec![RegionRoute { region: Region::new_test(RegionId::new(table_id, 1)), leader_peer: Some(Peer::empty(1)), - leader_status: Some(RegionStatus::Downgraded), + leader_state: Some(LeaderState::Downgrading), ..Default::default() }]; @@ -485,7 +485,7 @@ mod tests { assert!(ctx.volatile_ctx.table_route.is_none()); assert!(ctx.volatile_ctx.opening_region_guard.is_none()); assert_eq!(region_routes.len(), 1); - assert!(!region_routes[0].is_leader_downgraded()); + assert!(!region_routes[0].is_leader_downgrading()); assert!(region_routes[0].follower_peers.is_empty()); assert_eq!(region_routes[0].leader_peer.as_ref().unwrap().id, 2); } diff --git a/src/meta-srv/src/region/lease_keeper.rs b/src/meta-srv/src/region/lease_keeper.rs index a1065d4cbb..194f3710c8 100644 --- a/src/meta-srv/src/region/lease_keeper.rs +++ b/src/meta-srv/src/region/lease_keeper.rs @@ -62,8 +62,8 @@ fn renew_region_lease_via_region_route( // If it's a leader region on this datanode. if let Some(leader) = ®ion_route.leader_peer { if leader.id == datanode_id { - let region_role = if region_route.is_leader_downgraded() { - RegionRole::Follower + let region_role = if region_route.is_leader_downgrading() { + RegionRole::DowngradingLeader } else { RegionRole::Leader }; @@ -220,7 +220,7 @@ mod tests { use common_meta::kv_backend::memory::MemoryKvBackend; use common_meta::peer::Peer; use common_meta::region_keeper::MemoryRegionKeeper; - use common_meta::rpc::router::{Region, RegionRouteBuilder, RegionStatus}; + use common_meta::rpc::router::{LeaderState, Region, RegionRouteBuilder}; use store_api::region_engine::RegionRole; use store_api::storage::RegionId; use table::metadata::RawTableInfo; @@ -265,11 +265,11 @@ mod tests { Some((region_id, RegionRole::Follower)) ); - region_route.leader_status = Some(RegionStatus::Downgraded); + region_route.leader_state = Some(LeaderState::Downgrading); // The downgraded leader region on the datanode. assert_eq!( renew_region_lease_via_region_route(®ion_route, leader_peer_id, region_id), - Some((region_id, RegionRole::Follower)) + Some((region_id, RegionRole::DowngradingLeader)) ); } @@ -492,7 +492,7 @@ mod tests { .region(Region::new_test(region_id)) .leader_peer(Peer::empty(leader_peer_id)) .follower_peers(vec![Peer::empty(follower_peer_id)]) - .leader_status(RegionStatus::Downgraded) + .leader_state(LeaderState::Downgrading) .build() .unwrap(); diff --git a/src/meta-srv/src/test_util.rs b/src/meta-srv/src/test_util.rs index 576d1aa923..6b9ccc99a0 100644 --- a/src/meta-srv/src/test_util.rs +++ b/src/meta-srv/src/test_util.rs @@ -36,7 +36,7 @@ pub(crate) fn new_region_route(region_id: u64, peers: &[Peer], leader_node: u64) region, leader_peer, follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, } } diff --git a/src/metric-engine/src/engine.rs b/src/metric-engine/src/engine.rs index 358da1d216..42948aa6cd 100644 --- a/src/metric-engine/src/engine.rs +++ b/src/metric-engine/src/engine.rs @@ -37,7 +37,8 @@ use snafu::ResultExt; use store_api::metadata::RegionMetadataRef; use store_api::metric_engine_consts::METRIC_ENGINE_NAME; use store_api::region_engine::{ - RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetReadonlyResponse, + RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetRegionRoleStateResponse, + SettableRegionRoleState, }; use store_api::region_request::RegionRequest; use store_api::storage::{RegionId, ScanRequest}; @@ -201,14 +202,14 @@ impl RegionEngine for MetricEngine { Ok(()) } - fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<(), BoxedError> { + fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<(), BoxedError> { // ignore the region not found error for x in [ utils::to_metadata_region_id(region_id), utils::to_data_region_id(region_id), region_id, ] { - if let Err(e) = self.inner.mito.set_writable(x, writable) + if let Err(e) = self.inner.mito.set_region_role(x, role) && e.status_code() != StatusCode::RegionNotFound { return Err(e); @@ -217,11 +218,15 @@ impl RegionEngine for MetricEngine { Ok(()) } - async fn set_readonly_gracefully( + async fn set_region_role_state_gracefully( &self, region_id: RegionId, - ) -> std::result::Result { - self.inner.mito.set_readonly_gracefully(region_id).await + region_role_state: SettableRegionRoleState, + ) -> std::result::Result { + self.inner + .mito + .set_region_role_state_gracefully(region_id, region_role_state) + .await } /// Returns the physical region role. diff --git a/src/mito2/src/compaction/compactor.rs b/src/mito2/src/compaction/compactor.rs index 12b9dd5fef..d919633ba9 100644 --- a/src/mito2/src/compaction/compactor.rs +++ b/src/mito2/src/compaction/compactor.rs @@ -39,8 +39,7 @@ use crate::read::Source; use crate::region::opener::new_manifest_dir; use crate::region::options::RegionOptions; use crate::region::version::{VersionBuilder, VersionRef}; -use crate::region::ManifestContext; -use crate::region::RegionState::Writable; +use crate::region::{ManifestContext, RegionLeaderState, RegionRoleState}; use crate::schedule::scheduler::LocalScheduler; use crate::sst::file::{FileMeta, IndexType}; use crate::sst::file_purger::LocalFilePurger; @@ -129,7 +128,10 @@ pub async fn open_compaction_region( let manifest = manifest_manager.manifest(); let region_metadata = manifest.metadata.clone(); - let manifest_ctx = Arc::new(ManifestContext::new(manifest_manager, Writable)); + let manifest_ctx = Arc::new(ManifestContext::new( + manifest_manager, + RegionRoleState::Leader(RegionLeaderState::Writable), + )); let file_purger = { let purge_scheduler = Arc::new(LocalScheduler::new(mito_config.max_background_jobs)); @@ -379,7 +381,7 @@ impl Compactor for DefaultCompactor { // TODO: We might leak files if we fail to update manifest. We can add a cleanup task to remove them later. compaction_region .manifest_ctx - .update_manifest(Writable, action_list) + .update_manifest(RegionLeaderState::Writable, action_list) .await?; Ok(edit) diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index e9177d40bf..ed8cc92909 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -53,7 +53,7 @@ mod prune_test; #[cfg(test)] mod row_selector_test; #[cfg(test)] -mod set_readonly_test; +mod set_role_state_test; #[cfg(test)] mod truncate_test; @@ -77,7 +77,7 @@ use store_api::logstore::LogStore; use store_api::metadata::RegionMetadataRef; use store_api::region_engine::{ BatchResponses, RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, - SetReadonlyResponse, + SetRegionRoleStateResponse, SettableRegionRoleState, }; use store_api::region_request::{AffectedRows, RegionOpenRequest, RegionRequest}; use store_api::storage::{RegionId, ScanRequest}; @@ -436,22 +436,27 @@ impl EngineInner { Ok(scan_region) } - /// Set writable mode for a region. - fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<()> { + /// Converts the [`RegionRole`]. + fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<()> { let region = self .workers .get_region(region_id) .context(RegionNotFoundSnafu { region_id })?; - region.set_writable(writable); + region.set_role(role); Ok(()) } /// Sets read-only for a region and ensures no more writes in the region after it returns. - async fn set_readonly_gracefully(&self, region_id: RegionId) -> Result { + async fn set_region_role_state_gracefully( + &self, + region_id: RegionId, + region_role_state: SettableRegionRoleState, + ) -> Result { // Notes: It acquires the mutable ownership to ensure no other threads, // Therefore, we submit it to the worker. - let (request, receiver) = WorkerRequest::new_set_readonly_gracefully(region_id); + let (request, receiver) = + WorkerRequest::new_set_readonly_gracefully(region_id, region_role_state); self.workers.submit_to_worker(region_id, request).await?; receiver.await.context(RecvSnafu) @@ -459,7 +464,7 @@ impl EngineInner { fn role(&self, region_id: RegionId) -> Option { self.workers.get_region(region_id).map(|region| { - if region.is_readonly() { + if region.is_follower() { RegionRole::Follower } else { RegionRole::Leader @@ -547,22 +552,23 @@ impl RegionEngine for MitoEngine { self.get_region_statistic(region_id) } - fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<(), BoxedError> { + fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<(), BoxedError> { self.inner - .set_writable(region_id, writable) + .set_region_role(region_id, role) .map_err(BoxedError::new) } - async fn set_readonly_gracefully( + async fn set_region_role_state_gracefully( &self, region_id: RegionId, - ) -> Result { + region_role_state: SettableRegionRoleState, + ) -> Result { let _timer = HANDLE_REQUEST_ELAPSED - .with_label_values(&["set_readonly_gracefully"]) + .with_label_values(&["set_region_role_state_gracefully"]) .start_timer(); self.inner - .set_readonly_gracefully(region_id) + .set_region_role_state_gracefully(region_id, region_role_state) .await .map_err(BoxedError::new) } diff --git a/src/mito2/src/engine/alter_test.rs b/src/mito2/src/engine/alter_test.rs index b48dc2ccfb..2e75bf19fa 100644 --- a/src/mito2/src/engine/alter_test.rs +++ b/src/mito2/src/engine/alter_test.rs @@ -24,7 +24,7 @@ use common_recordbatch::RecordBatches; use datatypes::prelude::ConcreteDataType; use datatypes::schema::ColumnSchema; use store_api::metadata::ColumnMetadata; -use store_api::region_engine::RegionEngine; +use store_api::region_engine::{RegionEngine, RegionRole}; use store_api::region_request::{ AddColumn, AddColumnLocation, AlterKind, RegionAlterRequest, RegionOpenRequest, RegionRequest, }; @@ -213,8 +213,10 @@ async fn test_put_after_alter() { ) .await .unwrap(); - // Set writable. - engine.set_writable(region_id, true).unwrap(); + // Convert region to leader. + engine + .set_region_role(region_id, RegionRole::Leader) + .unwrap(); // Put with old schema. let rows = Rows { diff --git a/src/mito2/src/engine/catchup_test.rs b/src/mito2/src/engine/catchup_test.rs index 5f4dd3b15a..a9de0d6008 100644 --- a/src/mito2/src/engine/catchup_test.rs +++ b/src/mito2/src/engine/catchup_test.rs @@ -22,7 +22,7 @@ use common_recordbatch::RecordBatches; use common_wal::options::{KafkaWalOptions, WalOptions, WAL_OPTIONS_KEY}; use rstest::rstest; use rstest_reuse::{self, apply}; -use store_api::region_engine::{RegionEngine, SetReadonlyResponse}; +use store_api::region_engine::{RegionEngine, RegionRole, SetRegionRoleStateResponse}; use store_api::region_request::{RegionCatchupRequest, RegionOpenRequest, RegionRequest}; use store_api::storage::{RegionId, ScanRequest}; @@ -34,8 +34,8 @@ use crate::test_util::{ }; use crate::wal::EntryId; -fn get_last_entry_id(resp: SetReadonlyResponse) -> Option { - if let SetReadonlyResponse::Success { last_entry_id } = resp { +fn get_last_entry_id(resp: SetRegionRoleStateResponse) -> Option { + if let SetRegionRoleStateResponse::Success { last_entry_id } = resp { last_entry_id } else { unreachable!(); @@ -45,6 +45,8 @@ fn get_last_entry_id(resp: SetReadonlyResponse) -> Option { #[apply(single_kafka_log_store_factory)] async fn test_catchup_with_last_entry_id(factory: Option) { + use store_api::region_engine::SettableRegionRoleState; + common_telemetry::init_default_ut_logging(); let Some(factory) = factory else { return; @@ -102,7 +104,7 @@ async fn test_catchup_with_last_entry_id(factory: Option) { put_rows(&leader_engine, region_id, rows).await; let resp = leader_engine - .set_readonly_gracefully(region_id) + .set_region_role_state_gracefully(region_id, SettableRegionRoleState::Follower) .await .unwrap(); @@ -159,6 +161,8 @@ async fn test_catchup_with_last_entry_id(factory: Option) { #[apply(single_kafka_log_store_factory)] async fn test_catchup_with_incorrect_last_entry_id(factory: Option) { + use store_api::region_engine::SettableRegionRoleState; + common_telemetry::init_default_ut_logging(); let Some(factory) = factory else { return; @@ -217,7 +221,7 @@ async fn test_catchup_with_incorrect_last_entry_id(factory: Option source.status_code(), CompatReader { .. } => StatusCode::Unexpected, InvalidRegionRequest { source, .. } => source.status_code(), - RegionState { .. } => StatusCode::RegionNotReady, + RegionLeaderState { .. } => StatusCode::RegionNotReady, + &FlushableRegionState { .. } => StatusCode::RegionNotReady, JsonOptions { .. } => StatusCode::InvalidArguments, EmptyRegionDir { .. } | EmptyManifestDir { .. } => StatusCode::RegionNotFound, ArrowReader { .. } => StatusCode::StorageUnavailable, diff --git a/src/mito2/src/flush.rs b/src/mito2/src/flush.rs index 05561b6080..9606e92d04 100644 --- a/src/mito2/src/flush.rs +++ b/src/mito2/src/flush.rs @@ -36,7 +36,7 @@ use crate::metrics::{FLUSH_BYTES_TOTAL, FLUSH_ELAPSED, FLUSH_ERRORS_TOTAL, FLUSH use crate::read::Source; use crate::region::options::IndexOptions; use crate::region::version::{VersionControlData, VersionControlRef}; -use crate::region::{ManifestContextRef, RegionState}; +use crate::region::{ManifestContextRef, RegionLeaderState}; use crate::request::{ BackgroundNotify, FlushFailed, FlushFinished, OptionOutputTx, OutputTx, SenderDdlRequest, SenderWriteRequest, WorkerRequest, @@ -195,6 +195,8 @@ pub enum FlushReason { Alter, /// Flush periodically. Periodically, + /// Flush memtable during downgrading state. + Downgrading, } impl FlushReason { @@ -407,11 +409,23 @@ impl RegionFlushTask { info!("Applying {edit:?} to region {}", self.region_id); let action_list = RegionMetaActionList::with_action(RegionMetaAction::Edit(edit.clone())); + + let expected_state = if matches!(self.reason, FlushReason::Downgrading) { + RegionLeaderState::Downgrading + } else { + RegionLeaderState::Writable + }; // We will leak files if the manifest update fails, but we ignore them for simplicity. We can // add a cleanup job to remove them later. - self.manifest_ctx - .update_manifest(RegionState::Writable, action_list) + let version = self + .manifest_ctx + .update_manifest(expected_state, action_list) .await?; + info!( + "Successfully update manifest version to {version}, region: {}, reason: {}", + self.region_id, + self.reason.as_str() + ); Ok(edit) } diff --git a/src/mito2/src/region.rs b/src/mito2/src/region.rs index 8fc9095ae5..b05daf3da0 100644 --- a/src/mito2/src/region.rs +++ b/src/mito2/src/region.rs @@ -27,12 +27,16 @@ use common_telemetry::{error, info, warn}; use crossbeam_utils::atomic::AtomicCell; use snafu::{ensure, OptionExt}; use store_api::logstore::provider::Provider; +use store_api::manifest::ManifestVersion; use store_api::metadata::RegionMetadataRef; -use store_api::region_engine::RegionStatistic; +use store_api::region_engine::{RegionRole, RegionStatistic, SettableRegionRoleState}; use store_api::storage::RegionId; use crate::access_layer::AccessLayerRef; -use crate::error::{RegionNotFoundSnafu, RegionStateSnafu, RegionTruncatedSnafu, Result}; +use crate::error::{ + FlushableRegionStateSnafu, RegionLeaderStateSnafu, RegionNotFoundSnafu, RegionTruncatedSnafu, + Result, +}; use crate::manifest::action::{RegionMetaAction, RegionMetaActionList}; use crate::manifest::manager::RegionManifestManager; use crate::memtable::MemtableBuilderRef; @@ -59,11 +63,8 @@ impl RegionUsage { } } -/// State of the region. #[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub enum RegionState { - /// The region is opened but is still read-only. - ReadOnly, +pub enum RegionLeaderState { /// The region is opened and is writable. Writable, /// The region is altering. @@ -74,6 +75,14 @@ pub enum RegionState { Truncating, /// The region is handling a region edit. Editing, + /// The region is stepping down. + Downgrading, +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum RegionRoleState { + Leader(RegionLeaderState), + Follower, } /// Metadata and runtime status of a region. @@ -172,83 +181,91 @@ impl MitoRegion { /// Returns whether the region is writable. pub(crate) fn is_writable(&self) -> bool { - self.manifest_ctx.state.load() == RegionState::Writable + self.manifest_ctx.state.load() == RegionRoleState::Leader(RegionLeaderState::Writable) + } + + /// Returns whether the region is flushable. + pub(crate) fn is_flushable(&self) -> bool { + matches!( + self.manifest_ctx.state.load(), + RegionRoleState::Leader(RegionLeaderState::Writable) + | RegionRoleState::Leader(RegionLeaderState::Downgrading) + ) + } + + /// Returns whether the region is downgrading. + pub(crate) fn is_downgrading(&self) -> bool { + matches!( + self.manifest_ctx.state.load(), + RegionRoleState::Leader(RegionLeaderState::Downgrading) + ) } /// Returns whether the region is readonly. - pub(crate) fn is_readonly(&self) -> bool { - self.manifest_ctx.state.load() == RegionState::ReadOnly + pub(crate) fn is_follower(&self) -> bool { + self.manifest_ctx.state.load() == RegionRoleState::Follower } /// Returns the state of the region. - pub(crate) fn state(&self) -> RegionState { + pub(crate) fn state(&self) -> RegionRoleState { self.manifest_ctx.state.load() } - /// Sets the writable state. - pub(crate) fn set_writable(&self, writable: bool) { - if writable { - // Only sets the region to writable if it is read only. - // This prevents others updating the manifest. - match self - .manifest_ctx - .state - .compare_exchange(RegionState::ReadOnly, RegionState::Writable) - { - Ok(state) => info!( - "Set region {} to writable, previous state: {:?}", - self.region_id, state - ), - Err(state) => { - if state != RegionState::Writable { - warn!( - "Failed to set region {} to writable, current state: {:?}", - self.region_id, state - ) - } - } - } - } else { - self.manifest_ctx.state.store(RegionState::ReadOnly); - } + /// Sets the region role state. + pub(crate) fn set_role(&self, next_role: RegionRole) { + self.manifest_ctx.set_role(next_role, self.region_id); } /// Sets the altering state. /// You should call this method in the worker loop. pub(crate) fn set_altering(&self) -> Result<()> { - self.compare_exchange_state(RegionState::Writable, RegionState::Altering) + self.compare_exchange_state( + RegionLeaderState::Writable, + RegionRoleState::Leader(RegionLeaderState::Altering), + ) } /// Sets the dropping state. /// You should call this method in the worker loop. pub(crate) fn set_dropping(&self) -> Result<()> { - self.compare_exchange_state(RegionState::Writable, RegionState::Dropping) + self.compare_exchange_state( + RegionLeaderState::Writable, + RegionRoleState::Leader(RegionLeaderState::Dropping), + ) } /// Sets the truncating state. /// You should call this method in the worker loop. pub(crate) fn set_truncating(&self) -> Result<()> { - self.compare_exchange_state(RegionState::Writable, RegionState::Truncating) + self.compare_exchange_state( + RegionLeaderState::Writable, + RegionRoleState::Leader(RegionLeaderState::Truncating), + ) } /// Sets the editing state. /// You should call this method in the worker loop. pub(crate) fn set_editing(&self) -> Result<()> { - self.compare_exchange_state(RegionState::Writable, RegionState::Editing) + self.compare_exchange_state( + RegionLeaderState::Writable, + RegionRoleState::Leader(RegionLeaderState::Editing), + ) } /// Sets the region to readonly gracefully. This acquires the manifest write lock. - pub(crate) async fn set_readonly_gracefully(&self) { + pub(crate) async fn set_role_state_gracefully(&self, state: SettableRegionRoleState) { let _manager = self.manifest_ctx.manifest_manager.write().await; // We acquires the write lock of the manifest manager to ensure that no one is updating the manifest. // Then we change the state. - self.set_writable(false); + self.set_role(state.into()); } - /// Switches the region state to `RegionState::Writable` if the current state is `expect`. + /// Switches the region state to `RegionRoleState::Leader(RegionLeaderState::Writable)` if the current state is `expect`. /// Otherwise, logs an error. - pub(crate) fn switch_state_to_writable(&self, expect: RegionState) { - if let Err(e) = self.compare_exchange_state(expect, RegionState::Writable) { + pub(crate) fn switch_state_to_writable(&self, expect: RegionLeaderState) { + if let Err(e) = self + .compare_exchange_state(expect, RegionRoleState::Leader(RegionLeaderState::Writable)) + { error!(e; "failed to switch region state to writable, expect state is {:?}", expect); } } @@ -280,12 +297,16 @@ impl MitoRegion { /// Sets the state of the region to given state if the current state equals to /// the expected. - fn compare_exchange_state(&self, expect: RegionState, state: RegionState) -> Result<()> { + fn compare_exchange_state( + &self, + expect: RegionLeaderState, + state: RegionRoleState, + ) -> Result<()> { self.manifest_ctx .state - .compare_exchange(expect, state) + .compare_exchange(RegionRoleState::Leader(expect), state) .map_err(|actual| { - RegionStateSnafu { + RegionLeaderStateSnafu { region_id: self.region_id, state: actual, expect, @@ -303,17 +324,25 @@ pub(crate) struct ManifestContext { manifest_manager: tokio::sync::RwLock, /// The state of the region. The region checks the state before updating /// manifest. - state: AtomicCell, + state: AtomicCell, } impl ManifestContext { - pub(crate) fn new(manager: RegionManifestManager, state: RegionState) -> Self { + pub(crate) fn new(manager: RegionManifestManager, state: RegionRoleState) -> Self { ManifestContext { manifest_manager: tokio::sync::RwLock::new(manager), state: AtomicCell::new(state), } } + pub(crate) async fn manifest_version(&self) -> ManifestVersion { + self.manifest_manager + .read() + .await + .manifest() + .manifest_version + } + pub(crate) async fn has_update(&self) -> Result { self.manifest_manager.read().await.has_update().await } @@ -321,9 +350,9 @@ impl ManifestContext { /// Updates the manifest if current state is `expect_state`. pub(crate) async fn update_manifest( &self, - expect_state: RegionState, + expect_state: RegionLeaderState, action_list: RegionMetaActionList, - ) -> Result<()> { + ) -> Result { // Acquires the write lock of the manifest manager. let mut manager = self.manifest_manager.write().await; // Gets current manifest. @@ -332,8 +361,8 @@ impl ManifestContext { // after `set_readonly_gracefully()` is called. let current_state = self.state.load(); ensure!( - current_state == expect_state, - RegionStateSnafu { + current_state == RegionRoleState::Leader(expect_state), + RegionLeaderStateSnafu { region_id: manifest.metadata.region_id, state: current_state, expect: expect_state, @@ -376,18 +405,92 @@ impl ManifestContext { } // Now we can update the manifest. - manager.update(action_list).await.inspect_err( + let version = manager.update(action_list).await.inspect_err( |e| error!(e; "Failed to update manifest, region_id: {}", manifest.metadata.region_id), )?; - if self.state.load() == RegionState::ReadOnly { + if self.state.load() == RegionRoleState::Follower { warn!( - "Region {} becomes read-only while updating manifest which may cause inconsistency", + "Region {} becomes follower while updating manifest which may cause inconsistency, manifest version: {version}", manifest.metadata.region_id ); } - Ok(()) + Ok(version) + } + + /// Sets the [`RegionRole`]. + /// + /// ``` + /// +------------------------------------------+ + /// | +-----------------+ | + /// | | | | + /// +---+------+ +-------+-----+ +--v-v---+ + /// | Follower | | Downgrading | | Leader | + /// +---^-^----+ +-----+-^-----+ +--+-+---+ + /// | | | | | | + /// | +------------------+ +-----------------+ | + /// +------------------------------------------+ + /// + /// Transition: + /// - Follower -> Leader + /// - Downgrading Leader -> Leader + /// - Leader -> Follower + /// - Downgrading Leader -> Follower + /// - Leader -> Downgrading Leader + /// + /// ``` + pub(crate) fn set_role(&self, next_role: RegionRole, region_id: RegionId) { + match next_role { + RegionRole::Follower => { + self.state.store(RegionRoleState::Follower); + } + RegionRole::Leader => { + match self.state.fetch_update(|state| { + if matches!( + state, + RegionRoleState::Follower + | RegionRoleState::Leader(RegionLeaderState::Downgrading) + ) { + Some(RegionRoleState::Leader(RegionLeaderState::Writable)) + } else { + None + } + }) { + Ok(state) => info!( + "Convert region {} to leader, previous role state: {:?}", + region_id, state + ), + Err(state) => { + if state != RegionRoleState::Leader(RegionLeaderState::Writable) { + warn!( + "Failed to convert region {} to leader, current role state: {:?}", + region_id, state + ) + } + } + } + } + RegionRole::DowngradingLeader => { + match self.state.compare_exchange( + RegionRoleState::Leader(RegionLeaderState::Writable), + RegionRoleState::Leader(RegionLeaderState::Downgrading), + ) { + Ok(state) => info!( + "Convert region {} to downgrading region, previous role state: {:?}", + region_id, state + ), + Err(state) => { + if state != RegionRoleState::Leader(RegionLeaderState::Downgrading) { + warn!( + "Failed to convert region {} to downgrading leader, current role state: {:?}", + region_id, state + ) + } + } + } + } + } } } @@ -434,10 +537,10 @@ impl RegionMap { .context(RegionNotFoundSnafu { region_id })?; ensure!( region.is_writable(), - RegionStateSnafu { + RegionLeaderStateSnafu { region_id, state: region.state(), - expect: RegionState::Writable, + expect: RegionLeaderState::Writable, } ); Ok(region) @@ -460,6 +563,40 @@ impl RegionMap { } } + /// Gets flushable region by region id. + /// + /// Returns error if the region does not exist or is not operable. + fn flushable_region(&self, region_id: RegionId) -> Result { + let region = self + .get_region(region_id) + .context(RegionNotFoundSnafu { region_id })?; + ensure!( + region.is_flushable(), + FlushableRegionStateSnafu { + region_id, + state: region.state(), + } + ); + Ok(region) + } + + /// Gets flushable region by region id. + /// + /// Calls the callback if the region does not exist or is not operable. + pub(crate) fn flushable_region_or( + &self, + region_id: RegionId, + cb: &mut F, + ) -> Option { + match self.flushable_region(region_id) { + Ok(region) => Some(region), + Err(e) => { + cb.on_failure(e); + None + } + } + } + /// Remove region by id. pub(crate) fn remove_region(&self, region_id: RegionId) { let mut regions = self.regions.write().unwrap(); @@ -548,12 +685,70 @@ impl ManifestStats { #[cfg(test)] mod tests { - use crossbeam_utils::atomic::AtomicCell; + use std::sync::Arc; - use crate::region::RegionState; + use crossbeam_utils::atomic::AtomicCell; + use store_api::region_engine::RegionRole; + use store_api::storage::RegionId; + + use crate::region::{RegionLeaderState, RegionRoleState}; + use crate::test_util::scheduler_util::SchedulerEnv; + use crate::test_util::version_util::VersionControlBuilder; #[test] fn test_region_state_lock_free() { - assert!(AtomicCell::::is_lock_free()); + assert!(AtomicCell::::is_lock_free()); + } + + #[tokio::test] + async fn test_set_region_state() { + let env = SchedulerEnv::new().await; + let builder = VersionControlBuilder::new(); + let version_control = Arc::new(builder.build()); + let manifest_ctx = env + .mock_manifest_context(version_control.current().version.metadata.clone()) + .await; + + let region_id = RegionId::new(1024, 0); + // Leader -> Follower + manifest_ctx.set_role(RegionRole::Follower, region_id); + assert_eq!(manifest_ctx.state.load(), RegionRoleState::Follower); + + // Follower -> Leader + manifest_ctx.set_role(RegionRole::Leader, region_id); + assert_eq!( + manifest_ctx.state.load(), + RegionRoleState::Leader(RegionLeaderState::Writable) + ); + + // Leader -> Downgrading Leader + manifest_ctx.set_role(RegionRole::DowngradingLeader, region_id); + assert_eq!( + manifest_ctx.state.load(), + RegionRoleState::Leader(RegionLeaderState::Downgrading) + ); + + // Downgrading Leader -> Follower + manifest_ctx.set_role(RegionRole::Follower, region_id); + assert_eq!(manifest_ctx.state.load(), RegionRoleState::Follower); + + // Can't downgrade from follower (Follower -> Downgrading Leader) + manifest_ctx.set_role(RegionRole::DowngradingLeader, region_id); + assert_eq!(manifest_ctx.state.load(), RegionRoleState::Follower); + + // Set region role too Downgrading Leader + manifest_ctx.set_role(RegionRole::Leader, region_id); + manifest_ctx.set_role(RegionRole::DowngradingLeader, region_id); + assert_eq!( + manifest_ctx.state.load(), + RegionRoleState::Leader(RegionLeaderState::Downgrading) + ); + + // Downgrading Leader -> Leader + manifest_ctx.set_role(RegionRole::Leader, region_id); + assert_eq!( + manifest_ctx.state.load(), + RegionRoleState::Leader(RegionLeaderState::Writable) + ); } } diff --git a/src/mito2/src/region/opener.rs b/src/mito2/src/region/opener.rs index 64272a183b..b2a76490cc 100644 --- a/src/mito2/src/region/opener.rs +++ b/src/mito2/src/region/opener.rs @@ -28,6 +28,7 @@ use snafu::{ensure, OptionExt}; use store_api::logstore::provider::Provider; use store_api::logstore::LogStore; use store_api::metadata::{ColumnMetadata, RegionMetadata}; +use store_api::region_engine::RegionRole; use store_api::storage::{ColumnId, RegionId}; use crate::access_layer::AccessLayer; @@ -42,7 +43,9 @@ use crate::memtable::time_partition::TimePartitions; use crate::memtable::MemtableBuilderProvider; use crate::region::options::RegionOptions; use crate::region::version::{VersionBuilder, VersionControl, VersionControlRef}; -use crate::region::{ManifestContext, ManifestStats, MitoRegion, RegionState}; +use crate::region::{ + ManifestContext, ManifestStats, MitoRegion, RegionLeaderState, RegionRoleState, +}; use crate::region_write_ctx::RegionWriteCtx; use crate::request::OptionOutputTx; use crate::schedule::scheduler::SchedulerRef; @@ -169,8 +172,8 @@ impl RegionOpener { &expect.column_metadatas, &expect.primary_key, )?; - // To keep consistence with Create behavior, set the opened Region writable. - region.set_writable(true); + // To keep consistence with Create behavior, set the opened Region to RegionRole::Leader. + region.set_role(RegionRole::Leader); return Ok(region); } Ok(None) => { @@ -235,7 +238,7 @@ impl RegionOpener { // Region is writable after it is created. manifest_ctx: Arc::new(ManifestContext::new( manifest_manager, - RegionState::Writable, + RegionRoleState::Leader(RegionLeaderState::Writable), )), file_purger: Arc::new(LocalFilePurger::new( self.purge_scheduler, @@ -362,9 +365,10 @@ impl RegionOpener { let version_control = Arc::new(VersionControl::new(version)); if !self.skip_wal_replay { info!( - "Start replaying memtable at flushed_entry_id + 1 {} for region {}", + "Start replaying memtable at flushed_entry_id + 1: {} for region {}, manifest version: {}", flushed_entry_id + 1, - region_id + region_id, + manifest.manifest_version ); replay_memtable( &provider, @@ -377,7 +381,10 @@ impl RegionOpener { ) .await?; } else { - info!("Skip the WAL replay for region: {}", region_id); + info!( + "Skip the WAL replay for region: {}, manifest version: {}", + region_id, manifest.manifest_version + ); } let now = self.time_provider.current_time_millis(); @@ -388,7 +395,7 @@ impl RegionOpener { // Region is always opened in read only mode. manifest_ctx: Arc::new(ManifestContext::new( manifest_manager, - RegionState::ReadOnly, + RegionRoleState::Follower, )), file_purger, provider: provider.clone(), diff --git a/src/mito2/src/request.rs b/src/mito2/src/request.rs index d88bc994e9..1e4c6b8dc9 100644 --- a/src/mito2/src/request.rs +++ b/src/mito2/src/request.rs @@ -31,7 +31,7 @@ use prost::Message; use smallvec::SmallVec; use snafu::{ensure, OptionExt, ResultExt}; use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataRef}; -use store_api::region_engine::SetReadonlyResponse; +use store_api::region_engine::{SetRegionRoleStateResponse, SettableRegionRoleState}; use store_api::region_request::{ AffectedRows, RegionAlterRequest, RegionCatchupRequest, RegionCloseRequest, RegionCompactRequest, RegionCreateRequest, RegionDropRequest, RegionFlushRequest, @@ -483,11 +483,13 @@ pub(crate) enum WorkerRequest { }, /// The internal commands. - SetReadonlyGracefully { + SetRegionRoleStateGracefully { /// Id of the region to send. region_id: RegionId, + /// The [SettableRegionRoleState]. + region_role_state: SettableRegionRoleState, /// The sender of [SetReadonlyResponse]. - sender: Sender, + sender: Sender, }, /// Notify a worker to stop. @@ -587,11 +589,16 @@ impl WorkerRequest { pub(crate) fn new_set_readonly_gracefully( region_id: RegionId, - ) -> (WorkerRequest, Receiver) { + region_role_state: SettableRegionRoleState, + ) -> (WorkerRequest, Receiver) { let (sender, receiver) = oneshot::channel(); ( - WorkerRequest::SetReadonlyGracefully { region_id, sender }, + WorkerRequest::SetRegionRoleStateGracefully { + region_id, + region_role_state, + sender, + }, receiver, ) } diff --git a/src/mito2/src/test_util.rs b/src/mito2/src/test_util.rs index cd449e53fa..0bd85747c0 100644 --- a/src/mito2/src/test_util.rs +++ b/src/mito2/src/test_util.rs @@ -52,7 +52,7 @@ use rskafka::client::{Client, ClientBuilder}; use rskafka::record::Record; use rstest_reuse::template; use store_api::metadata::{ColumnMetadata, RegionMetadataRef}; -use store_api::region_engine::RegionEngine; +use store_api::region_engine::{RegionEngine, RegionRole}; use store_api::region_request::{ RegionCloseRequest, RegionCreateRequest, RegionDeleteRequest, RegionFlushRequest, RegionOpenRequest, RegionPutRequest, RegionRequest, @@ -1114,6 +1114,8 @@ pub async fn reopen_region( .unwrap(); if writable { - engine.set_writable(region_id, true).unwrap(); + engine + .set_region_role(region_id, RegionRole::Leader) + .unwrap(); } } diff --git a/src/mito2/src/test_util/scheduler_util.rs b/src/mito2/src/test_util/scheduler_util.rs index c1b85279de..ba777b157f 100644 --- a/src/mito2/src/test_util/scheduler_util.rs +++ b/src/mito2/src/test_util/scheduler_util.rs @@ -31,7 +31,7 @@ use crate::config::MitoConfig; use crate::error::Result; use crate::flush::FlushScheduler; use crate::manifest::manager::{RegionManifestManager, RegionManifestOptions}; -use crate::region::{ManifestContext, ManifestContextRef, RegionState}; +use crate::region::{ManifestContext, ManifestContextRef, RegionLeaderState, RegionRoleState}; use crate::request::WorkerRequest; use crate::schedule::scheduler::{Job, LocalScheduler, Scheduler, SchedulerRef}; use crate::sst::index::intermediate::IntermediateManager; @@ -124,7 +124,7 @@ impl SchedulerEnv { ) .await .unwrap(), - RegionState::Writable, + RegionRoleState::Leader(RegionLeaderState::Writable), )) } diff --git a/src/mito2/src/worker.rs b/src/mito2/src/worker.rs index c2fbc80982..e790ed08c1 100644 --- a/src/mito2/src/worker.rs +++ b/src/mito2/src/worker.rs @@ -39,7 +39,7 @@ use prometheus::IntGauge; use rand::{thread_rng, Rng}; use snafu::{ensure, ResultExt}; use store_api::logstore::LogStore; -use store_api::region_engine::SetReadonlyResponse; +use store_api::region_engine::{SetRegionRoleStateResponse, SettableRegionRoleState}; use store_api::storage::RegionId; use tokio::sync::mpsc::{Receiver, Sender}; use tokio::sync::{mpsc, oneshot, watch, Mutex}; @@ -734,8 +734,13 @@ impl RegionWorkerLoop { // For background notify, we handle it directly. self.handle_background_notify(region_id, notify).await; } - WorkerRequest::SetReadonlyGracefully { region_id, sender } => { - self.set_readonly_gracefully(region_id, sender).await; + WorkerRequest::SetRegionRoleStateGracefully { + region_id, + region_role_state, + sender, + } => { + self.set_role_state_gracefully(region_id, region_role_state, sender) + .await; } WorkerRequest::EditRegion(request) => { self.handle_region_edit(request).await; @@ -834,22 +839,23 @@ impl RegionWorkerLoop { } } - /// Handles `set_readonly_gracefully`. - async fn set_readonly_gracefully( + /// Handles `set_region_role_gracefully`. + async fn set_role_state_gracefully( &mut self, region_id: RegionId, - sender: oneshot::Sender, + region_role_state: SettableRegionRoleState, + sender: oneshot::Sender, ) { if let Some(region) = self.regions.get_region(region_id) { // We need to do this in background as we need the manifest lock. common_runtime::spawn_global(async move { - region.set_readonly_gracefully().await; + region.set_role_state_gracefully(region_role_state).await; let last_entry_id = region.version_control.current().last_entry_id; - let _ = sender.send(SetReadonlyResponse::success(Some(last_entry_id))); + let _ = sender.send(SetRegionRoleStateResponse::success(Some(last_entry_id))); }); } else { - let _ = sender.send(SetReadonlyResponse::NotFound); + let _ = sender.send(SetRegionRoleStateResponse::NotFound); } } } diff --git a/src/mito2/src/worker/handle_catchup.rs b/src/mito2/src/worker/handle_catchup.rs index 505c994d36..cacd563ed7 100644 --- a/src/mito2/src/worker/handle_catchup.rs +++ b/src/mito2/src/worker/handle_catchup.rs @@ -20,6 +20,7 @@ use common_telemetry::info; use common_telemetry::tracing::warn; use snafu::ensure; use store_api::logstore::LogStore; +use store_api::region_engine::RegionRole; use store_api::region_request::{AffectedRows, RegionCatchupRequest}; use store_api::storage::RegionId; use tokio::time::Instant; @@ -47,7 +48,8 @@ impl RegionWorkerLoop { // Utilizes the short circuit evaluation. let region = if !is_mutable_empty || region.manifest_ctx.has_update().await? { - info!("Reopening the region: {region_id}, empty mutable: {is_mutable_empty}"); + let manifest_version = region.manifest_ctx.manifest_version().await; + info!("Reopening the region: {region_id}, empty mutable: {is_mutable_empty}, manifest version: {manifest_version}"); let reopened_region = Arc::new( RegionOpener::new( region_id, @@ -112,7 +114,7 @@ impl RegionWorkerLoop { } if request.set_writable { - region.set_writable(true); + region.set_role(RegionRole::Leader); } Ok(0) diff --git a/src/mito2/src/worker/handle_drop.rs b/src/mito2/src/worker/handle_drop.rs index 51b42acb40..a569f22360 100644 --- a/src/mito2/src/worker/handle_drop.rs +++ b/src/mito2/src/worker/handle_drop.rs @@ -28,7 +28,7 @@ use store_api::storage::RegionId; use tokio::time::sleep; use crate::error::{OpenDalSnafu, Result}; -use crate::region::{RegionMapRef, RegionState}; +use crate::region::{RegionLeaderState, RegionMapRef}; use crate::worker::{RegionWorkerLoop, DROPPING_MARKER_FILE}; const GC_TASK_INTERVAL_SEC: u64 = 5 * 60; // 5 minutes @@ -62,7 +62,7 @@ where // Sets the state back to writable. It's possible that the marker file has been written. // We set the state back to writable so we can retry the drop operation. - region.switch_state_to_writable(RegionState::Dropping); + region.switch_state_to_writable(RegionLeaderState::Dropping); })?; region.stop().await; diff --git a/src/mito2/src/worker/handle_flush.rs b/src/mito2/src/worker/handle_flush.rs index 14a70225bb..b2bc5fd2e8 100644 --- a/src/mito2/src/worker/handle_flush.rs +++ b/src/mito2/src/worker/handle_flush.rs @@ -36,16 +36,18 @@ impl RegionWorkerLoop { request: RegionFlushRequest, mut sender: OptionOutputTx, ) { - let Some(region) = self.regions.writable_region_or(region_id, &mut sender) else { + let Some(region) = self.regions.flushable_region_or(region_id, &mut sender) else { return; }; - let mut task = self.new_flush_task( - ®ion, - FlushReason::Manual, - request.row_group_size, - self.config.clone(), - ); + let reason = if region.is_downgrading() { + FlushReason::Downgrading + } else { + FlushReason::Manual + }; + + let mut task = + self.new_flush_task(®ion, reason, request.row_group_size, self.config.clone()); task.push_sender(sender); if let Err(e) = self.flush_scheduler diff --git a/src/mito2/src/worker/handle_manifest.rs b/src/mito2/src/worker/handle_manifest.rs index de5f4e563d..e97b30afec 100644 --- a/src/mito2/src/worker/handle_manifest.rs +++ b/src/mito2/src/worker/handle_manifest.rs @@ -27,7 +27,7 @@ use crate::error::{RegionBusySnafu, RegionNotFoundSnafu, Result}; use crate::manifest::action::{ RegionChange, RegionEdit, RegionMetaAction, RegionMetaActionList, RegionTruncate, }; -use crate::region::{MitoRegionRef, RegionState}; +use crate::region::{MitoRegionRef, RegionLeaderState, RegionRoleState}; use crate::request::{ BackgroundNotify, OptionOutputTx, RegionChangeResult, RegionEditRequest, RegionEditResult, TruncateResult, WorkerRequest, @@ -84,7 +84,7 @@ impl RegionWorkerLoop { }; if !region.is_writable() { - if region.state() == RegionState::Editing { + if region.state() == RegionRoleState::Leader(RegionLeaderState::Editing) { self.region_edit_queues .entry(region_id) .or_insert_with(|| RegionEditQueue::new(region_id)) @@ -159,7 +159,7 @@ impl RegionWorkerLoop { } // Sets the region as writable. - region.switch_state_to_writable(RegionState::Editing); + region.switch_state_to_writable(RegionLeaderState::Editing); let _ = edit_result.sender.send(edit_result.result); @@ -199,8 +199,9 @@ impl RegionWorkerLoop { RegionMetaActionList::with_action(RegionMetaAction::Truncate(truncate.clone())); let result = manifest_ctx - .update_manifest(RegionState::Truncating, action_list) - .await; + .update_manifest(RegionLeaderState::Truncating, action_list) + .await + .map(|_| ()); // Sends the result back to the request sender. let truncate_result = TruncateResult { @@ -241,8 +242,9 @@ impl RegionWorkerLoop { let result = region .manifest_ctx - .update_manifest(RegionState::Altering, action_list) - .await; + .update_manifest(RegionLeaderState::Altering, action_list) + .await + .map(|_| ()); let notify = WorkerRequest::Background { region_id: region.region_id, notify: BackgroundNotify::RegionChange(RegionChangeResult { @@ -291,7 +293,7 @@ impl RegionWorkerLoop { } // Sets the region as writable. - region.switch_state_to_writable(RegionState::Altering); + region.switch_state_to_writable(RegionLeaderState::Altering); change_result.sender.send(change_result.result.map(|_| 0)); } @@ -338,6 +340,7 @@ async fn edit_region( let action_list = RegionMetaActionList::with_action(RegionMetaAction::Edit(edit)); region .manifest_ctx - .update_manifest(RegionState::Editing, action_list) + .update_manifest(RegionLeaderState::Editing, action_list) .await + .map(|_| ()) } diff --git a/src/mito2/src/worker/handle_truncate.rs b/src/mito2/src/worker/handle_truncate.rs index da5b74e511..863b1961a3 100644 --- a/src/mito2/src/worker/handle_truncate.rs +++ b/src/mito2/src/worker/handle_truncate.rs @@ -20,7 +20,7 @@ use store_api::storage::RegionId; use crate::error::RegionNotFoundSnafu; use crate::manifest::action::RegionTruncate; -use crate::region::RegionState; +use crate::region::RegionLeaderState; use crate::request::{OptionOutputTx, TruncateResult}; use crate::worker::RegionWorkerLoop; @@ -63,7 +63,7 @@ impl RegionWorkerLoop { }; // We are already in the worker loop so we can set the state first. - region.switch_state_to_writable(RegionState::Truncating); + region.switch_state_to_writable(RegionLeaderState::Truncating); match truncate_result.result { Ok(()) => { diff --git a/src/operator/src/tests/partition_manager.rs b/src/operator/src/tests/partition_manager.rs index bc79079036..2f1ffeffb5 100644 --- a/src/operator/src/tests/partition_manager.rs +++ b/src/operator/src/tests/partition_manager.rs @@ -142,7 +142,7 @@ pub(crate) async fn create_partition_rule_manager( }, leader_peer: Some(Peer::new(3, "")), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { @@ -173,7 +173,7 @@ pub(crate) async fn create_partition_rule_manager( }, leader_peer: Some(Peer::new(2, "")), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, RegionRoute { @@ -196,7 +196,7 @@ pub(crate) async fn create_partition_rule_manager( }, leader_peer: Some(Peer::new(1, "")), follower_peers: vec![], - leader_status: None, + leader_state: None, leader_down_since: None, }, ]), diff --git a/src/query/src/optimizer/test_util.rs b/src/query/src/optimizer/test_util.rs index 6b14a5af52..58e014afff 100644 --- a/src/query/src/optimizer/test_util.rs +++ b/src/query/src/optimizer/test_util.rs @@ -28,7 +28,8 @@ use store_api::metadata::{ ColumnMetadata, RegionMetadata, RegionMetadataBuilder, RegionMetadataRef, }; use store_api::region_engine::{ - RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetReadonlyResponse, + RegionEngine, RegionRole, RegionScannerRef, RegionStatistic, SetRegionRoleStateResponse, + SettableRegionRoleState, }; use store_api::region_request::RegionRequest; use store_api::storage::{ConcreteDataType, RegionId, ScanRequest}; @@ -89,14 +90,15 @@ impl RegionEngine for MetaRegionEngine { Ok(()) } - fn set_writable(&self, _region_id: RegionId, _writable: bool) -> Result<(), BoxedError> { + fn set_region_role(&self, _region_id: RegionId, _role: RegionRole) -> Result<(), BoxedError> { unimplemented!() } - async fn set_readonly_gracefully( + async fn set_region_role_state_gracefully( &self, _region_id: RegionId, - ) -> Result { + _region_role_state: SettableRegionRoleState, + ) -> Result { unimplemented!() } diff --git a/src/store-api/src/region_engine.rs b/src/store-api/src/region_engine.rs index 483d3cc1ad..850b9ad3e2 100644 --- a/src/store-api/src/region_engine.rs +++ b/src/store-api/src/region_engine.rs @@ -36,9 +36,32 @@ use crate::metadata::RegionMetadataRef; use crate::region_request::{RegionOpenRequest, RegionRequest}; use crate::storage::{RegionId, ScanRequest}; -/// The result of setting readonly for the region. +/// The settable region role state. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum SettableRegionRoleState { + Follower, + DowngradingLeader, +} + +impl From for RegionRole { + fn from(value: SettableRegionRoleState) -> Self { + match value { + SettableRegionRoleState::Follower => RegionRole::Follower, + SettableRegionRoleState::DowngradingLeader => RegionRole::DowngradingLeader, + } + } +} + +/// The request to set region role state. #[derive(Debug, PartialEq, Eq)] -pub enum SetReadonlyResponse { +pub struct SetRegionRoleStateRequest { + region_id: RegionId, + region_role_state: SettableRegionRoleState, +} + +/// The response of setting region role state. +#[derive(Debug, PartialEq, Eq)] +pub enum SetRegionRoleStateResponse { Success { /// Returns `last_entry_id` of the region if available(e.g., It's not available in file engine). last_entry_id: Option, @@ -46,8 +69,8 @@ pub enum SetReadonlyResponse { NotFound, } -impl SetReadonlyResponse { - /// Returns a [SetReadonlyResponse::Success] with the `last_entry_id`. +impl SetRegionRoleStateResponse { + /// Returns a [SetRegionRoleStateResponse::Success] with the `last_entry_id`. pub fn success(last_entry_id: Option) -> Self { Self::Success { last_entry_id } } @@ -58,6 +81,7 @@ pub struct GrantedRegion { pub region_id: RegionId, pub region_role: RegionRole, } + impl GrantedRegion { pub fn new(region_id: RegionId, region_role: RegionRole) -> Self { Self { @@ -85,12 +109,18 @@ impl From for GrantedRegion { } } +/// The role of the region. +/// TODO(weny): rename it to `RegionRoleState` #[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] pub enum RegionRole { // Readonly region(mito2) Follower, // Writable region(mito2), Readonly region(file). Leader, + // Leader is downgrading to follower. + // + // This state is used to prevent new write requests. + DowngradingLeader, } impl Display for RegionRole { @@ -98,6 +128,7 @@ impl Display for RegionRole { match self { RegionRole::Follower => write!(f, "Follower"), RegionRole::Leader => write!(f, "Leader"), + RegionRole::DowngradingLeader => write!(f, "Leader(Downgrading)"), } } } @@ -113,6 +144,7 @@ impl From for PbRegionRole { match value { RegionRole::Follower => PbRegionRole::Follower, RegionRole::Leader => PbRegionRole::Leader, + RegionRole::DowngradingLeader => PbRegionRole::DowngradingLeader, } } } @@ -122,6 +154,7 @@ impl From for RegionRole { match value { PbRegionRole::Leader => RegionRole::Leader, PbRegionRole::Follower => RegionRole::Follower, + PbRegionRole::DowngradingLeader => RegionRole::DowngradingLeader, } } } @@ -331,20 +364,21 @@ pub trait RegionEngine: Send + Sync { /// Stops the engine async fn stop(&self) -> Result<(), BoxedError>; - /// Sets writable mode for a region. + /// Sets [RegionRole] for a region. /// /// The engine checks whether the region is writable before writing to the region. Setting /// the region as readonly doesn't guarantee that write operations in progress will not /// take effect. - fn set_writable(&self, region_id: RegionId, writable: bool) -> Result<(), BoxedError>; + fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<(), BoxedError>; - /// Sets readonly for a region gracefully. + /// Sets region role state gracefully. /// /// After the call returns, the engine ensures no more write operations will succeed in the region. - async fn set_readonly_gracefully( + async fn set_region_role_state_gracefully( &self, region_id: RegionId, - ) -> Result; + region_role_state: SettableRegionRoleState, + ) -> Result; /// Indicates region role. /// From c1e8084af646acac341ca114c7d499cdabb0d5c1 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Mon, 30 Sep 2024 16:53:59 +0800 Subject: [PATCH 051/128] feat: add `add_handler_after`, `add_handler_before`, `replace_handler` (#4788) * feat: add `add_handler_after`, `add_handler_before`, `replace_handler` * chore: apply suggestions from CR * test: add more tests * feat: use `Vec` instead of `LinkedList` * Update src/meta-srv/src/lib.rs Co-authored-by: Yingwen --------- Co-authored-by: Yingwen --- src/meta-srv/src/error.rs | 10 +- src/meta-srv/src/handler.rs | 411 +++++++++++++++++++++++----- src/meta-srv/src/metasrv/builder.rs | 4 +- 3 files changed, 360 insertions(+), 65 deletions(-) diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index 7c8d17cbe0..4c3d974c0a 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -733,6 +733,13 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("Handler not found: {}", name))] + HandlerNotFound { + name: String, + #[snafu(implicit)] + location: Location, + }, } impl Error { @@ -803,7 +810,8 @@ impl ErrorExt for Error { | Error::InitExportMetricsTask { .. } | Error::ProcedureNotFound { .. } | Error::TooManyPartitions { .. } - | Error::TomlFormat { .. } => StatusCode::InvalidArguments, + | Error::TomlFormat { .. } + | Error::HandlerNotFound { .. } => StatusCode::InvalidArguments, Error::LeaseKeyFromUtf8 { .. } | Error::LeaseValueFromUtf8 { .. } | Error::InvalidRegionKeyFromUtf8 { .. } diff --git a/src/meta-srv/src/handler.rs b/src/meta-srv/src/handler.rs index 5363b6c548..9d82920c08 100644 --- a/src/meta-srv/src/handler.rs +++ b/src/meta-srv/src/handler.rs @@ -232,17 +232,6 @@ pub struct HeartbeatHandlerGroup { } impl HeartbeatHandlerGroup { - pub(crate) fn new(pushers: Pushers) -> Self { - Self { - handlers: vec![], - pushers, - } - } - - fn add_handler(&mut self, handler: impl HeartbeatHandler + 'static) { - self.handlers.push(NameCachedHandler::new(handler)); - } - /// Registers the heartbeat response [`Pusher`] with the given key to the group. pub async fn register_pusher(&self, key: impl AsRef, pusher: Pusher) { let key = key.as_ref(); @@ -458,25 +447,34 @@ pub struct HeartbeatHandlerGroupBuilder { region_failure_handler: Option, /// The handler to handle region lease. - region_lease_handler: RegionLeaseHandler, + region_lease_handler: Option, /// The plugins. plugins: Option, /// The heartbeat response pushers. pushers: Pushers, + + /// The group of heartbeat handlers. + handlers: Vec, } impl HeartbeatHandlerGroupBuilder { - pub fn new(pushers: Pushers, region_lease_handler: RegionLeaseHandler) -> Self { + pub fn new(pushers: Pushers) -> Self { Self { region_failure_handler: None, - region_lease_handler, + region_lease_handler: None, plugins: None, pushers, + handlers: vec![], } } + pub fn with_region_lease_handler(mut self, handler: Option) -> Self { + self.region_lease_handler = handler; + self + } + /// Sets the [`RegionFailureHandler`]. pub fn with_region_failure_handler(mut self, handler: Option) -> Self { self.region_failure_handler = handler; @@ -489,10 +487,10 @@ impl HeartbeatHandlerGroupBuilder { self } - /// Builds the group of heartbeat handlers. - pub fn build(self) -> HeartbeatHandlerGroup { + /// Adds the default handlers. + pub fn add_default_handlers(mut self) -> Self { // Extract the `PublishHeartbeatHandler` from the plugins. - let publish_heartbeat_handler = if let Some(plugins) = self.plugins { + let publish_heartbeat_handler = if let Some(plugins) = self.plugins.as_ref() { plugins .get::() .map(|publish| PublishHeartbeatHandler::new(publish.clone())) @@ -500,39 +498,94 @@ impl HeartbeatHandlerGroupBuilder { None }; - // TODO(weny): Considers classifying handlers - // to make it easier for upper layers to customize handler groups. - let mut group = HeartbeatHandlerGroup::new(self.pushers); - group.add_handler(ResponseHeaderHandler); + self.add_handler_last(ResponseHeaderHandler); // `KeepLeaseHandler` should preferably be in front of `CheckLeaderHandler`, // because even if the current meta-server node is no longer the leader it can // still help the datanode to keep lease. - group.add_handler(DatanodeKeepLeaseHandler); - group.add_handler(FlownodeKeepLeaseHandler); - group.add_handler(CheckLeaderHandler); - group.add_handler(OnLeaderStartHandler); - group.add_handler(ExtractStatHandler); - group.add_handler(CollectDatanodeClusterInfoHandler); - group.add_handler(CollectFrontendClusterInfoHandler); - group.add_handler(CollectFlownodeClusterInfoHandler); - group.add_handler(MailboxHandler); - group.add_handler(self.region_lease_handler); - group.add_handler(FilterInactiveRegionStatsHandler); - if let Some(region_failure_handler) = self.region_failure_handler { - group.add_handler(region_failure_handler); + self.add_handler_last(DatanodeKeepLeaseHandler); + self.add_handler_last(FlownodeKeepLeaseHandler); + self.add_handler_last(CheckLeaderHandler); + self.add_handler_last(OnLeaderStartHandler); + self.add_handler_last(ExtractStatHandler); + self.add_handler_last(CollectDatanodeClusterInfoHandler); + self.add_handler_last(CollectFrontendClusterInfoHandler); + self.add_handler_last(CollectFlownodeClusterInfoHandler); + self.add_handler_last(MailboxHandler); + if let Some(region_lease_handler) = self.region_lease_handler.take() { + self.add_handler_last(region_lease_handler); + } + self.add_handler_last(FilterInactiveRegionStatsHandler); + if let Some(region_failure_handler) = self.region_failure_handler.take() { + self.add_handler_last(region_failure_handler); } if let Some(publish_heartbeat_handler) = publish_heartbeat_handler { - group.add_handler(publish_heartbeat_handler); + self.add_handler_last(publish_heartbeat_handler); } - group.add_handler(CollectStatsHandler::default()); + self.add_handler_last(CollectStatsHandler::default()); - group + self + } + + /// Builds the group of heartbeat handlers. + pub fn build(self) -> HeartbeatHandlerGroup { + HeartbeatHandlerGroup { + handlers: self.handlers.into_iter().collect(), + pushers: self.pushers, + } + } + + /// Adds the handler after the specified handler. + pub fn add_handler_after( + &mut self, + target: &'static str, + handler: impl HeartbeatHandler + 'static, + ) -> Result<()> { + if let Some(pos) = self.handlers.iter().position(|x| x.name == target) { + self.handlers + .insert(pos + 1, NameCachedHandler::new(handler)); + return Ok(()); + } + + error::HandlerNotFoundSnafu { name: target }.fail() + } + + /// Adds the handler before the specified handler. + pub fn add_handler_before( + &mut self, + target: &'static str, + handler: impl HeartbeatHandler + 'static, + ) -> Result<()> { + if let Some(pos) = self.handlers.iter().position(|x| x.name == target) { + self.handlers.insert(pos, NameCachedHandler::new(handler)); + return Ok(()); + } + + error::HandlerNotFoundSnafu { name: target }.fail() + } + + /// Replaces the handler with the specified name. + pub fn replace_handler( + &mut self, + target: &'static str, + handler: impl HeartbeatHandler + 'static, + ) -> Result<()> { + if let Some(pos) = self.handlers.iter().position(|x| x.name == target) { + self.handlers[pos] = NameCachedHandler::new(handler); + return Ok(()); + } + + error::HandlerNotFoundSnafu { name: target }.fail() + } + + fn add_handler_last(&mut self, handler: impl HeartbeatHandler + 'static) { + self.handlers.push(NameCachedHandler::new(handler)); } } #[cfg(test)] mod tests { + use std::assert_matches::assert_matches; use std::sync::Arc; use std::time::Duration; @@ -541,17 +594,9 @@ mod tests { use common_meta::sequence::SequenceBuilder; use tokio::sync::mpsc; - use crate::handler::check_leader_handler::CheckLeaderHandler; - use crate::handler::collect_cluster_info_handler::{ - CollectDatanodeClusterInfoHandler, CollectFlownodeClusterInfoHandler, - CollectFrontendClusterInfoHandler, - }; + use super::{HeartbeatHandlerGroupBuilder, Pushers}; + use crate::error; use crate::handler::collect_stats_handler::CollectStatsHandler; - use crate::handler::extract_stat_handler::ExtractStatHandler; - use crate::handler::filter_inactive_region_stats::FilterInactiveRegionStatsHandler; - use crate::handler::keep_lease_handler::{DatanodeKeepLeaseHandler, FlownodeKeepLeaseHandler}; - use crate::handler::mailbox_handler::MailboxHandler; - use crate::handler::on_leader_start_handler::OnLeaderStartHandler; use crate::handler::response_header_handler::ResponseHeaderHandler; use crate::handler::{HeartbeatHandlerGroup, HeartbeatMailbox, Pusher}; use crate::service::mailbox::{Channel, MailboxReceiver, MailboxRef}; @@ -621,24 +666,13 @@ mod tests { (mailbox, receiver) } - #[tokio::test] - async fn test_handler_name() { - let mut group = HeartbeatHandlerGroup::default(); - group.add_handler(ResponseHeaderHandler); - group.add_handler(DatanodeKeepLeaseHandler); - group.add_handler(FlownodeKeepLeaseHandler); - group.add_handler(CheckLeaderHandler); - group.add_handler(OnLeaderStartHandler); - group.add_handler(ExtractStatHandler); - group.add_handler(CollectDatanodeClusterInfoHandler); - group.add_handler(CollectFrontendClusterInfoHandler); - group.add_handler(CollectFlownodeClusterInfoHandler); - group.add_handler(MailboxHandler); - group.add_handler(FilterInactiveRegionStatsHandler); - group.add_handler(CollectStatsHandler::default()); + #[test] + fn test_handler_group_builder() { + let group = HeartbeatHandlerGroupBuilder::new(Pushers::default()) + .add_default_handlers() + .build(); let handlers = group.handlers; - assert_eq!(12, handlers.len()); let names = [ @@ -660,4 +694,255 @@ mod tests { assert_eq!(handler.name, name); } } + + #[test] + fn test_handler_group_builder_add_before() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + builder + .add_handler_before( + "FilterInactiveRegionStatsHandler", + CollectStatsHandler::default(), + ) + .unwrap(); + + let group = builder.build(); + let handlers = group.handlers; + assert_eq!(13, handlers.len()); + + let names = [ + "ResponseHeaderHandler", + "DatanodeKeepLeaseHandler", + "FlownodeKeepLeaseHandler", + "CheckLeaderHandler", + "OnLeaderStartHandler", + "ExtractStatHandler", + "CollectDatanodeClusterInfoHandler", + "CollectFrontendClusterInfoHandler", + "CollectFlownodeClusterInfoHandler", + "MailboxHandler", + "CollectStatsHandler", + "FilterInactiveRegionStatsHandler", + "CollectStatsHandler", + ]; + + for (handler, name) in handlers.iter().zip(names.into_iter()) { + assert_eq!(handler.name, name); + } + } + + #[test] + fn test_handler_group_builder_add_before_first() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + builder + .add_handler_before("ResponseHeaderHandler", CollectStatsHandler::default()) + .unwrap(); + + let group = builder.build(); + let handlers = group.handlers; + assert_eq!(13, handlers.len()); + + let names = [ + "CollectStatsHandler", + "ResponseHeaderHandler", + "DatanodeKeepLeaseHandler", + "FlownodeKeepLeaseHandler", + "CheckLeaderHandler", + "OnLeaderStartHandler", + "ExtractStatHandler", + "CollectDatanodeClusterInfoHandler", + "CollectFrontendClusterInfoHandler", + "CollectFlownodeClusterInfoHandler", + "MailboxHandler", + "FilterInactiveRegionStatsHandler", + "CollectStatsHandler", + ]; + + for (handler, name) in handlers.iter().zip(names.into_iter()) { + assert_eq!(handler.name, name); + } + } + + #[test] + fn test_handler_group_builder_add_after() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + builder + .add_handler_after("MailboxHandler", CollectStatsHandler::default()) + .unwrap(); + + let group = builder.build(); + let handlers = group.handlers; + assert_eq!(13, handlers.len()); + + let names = [ + "ResponseHeaderHandler", + "DatanodeKeepLeaseHandler", + "FlownodeKeepLeaseHandler", + "CheckLeaderHandler", + "OnLeaderStartHandler", + "ExtractStatHandler", + "CollectDatanodeClusterInfoHandler", + "CollectFrontendClusterInfoHandler", + "CollectFlownodeClusterInfoHandler", + "MailboxHandler", + "CollectStatsHandler", + "FilterInactiveRegionStatsHandler", + "CollectStatsHandler", + ]; + + for (handler, name) in handlers.iter().zip(names.into_iter()) { + assert_eq!(handler.name, name); + } + } + + #[test] + fn test_handler_group_builder_add_after_last() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + builder + .add_handler_after("CollectStatsHandler", ResponseHeaderHandler) + .unwrap(); + + let group = builder.build(); + let handlers = group.handlers; + assert_eq!(13, handlers.len()); + + let names = [ + "ResponseHeaderHandler", + "DatanodeKeepLeaseHandler", + "FlownodeKeepLeaseHandler", + "CheckLeaderHandler", + "OnLeaderStartHandler", + "ExtractStatHandler", + "CollectDatanodeClusterInfoHandler", + "CollectFrontendClusterInfoHandler", + "CollectFlownodeClusterInfoHandler", + "MailboxHandler", + "FilterInactiveRegionStatsHandler", + "CollectStatsHandler", + "ResponseHeaderHandler", + ]; + + for (handler, name) in handlers.iter().zip(names.into_iter()) { + assert_eq!(handler.name, name); + } + } + + #[test] + fn test_handler_group_builder_replace() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + builder + .replace_handler("MailboxHandler", CollectStatsHandler::default()) + .unwrap(); + + let group = builder.build(); + let handlers = group.handlers; + assert_eq!(12, handlers.len()); + + let names = [ + "ResponseHeaderHandler", + "DatanodeKeepLeaseHandler", + "FlownodeKeepLeaseHandler", + "CheckLeaderHandler", + "OnLeaderStartHandler", + "ExtractStatHandler", + "CollectDatanodeClusterInfoHandler", + "CollectFrontendClusterInfoHandler", + "CollectFlownodeClusterInfoHandler", + "CollectStatsHandler", + "FilterInactiveRegionStatsHandler", + "CollectStatsHandler", + ]; + + for (handler, name) in handlers.iter().zip(names.into_iter()) { + assert_eq!(handler.name, name); + } + } + + #[test] + fn test_handler_group_builder_replace_last() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + builder + .replace_handler("CollectStatsHandler", ResponseHeaderHandler) + .unwrap(); + + let group = builder.build(); + let handlers = group.handlers; + assert_eq!(12, handlers.len()); + + let names = [ + "ResponseHeaderHandler", + "DatanodeKeepLeaseHandler", + "FlownodeKeepLeaseHandler", + "CheckLeaderHandler", + "OnLeaderStartHandler", + "ExtractStatHandler", + "CollectDatanodeClusterInfoHandler", + "CollectFrontendClusterInfoHandler", + "CollectFlownodeClusterInfoHandler", + "MailboxHandler", + "FilterInactiveRegionStatsHandler", + "ResponseHeaderHandler", + ]; + + for (handler, name) in handlers.iter().zip(names.into_iter()) { + assert_eq!(handler.name, name); + } + } + + #[test] + fn test_handler_group_builder_replace_first() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + builder + .replace_handler("ResponseHeaderHandler", CollectStatsHandler::default()) + .unwrap(); + + let group = builder.build(); + let handlers = group.handlers; + assert_eq!(12, handlers.len()); + + let names = [ + "CollectStatsHandler", + "DatanodeKeepLeaseHandler", + "FlownodeKeepLeaseHandler", + "CheckLeaderHandler", + "OnLeaderStartHandler", + "ExtractStatHandler", + "CollectDatanodeClusterInfoHandler", + "CollectFrontendClusterInfoHandler", + "CollectFlownodeClusterInfoHandler", + "MailboxHandler", + "FilterInactiveRegionStatsHandler", + "CollectStatsHandler", + ]; + + for (handler, name) in handlers.iter().zip(names.into_iter()) { + assert_eq!(handler.name, name); + } + } + + #[test] + fn test_handler_group_builder_handler_not_found() { + let mut builder = + HeartbeatHandlerGroupBuilder::new(Pushers::default()).add_default_handlers(); + let err = builder + .add_handler_before("NotExists", CollectStatsHandler::default()) + .unwrap_err(); + assert_matches!(err, error::Error::HandlerNotFound { .. }); + + let err = builder + .add_handler_after("NotExists", CollectStatsHandler::default()) + .unwrap_err(); + assert_matches!(err, error::Error::HandlerNotFound { .. }); + + let err = builder + .replace_handler("NotExists", CollectStatsHandler::default()) + .unwrap_err(); + assert_matches!(err, error::Error::HandlerNotFound { .. }); + } } diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 662de433ab..88d7e5bd6d 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -358,9 +358,11 @@ impl MetasrvBuilder { memory_region_keeper.clone(), ); - HeartbeatHandlerGroupBuilder::new(pushers, region_lease_handler) + HeartbeatHandlerGroupBuilder::new(pushers) .with_plugins(plugins.clone()) .with_region_failure_handler(region_failover_handler) + .with_region_lease_handler(Some(region_lease_handler)) + .add_default_handlers() .build() } }; From caf5f2c7a5650d0cf86f2fb8e151fc265580e59f Mon Sep 17 00:00:00 2001 From: Yingwen Date: Tue, 1 Oct 2024 11:35:04 +0800 Subject: [PATCH 052/128] docs: add TM to logos (#4789) --- docs/logo-text-padding-dark.png | Bin 25777 -> 36537 bytes docs/logo-text-padding.png | Bin 18102 -> 25396 bytes 2 files changed, 0 insertions(+), 0 deletions(-) diff --git a/docs/logo-text-padding-dark.png b/docs/logo-text-padding-dark.png index fe0c976ce1cbdaf09904c7caa0e1bdd7bdbd87eb..51bc012a83fa28d8688c3f56db3e81513349eb22 100644 GIT binary patch literal 36537 zcmdqI^;;WC^FADcdkYkoQlPj7EyatwyO*HB-6@n7D9|FsA-G#{Pm8;|Ly;DOThVWN zp6BzN^PY45g7=qPgln@qvpX|8_dT-_YAUk0Smam$0038B?v(}rfV_+N`x*lc@yVkN z`2het1<1dW(DF9f&%jKk(>}jg6%Z+-1AbxtqEslW_J~MA{89H4A~kgv2`y1+2~>3= z^@7PT75pfN$eaxz7E2`7K_pW}@QtY!e~JozTI5rVK7OSg|8sCjuZde_yuK0XzO^qW}2+e+B_K#_pc~IruNX z9797ff;N1WK4SjooWF;K99;f&ng9JVg#YOpunm{vjq2BbktRJD+r0z*e}n%jRV@+{ z$u7kCu|L+o>R|-E5&4^H|Ct7gi_s|V6dJXr`LBAc0hbT|y3C*2K*V9Fz7$3%S#SQb zU4M#3vKxs1=itA{WP=Rk7+YL>c185(;J@s8qxhxYza2!nIK?C9kQFqNuZ91r2Mo1& z75OjA{He`Ca4_~6eR*=mQ#8a8QvW&lFEaiA{Ror!fMCx@!G*xx{#~dS;EFqV z8NillK9mM9k~zs5rk)%!K{Z1MU>A^W-tVTjtGG7KA5V{>`m&z2R{qP8=RGq6n<6={ z;YbQh>$zA)Y+M58P+_M42YjJvz}7S6nhdUQ+?EZiLXk$pV^1Up?%dt|B$!!xjMb zz$)ci)4L#eBn=jn{9&;Iu!043TI#YJj|o79G0NSuPKO~S&$l48a#9DC-G~50ao1b{ z4S>ez?^3qCo*^q(+=qMrU+4eB7?cUQYMhV+0kBwr5z5*n;2w8P-PRmn4e-Vd_Sg*d z$Ak4g8xLR&t^>S$EdDsyXp3Q+q1{wfV^dy9jQ{%fjm^>>W7GHpAHk{Y$5x)<@52u9 zyUAHdM-whlXyhBSfE8tpDD;%%L#Xweademe1)l#IVhBADTyO<}L+z6gLbnU~%+3#q zVy-;`#76YS3yNjrc^2!YrS#|DX#s?h1e3qx_lok?oYF6BQe9RDnP9F-jdNW2&>yO# z?lo4gpg?J0@z2kH`PK?@jg=VQTyYyi-ZSb1wYoA0~6eSA`>0S2-i)A9(s7C>Y0jJ=YIWq0%AM z%U7}+VXmh1#Y20z!$aRAg9w7*IaEQz+NZipE&JH&-x_J-2IG1#@0L(~nVA1w8$vEn zqGWv3FOf+rE!Cz_j`cR~;1;=iFY=IztE1tK6MOrfEY{*hbh-QD@&4qLg&?DYQYIbTTnW%&pscb(EM9qT1XWq4-uV=_&~8)2+*>CTU9seB%$Y&9Un#xR0l+V55?{A z#tQ}^y_DiU0@R~b#bTe;w+HyU{!6U-_#}i9r8#bhiYm&Up*Y?c2z_P!Ff*ZMnho)gf8z{Wo*DVE7m+E+$GmJ3wAZ0O9G5?_gh`df@M`zTPX% zPM(h`Nuf@Bvq;BL?HL(2*!^Z1)%RvaR21|tZr2wnt^&Nv7Nls28+fwOlu0wqXR4sV zaYQ&YGyqs3n^g}I4nFK+Sgy@I0E(1G!8iX5s{d0?{Dz=U3L1Y~nu1$GNfE5PQS(3& z+yJ1Lk+8u`J?jHL_snclJ$`Yd<7PWHTwiA>ha(4!=o!^h)x^`F|4pdCiWkptut4xNq@SxBt`4(B<=AHz?M^RTJ-3D-%cy&1bTJmW3qXfpa^fkR z{Rk&DWNM^6eJqI$iPH5|5s^l}Zl2o~oS7FVc>xoDU(+2Yq!q=N?MoI__s2o)fF#oQ zNP=XrUb*|pA^Mdhz#NMLePJHLtbGvX71I>3V4t%EcNqSLy=*W7c zbRFCyyA>I(-I+9`cRzw?q)!?wle)de?$97|(AEZay-ogoJO!glP8|7`!yA%@28(oJ7!-m{SH< zATPW+WkkMu6g>K=brJR2zp>Qy#k>4~iz)IW0#Xx7D~bfBASIP|0~T{>7)Fj=Te)3N zwyat?qa*@w{%(rfoFV#;b|=)bfJY^AMU_{}BGm>9R;_GCnOKi;ZXMI>PnLM5UyP1yRp_~x!zK{jQ zN8Mlo>LW&dtc+-k=a-XM7F#SPjG+ykRNTF8T%<`C7MzbN8?;MbSbAJfEqHiv{}A#G6L> z;YceE;hk3h6E{9jGE^^1SsXy`v>&#prAWfB(7~ z{bo-i7VJj;-qzI=7WK9$G?Y%0EDM`RS2sA1Al_FOIve*4^f!kdX-zw^E zA`dr$GCodyJBoheU`@V4(ksqXYNnftBv8~U0f88)wZ}Zb6)*z-<_6Fx%;Xf_pVHO* z^UCIw^1-SablWB+& ztqfW3*X#hY@wdhk>5^hzA=C;io1U(gV#V*1$d5#kGn0;wGDMexaIU7_07qU1lASZS z{9S)DYQpyfRsE3LV?bs;l@nkQiFC3w67|7VtL3)klXyEnOVEW=+(4_SzPPGDxLtA( zCd>~2V5HoQ?+_o-t9G(PpzH|6KAO0jJ)$`S#sKy5mfo^?2a3wWkW0`A=b zvaj+>IT?P*-KMmrt>UfCAMdmQp8LIArQjrI!fquPtowd&01P1Rz-1(cqoP_cRp^UG z$sSxuiC}D$@sX@eZ93-Zz?S~u(2OWU4z$5ZNP0xh3RrLN@cOsnQGEMIT@5>4`?LA& z<2^+7O$WHa$6{4qS%Vd*y`BORO%!>yX0D0*?KC=NvRcHQ-kO-LDvW2ohLNG*VeH3g z*Qk>_@>>WYEyDz6-v>=NCruw6Qg39+l9*|iU3PkVvu|W4cA@3kzbVRABkh)%3GF>6 zrM8Ckps{U~OtUels|mfS{*IwPEMS`+MCNP(VfAmmttsZFmCv|y`+KxV7Z*mrtIW-5 zArC`Cr*4IGKxr@U_l#TRQi)j@^gH@2>G94&XJh@XmuT`V^JkLz96qeMcC>vA3olYn zlAaA>zclv_$G~fPuTz{khAF&Z&tH})YGclxt{74=!fRp^Lm~#Fh zZ5pe07AyRalWd(zh?1^`mNWePw*0*{+0g(KZRmZ^ zEXmMML9eRd&YPrTLw(%cf4Pdc-ymQRPO)y}g$DfR_S*IxE2*g1H4_K>s*Dxqz8rz3+Z0c3CE1QC}`CDm`vUyIgvPh9UACtU)#{vjxg?J+`7q+L)$l?SyEHVuDgOV4zd zy~k_JV@@-oEav4UgOYn4#yF3MlV^s05=iLe&aaIWXiKG*EKpk=&B!W8?8Iq$|FN-L zhQDF6+;v_K1cc>}4AV5Ev@5%*A`|^G5-RvGCwwI?V_6U93Js_Sh2)x_8+#;j+e}lV zR_6tg77}WuH>dprm_X@%bR&2V4e0~4N5QCAvxzDIRVvdHcoCUv0X%PI61X_G!-|!t^Yn*I*bj0YBLGGYz))r z)wJ8+|4cAu6wu|V`Km0?)rt%TdW^)-4BWzSp^;p?&qcO_F`t$wYKE{WJKYaAAl6fiKX}tF`_@_}<7ykKtNMwKPmrbX{>#&E}b#?NU z1+j0~hNPV>FaE&`xH7B(MIKdJ**E^dIYB&pJ5 zk2s|+`L?a{#iirl(=5RdBg{r|e3;>nI*7zADU29txX_*lYppYv59f2z>H72Dd zzO0|WUm5*}m3|~axIgA=^r(7d^;MzOvXm%sGofGsaGQPLg@xgssF`W$`cu*i)M|Cm z>0oxOLYgroJjyq?k{#i_`qWZFfcq9S6Omu76A8pxOwk zgyIpUk;7t5cox!Jqx=|2IZ`W~BFWL$?hCe=BYr?-SIO0}XT3U5cPhAY%C0f5HC^FD zQbaey!3gqEV*whke_0TUY>S*zWTw~Or+f!6ieJ6% zI@gQ1QxI^=OJvnDafmfm>lDWI<_uOgBr6@hx&LN&B~C2arct{Ikh3XlD;$1LhZg?p z2fy{!|Ghu^7!@{`naSXjt zo6k4^&T@p%j;27QO211U(TP{9zDZwA{Kz9z0du+Y+%H;Jkx@ijPt4Oz?uBU_E{s^D z{b+*~0cA<9A8xR^G8Myr%_iKnHvMw}Lc{`=AF88jJu4;`q;n0|mAA57tQKPIIK3_; zaTy5&31Dx)J$*%^K}bwZ%a(w$PvgnMIBp;;!P?pn1DUG{-6;ok6qzi8Z3-LZ^q#Ev zr>4qObN@YW@m>T^fQ#fnEr#qwmu52;aQuZzWCtAVw5i|h1Bn@d2YXD|&uS#k+9MF!tX7S}$_r|;!NG#_+ zn>U0GH0B)J)G!W`qdJ9L{j8_+mI6su;IOjob{>P73QdUQ+V||Zj6#7GOr(L$X1KQ- zjTAmXt_?@TsX^KA;ZI6OffsMgE1wUJGf$5{$QjzBZ>+rsX2k4-=0O!6;5$R7*8WdIWV%t&FlfD-zx{`KwO98F(( zaEg5LprJ~KF0JR=fr7YOOF2uY&HvIjh=5_tJ~rsL_$JfvSdr9I6*cbaqJUy;`rQL< z+ef_x`!s3c2~!lP-j6besbHEX!PvSLFsrRaZ!aC>cQJI_Msy~MYkYuu3^>WyfjOWz zq2%_p_#(41OTdLcX*)^V$MTGvAY2$yK1)Ydj_NEgAE^6M484d^b1Lg4%fu0+OJU=7 z6i|V7v}bdh6?f}53p*!7VAH20z~hQvy-2NX5y-nT6pBMI=FQ1$2>jUt( zH=+SE{TpiZqV2AVtqm!9580x^V>qU!f3BJRc{!hdfh zmci&DS7bl>tVVU${In39_x;`_{$C;}p^jUzjBekV9*LDCv=M{GP0+&9X z_*Pye(vff+7N+Z9viL04S+?k6Cr*dlriF@>hZWMO-SY^&e49!wq3REwIpq&b&P7YepvansaU0yq3`!6U&ku#lHvFz1w2 zRMPX%!-)+p9CCX6sfW(0+L&C6#bUn?`%uDq-GM4*fe8N`Me}!Ya%Cl7fG~(!&wU=^aRHOokI=%d>dEQmoK#$I~Nt{Zk0+zsa6D6*n@&@CDS6% zU*X9mB&*A(i^^!=8~u?Z!-#-Ft!E|aKV#@0wi`3V8y@XChlu{-`Ej?d03ZQgk;}uI zN1L{e4EacQ@-jTxdHhrJJm;2E>Bka*0aZ5EkG)l~z;)!+515@(Sn5?|AEW|`U{&;u zlkT_?StP-9Y095xCaOXEMk`A*aDDW{(Dg<^AHAB~2f|$Mz#m|fEHJ>T@L^Wu<7`OW z!rO&A`uanf?ec7@7j+pa5cIWB^)L0a!!E^bwHTF=o|mfhzjAA-+S(AY3Rz6^Te8?IJ+)!+vG%E68-8qk$@{A94Ey|Cyhnnp`-f=jCS6vrLsZ8g>4aR zXM*KMz}o2~W@jE_hA~!YKh9tf@_f;J z2Ua-MB>04yWg!@eu|RO-s=#Z;&U+JcusDqJbY;MBZqs+BOQ_n=1GM8ufi>4s`=KP4 z$|2E&0!1C`0I<}4fjoCHCtReIP2~hGn6OcBuXXFq8aiWylJ6ZNNb8#4DXTjf63CDQ zCIU$KDuRE*-oJc8I*0I2xD0d<<(f3G?H!H0>%4dc!#!G@ z)5((JvMh|C74pbimdWjMP(izdSFsA07>)r{p58}0J{^A|*ywK`=#i!B`vs!qN3=MD zqsl=_?bKzND;?GQt?ChKStt1c_XaI8HA|7!Tk|Mq81M zTyP`|Hcq0dz2iY|Yn#=%G|6ndQ5?7>Y)w2-!F_fd3|2_+($Qf!TA&OcAg3P3WYAei z&s818bSFXrEk5U{NtkVYP@s$&iIIePpI)r->Udp68K6R>B{OsuHLEwjbu@!Y&zbc)OPvU+$k?Ed&Cs!!jQt#k^6jPBcqa2Pd3kMl z;t87X)ca1fa36_*z)Akuk)sSROHCa>r_pjPa^Gw z=tsE*5MZ?WaQJyZ| zj=TD(y>ZP(V;n$33zKY(dt4KE`DYuxG!W>#nke?p| zce;tU4}7C2e<85C?1#u!KZe8TXFcwoGcP^wH+ZsJ5>i3$a_vOz$=5IPe~|-*dY!=P z`bl9j&Y!o`emWm$eATXWf~)V6&bb+e8HXpjs4)IL@+1;*S_h)my ztp-0ZlV-NO4hvG5ck(zLu-L0nPnVF4!i0NMIe~JtD0)z>cpC;t4;@YCUWH$#$zsn{ zx3VpXe3`H}eZTkRol%@Yi1$LA`8{9Wu~8j2s5>=?n&2KmQ zE)EUZ^W7acAlQBJK83k|FEATjY2FdDKVIYFcv}TZhXC$b7hjki)P6yz2R3p%tHnt~ zU9`fz8j{6@v08>WM&zR)E#u_$Ip0R&{{WirRLp)@_{o%qmumE6h`kOEK$9u*KA+O@ zdf-?Soc0-Uv3`2qO77km&PO@kk&a+X$G~0_nCY{oTZU4xLbcH%<2sW4U6Nj@9U3Jk z=h>U1o-N)xw~IobNmJpP1DoC^y@L29qF`MDx%Fo10Y>bQK&IeRDg5M|j3uKiNsUn) z0YnGsuxrS7*C|xVxUA8{(pk|sSJL6K8EvITnUd%ILN)*Yp_~AmY zzOp2uJR48=0FUuIVgxQS@ZXz;CbF$f-whTiZ52a`Y({4W!;vD(_vkgj*ORlHYhhr8 zy2OSWK0oT7Ete&W)Kk%?$|Rb~W@>0eeYNIVjf(W>Sh8!p> zLu0d(_8UIk8lh)}sxD&=2_XWr&~-rjvoQ}5_jY{@_wE)`3LgTiyP?+i-@j%vZot#U z3lW^-$`RJRxK&E-SkvbdPLC#WlC{Yky)%6GYcGbvgjm$7Ghl*ogyN{VoZDZ!BG{1B+>|fDnL%b zs1X7B$54i=@S-X5Y9R&=V`sXiy~ zImWVAXBd5SF15XSTAv``h43es_=c{XcBruItW32()dR&uyHRbqFTm;;&*R-p)lNdm zOP<%a#}E|*hqNgik^y#6$SOZ1sKcU7vGW$Z@T+_4Bw+}0H`b0Kx@x-hfDJKaPTh!; z+Qd4fU^g4&tv<*;wJ9(*+A$*Iu32HX1HZ_a+A#%qJ{PWO4`2+rXc|!k!r8_wKLzp! z=kh=~E7L;d0~tcA{V^EFxDnA7-M9oPE)n{9lGioFmK0*8_59%# ztCMx?4GZuWVn0yaK{G#yo)F z8RV(XiBwEQU%?UG>=IH(73dyKUhe zomm0ppBd+OZE&Lo2bsA)+eigm7=TcEGs%vkLs(2R1dkfg2w-IrRupdt5{J1ijOj%U z^pLY~ls|q3$Jp8bX4a(5L@HE0z=lmpBZrk6<+h#^?|xG|6-u9Y4;LV%-V64Rw!gj9 z;YPWWd->`~OIu`8+QYq_Sd2O=1Q_+5KjMZ~+f_EObxWcK!? z3!K~g@FC9HVP(LIss`ug@0%*`>6kpQ?f%x_yrqdVShwdPV54zz+9{QoWb5rEUtEW3 z$hWi?AwKd?8AD8vf2PrvCznjyfEWY?)LGFL$1g?b#Zq(H0+UuAZ9N+3RFuVz>rezG zxj@bhn=sYz$pVo8ZT3G`Pin2`PqE;`ZYwD<%aKI`?$7jbb$-<3nyE=9zXGlKA#0-@ z)Ka{QRHHndBtSQrUhAN*-=irY#(u-5BFZ@4B(3(YIQkyBb8G>BL$h~90&Xx8*=?au zc9huPpwF6A@{(J5M`2uLd_=~EhTz=={wX#s+>#TGArHGc=z~tRG&yJRM+vEgDMkh?PT$L7A7|*K2FUh}hp}P%%9J)cZ&J$c*Wp zDxT%<3oanJ9R?JYbW&wTJ=?-v8{QEKNk_SEqc=P!G9F$TI-}AdE$CWy6c=H0yU~A? zFlqd7gX9puea9pl68JPll>5}3(F%Oxt2I)Q^xc2bNk!v$*Vz746-uJaP;i#7#J>R> zbJ&n-^9+8x-eTW{uFyGEqW!VBYZhy57DekhYdJG_can^~V14V$fF;F+(BY|e4PLlR zI`Z1F@&{Yq)~28^lJEmxZ?>h~}LL zT#$nq*kHPqyZ3eA$1b!YuW+ch+J)NRUP6DgI}a zU7_JLssW$z!)2UXk^pp?@aSFzn(@1ER#>|3<^r52+|>#Z%?wKs!H1zEwF>%4PR1~i z+%LxOL4!!5ZUhn=3_aU9t744`GtshW2w`n z+yV5x1U;LT8;SIxt-iu)&{CorR}7|Or$sI8ku9;J;3hql#|=)OhG zi{^gOp|xcw$>_52Y_U)+_zO5(4;R%z)1YVlV~>>d%_k&IEL(fjmvpr&zJ1=&2&dhl zTfg1b>BZcwd|MDIf6oYo94K2fcqPJE3+3nw^{V1?=y|L^X%$@fK*04}_ai_2E@|Sh z6#J%!l(wEWL=Y^o#XJxNX1}Z7{!%}~zUgp}4{T&KIs0WunZvyV=TjYa5CI7H9>+&! zFR^SFbjcco#iI`fG-n=TkDmkX$>ApNGv-E$o5Zs;Zwc|JTe4Vw6qHyNO+=n}@7_z-df!jf6Ote$m#u37?bwqqksL^$_b1qWV*}pc z_;i1vAAD^Tr;xKa_CgujOF0@DsJIgE?x^g)a-OoZgP=<|zts6*oyVzMe|54!`OI0I zL;5Ud(^!ka*O1QEC_n{~bx&7OHS?5>X?AJ5@I-xF--BTITUb4j9hTPkvL@eyo5b;@ za8@T`XH4)|d&kCG{{cgrgR_rCTGO_2OmM44L7SS8=g4e)MRLuJ7GkS6Ats6W)o6IA zKE|}{%|So$^IV4j?Gy{zaKrr-#q%*{Z?ypdrpvp zs*uFFsS||^cRDc~J~;TxS?=9e6XCE|HgB5N%Jvx9>y>5fudJc0zY_E9l36wTgodY{ zyPoTi_I@h=@OVu382&DwJ%yOd5v)A4zc#lO@WMRzM%iUMsBb?w9*w6)87T~Wji1rO zD$gQhJ`rjZ#VgwdtW+&&l)D&YR@zwD%rB%y__n5@Gtz#Ca=F0C+QnsI~``exk zQP4AUMr#o>ZuvUxuO~{6d_3+=YRHZ|46LDgDO=D)?Mqyo8XZViC-`Z`k1L^WC8{7osC={ zYC8pI9e-m4YASP09$3N=azeQ_%gT`Hyg}w}XClxL!?Q(H(Z$}JXB58zZ44uK;`lyh z`h$HevtGli@N$IZt5$44#|&0L%2&TD0iAo2ob02okQOJZn(huIcET&lIn&VlkO{6tW}fli)x0YNmA-iWIU;)RK-qf)hQY8wz%Km;COaDl zpxD;m7#aGGRX9PoPDwcGtbh#9DPntPGPSTUq~t@dLL(s05OFJR)S-vpy~O6ZH*)Iv4F#&SY8LsYHcysb_${aBq}skQ3odK0?Y zyEb@fI>3Svw5wJ=@M1 zOLp?$0ONQSw}sKos%LMvY|V2TiX~DH^QZv>8FQ3ahe5>$DuMC+0s7kEG6{k|3G zKnk}BXfwO|9`|_Vlf?*OYQfT$&MLYrO?Ui21CJaEqNby#cPEkP5FVQ-L)z6bVXp}D zMLT{?XQD=4;Hm1$4f3? zmtxhLAJx}CQ&zF8z&+l<04mG*dNeYIp}^HVOL3Cp=j;Yb5PQptis+|S)aH~QO?k_z z-K!(@%m?c8&JS@HFhmmsK^iE;5zJzFoqr|33Ach2v=R5#nhEFnHKj1uZf^*_lLYnJ z)W;GZxd*K-?*q!uqL#1fp3<6g#Kt%AvM=ESuiU9uP*07ad*4!0P|ROc&K$8;yj|-R z@D=-kh+;;d%Wz8!M;~x47imJDuy+@py&xxwFJ(|g7^6MIvA~)z0?x4&XNc0UETBWC zv0ZXLT{%2BZ%$WNOAG|m+s00VEZMP7_KZ56FGi0M_m(4BTY;7 zR+;4eh@HO?r^>EfO!a0;hk2nehia`eRG7dzA)M?xS+%>ppJX}u0LdeQL7y9&Cx&ms z=y1*#DtgRxM_r2*rr{&qL;Hp5IH?;8CtT(2is^-cDWjG=*_J(ynf9`!TeeA1kEieGjG?p2!EOB6e~?UCShL{_JJ4e{9)% z*CMuPk;iYmUKl|IZcBzE#ADsWigJ`#g=i-3!;9Lwx(DU%{avj=lQ|u)qAf;tzMXNL zv&20Jq@fGc+?^TnqDK+P7c%3QdA%Oar9f$hZ5zGVpN=l+b1xEK6;;tobvG@zGs?Co zL7A?bQQ@e&@O=-m5mBa(6!aZrgUp^4Au{+-%)xgQ*I&#lZV3Wl5h}O)g?-i$A10 zBB6k#plK;4U5P|R`d;%|KoOPp%YI5rM&!Qd{OTqlvmdqQTR*eohv2QTs^gseQs?Ov zsD1f$AMKFA_{=wNG5{51&J)|8>a*}OPV+Z2jL5)iJ06#&J*9~ zZ!-&93lci$YEu<=8#NbFQZ@#}sC?3>rp})J^3w8t<2jiGu1i~y1{#}(4JhWMv+6I- z%CT_o*3M)D?~0g)w)e{Y`qwB?QUnd%n=D|aB1bd&XF&uJa@}L7XePR+JK5%X=i;q=h@^Y|Lz6^CqE1QG1)f(hh7eg?X}z zA@oZgLNs}K#^&`rEm=y|qu&tPTTw=^lO3f$d5lf4TVv$R1n|K2bJ!yp;zzS>hx;JAx-Ych{ z-(?*@w$%E&HwLf`{3%SC>5?q1c#8(|+PR~C;w?WL>s}r@GuG^rnJMUsOB+t;$eBd= zuPW09 z#&>QOwzkQ4v=HojlCia@8Head1* zv_Jj{ywnJ9FQ-qi+wi7ZWl%*pE|{IZ{^wmB)7a+eRp3a<{AuHg0`N*A&fJF)oGgI{ zG!~wc=w~?%Cg3W_4Dz*gx&rX6Jt2bW1CL_=EMkoHr#VU z+wIzmEJmMZU?>4ah1S<6NnqIb=Sj2QCt_^_vc^G_=Gsr09V~BC>e}Aqoop*r;}Pxp zu)kiC`I3>9t-ne*86zNu;yZo@Q;Uo10XH`De*5^!wdidAN0o2S@~ay13u--4K7RY= zm_(Bntgtvv2E)i#VDfznz0v6I>ylO0c=XZQZV??=AZ;e5r7YFYy7K6A6n|Pt18sxL zhzeu1^6u=MyFS1)Sn10<1m;7d#pX}Ufzu@M$&-UTa4)-%blWJIOhX9P>9_SU$ec9N zR^x^?C8oNxUKA>h0PiRPheN?KQizZi;DS3TlfS@|EK?zsPUI8FW_y7lgmDHg~+9Cn{O46jLA1bgR^Kk&Bjp6vrk!DfC~nm4h)VL9}Zd2eyRxT zP}Uc~bVuT;Ns_r#r{HEy{nGe^VAln1Epqkc{a(5{B7`kjCDxc%SQyl_%8ooMb{HrwS43?Vj4r7V z>915pM|7xn8Wu02`u;qeXY;y3C|=Uw04|`9rLpsYHqorDs^f+3@TH;DBPwH=9Rjb+ zDND8!^6z54SAcG}e^Tj2B-ik7QU3z#gr=r^bJUigtsFodV05w7IfYNTNQPM$W*O$s z)}!v3<~;JVE*^Mwd2-@e4<7COxU0IftV~E}!n}YvP9JJf{JSKRuV9BdZ}?* zFTpIC?rGw)eB2=IM=#hn_vjAud!-q=^0=St2_GVM{hR)oNlpeFnFq`hDA;Dbm%-b( zAs6G8GMO!G0?lgdyI>}Il+Xm?wZAm06p0SwiDo3nRlJrhdNT7$adX8HR)IWSC0OG9 zNs>WkMn3)3J?;p*w$H0wbC$LXWUfa4sVD{Q=LgTS+U91O8+1mSslRRR<+Fr-sn)sP zx8@4#Dtjo=vQoQvP*VXH?6>($<%*`|{j_h%r~eo)KI$bB4Efyj8f|hcs+~Z&_@$*?RQ#H) z0O&O`?fI44oWx<|XbQ9Xywd4UjgS@Wr}1{Q@r~`t-?!rRcbLY`jctDVJsMS)FF##i z&-$=V6nFa7GJmVmyBGV6SAJF|VtAnQR+peI+r%&Tu>@;ncFw~T1k&~-cE<93d`$`A zY*IK`dU>4&Z@{>wl`l=NxT!(+(W*-4UC(urng?zJrh9kRhZ=CWHO_~lRQi!?|00`8 z9y0@yL1r-%*gNzRE#vi9rDqGOUxR(une_@WXs=|82Zjk5Q*uI5mQkO^?jQNZaHhpu zC>XEuGKN&_FN}^nIzU0}jj{IjsnU`dDBW5|%2wB+kQeh4q69u}{}p_A%#eb(-dh=r zq1NZo!(Qo|@&y(C7GPC$C|B-F*Y@vzJ?6Qlq*i@8ch_cnG9es-;DX_1o%lMH?OWIEZs+8gvFj)Hx8zxS_*XK#+aS&NJJwCW1FH~-O|lm2m}(bCl2gN=pWH@ z?9ptu(-OY;qi_0~uTaa!Iriy{Nt$0;TNayUPyPu0MrGlxm^~zt~M~^g~?Gekn`-VgxG@ z2>q%;INT`8(aHzTP9gf(^l+C3>@&jts<;6%)7W}SjzN;ILPB2^#@lhk&jl$`k!EqV ziZgzn<$QOQ@9=ow9X$B|dI9KLYP76em3V>`Dg#Kns^yV?y0Na2ArCSM z_)b5TmMq`NQHsDVa13!iMM6S5ut_BP+JDOf3d;@qTQ!Y%biG2UEEAqXY zO!dMfu9!cVM>KH58GNx|Q@za^fjh~Ecu1L~bP$&t<<~qvtZdIP_W5!#0k^X$Wn_uQ zLlooe>7^rbAeS00A>FuR!xvvM(Ao(~FF8Ij{061KpSDn_XE|5fK?9A@i|=N1KmDbx-g=lxJIM{In*_CQ zu0~6Z=#q8jIzt7*YvCJIXD>I#-pO-h4z)$0HpdZ(rRBA*E?EGn9_X}w^B8l?$B_kn zWyXj~?UaB#CE=A(@c{G-Sl#HxnP`?;!~QBQp9LRB(6hwY2Y^qEHOKmDNyTelFE0ow z97^K`xQw2u(wCb#&o)_{)?1xdic3kvq_R|sfG#MEhdtN_c)C@G)7&nkImBASSD{8OgQ2&$S* zSINfiYyWA%c&V_%XF(e}D57SQMl0gBVX$w7eMlySVxtpX@fTfcbGyAXWKI zA_ceM?#1F_NjR0a324rR4H@`A<8K1T=0vI9<*m_>rJ669wQS=~DUn8ow zqvZ4jgu}rCI9kPx*WvZQno7)X=22oDPf{4>pkU2- zGRt-l<*LF|c3=h=5aT9!K3*&xa3^Z>_#V%7+V6eJkNeLU+R~U-M6*B>lMCFq%1qm% zK1F5h_mLq_YFd>J?G5O!{1YTZirjUL2u>3}$>#j5M&DAib3I{cnc?!U97Mey4Tci5 z*dB?|5)=>ynzpDx2PrPf>9;lx_#?#MH`?3VB<3*d2p_-1@tze=jF9i#NZk+!OXrQ4 zJ;!IX{!00?Zin=7{VCSVnhu5XGX7dZKe#mNhO9h%V>Fgidoqa}lX5X_O9?iVFy_>g z6!jcmjJ!K*bQ>?+q;ILatv_m7vH>62TRVm`a{OyDEr8-$o;OgO;7JJX354LXxDyEO?#|*ai%SR)f(Ca; zupq(R6C4&;oZ#-TxXU}azkC0$-m5ybMFD%}%=FhiJ>6gTxsOF`^<~~Q`Rx=$_`z*&^H~m%f5XR@+dLfnENVYodrfMv)=rQ<&Snxu*O}tG$?TmD_h*!wT3e=I zox?X&($8gSQ=&WA4d23&+>Z+q7Vi@rCmnBvRA=i!QErfW?ne*hMGE0DwF6IhH|fiS zxZNOBlP5aO0Jhx~`N(=>oNGrlL4T#L?~gB*GAHZD?chi!C;6VYtwlkHvBpwO8ciP5 z06C7NEw|c(K?yDrFa8aaq1S$P64ghGRy{h;@SB7B#WSa}#azgzgH(DPlE`PFmx=EQ z4H9}Y&Tr_yH5;A$b%9}UzfL%nXVzjLHwc4tkW?T&m>gA?1)byPT8N@m%aJ^X)VuxbI zlj5>eFa8V)z)ktzhT%CaZm^8(^JBawfbyfPww%i1>(a%D~0Tta^K{7k%2os3g_%3 zUG@9RiT;fb_asLiNmB|78rEN%Q1V~Km9u)JfQ3K_Lyvaj=NR^P-``E^wcG*?80wL! zW0T7it2RgJNH)Hb_&9?Sh(0Zh)!^#COAvA?^mcC&OJS=2&0bwA{MI+y;}7mFvAo0Q z_KzDsb$@5!nS0cWE&ryRsd@WI;Yt(b_0Y+VZ%2ZAIcH=cti?F?em@D|M+TaX%O;$|nZN<4nF(tg-7FkEGjG_Z1VWPR0zCKN@6hV7)a#3=h z8Qaw{{$yu#sS>E)4uXw46SnYs3L673|kyPT=yE~}Ne?nedc z_%-U%0)-%P)PvV?v2g~1>l|db+epHoxi7kpFN5IwbU%NM9M9(wHM)TW_ez2jos}+n z^q2nFAHV)Fh&4!CK6_qvaQ|*vA7i&rX9UTx<+I#Jw*W1APH%FMFNAzOy^qF|uGqpw zcG+QBoJ#r~a5Bl#iC8Z5EbJ>J3RO5HY{TBxbh?%pbK6YJzoXlA1F5&?LTKycZ*C%J zH{Xi_Mf9d(^ZDdV#W%Hma(jtez(1VIdox+s%#YH>Hyh+KNqTfJi`yYOxypzWVYz>s z0+$ljeS9AUYA1ByH?YEThqyq9uIlggECm@r6C?W8%+~24rMOlqxZzwOBa@z#&(1%r z8_I9z;@%n~U`wXOc2LCY(0$<_UN2a4j&GW&<^@^iOe0BZ6&{Sm<~UYnqTcF`nT2t( zZv*{z(g!6JTM)+9f|F($Io{4fU`3zXg*;)gblyIZGc`+#(DG3)M1?Nf8Ly#%wBn4k ziZljFp?8z)J=Zy3JTK`#Kk|te=vdecPM4#Yl&~k3rv2GOFiKq&2e_Pk1VQuoQo=$(OVozY zs+TPz$ieL#15-5IHSb`W^Nq$W`QKzy)-24@OKtbSmW(*H4urC+@84R0wUK7uE{Y6shBOKzqZ#c}0gokx6fofA*FZe!Wi`{b8B z@o^+pw*B44wJbo*8h-al`Q%d9$J>xNi8tf9v_06{i-bb)^dR32x^_lR$O&ea|Fel? z7?ELBh6LNF^Mh`DASc@Kh(FEt^{gSlOK&A~`13OlwfVD8THVGSV=bEBsmziY>oZ9< zCVIqc72Lk1a0~0=_GCO!Jd|4jS}ICp4tXR)ra`th3G!?)9b(`@l=m~Weq(QORU$l( z>Ow@B0Ahd8WidA1W%91Uu-mJhW=FH@WmYA@=o@67F5&d#b7?ki!3x~6SusO{#>t6mwl!xA1JM7GnPotOhHvOt_ zB2?AARE!rCYGCuVmA=caS&1G8uqAs)m!T2 zIns>~Oz551{#Gy#@G|G4r+-&}ddZjHA=i__Q|p@JFlT;2S{E@B7q&O(ahw>p;U?Qh zu~##bJ~f2W#7!->QWm#Qj5G$tMTV)F<&wm>y&5P3fk(aMC;%pS{A7qv-S9^-ZNe2{ z`WZQ#R;w6<_b<$ z#nI*zNMaot)EJ=Nw=k1+6J&iEcRMtgt)R9fieCmCVqCvjJS?;XZ>SfV-iN*8d5F|C z?8l{)#v&?dj`OgLi+Laa?r8489v~z9&VSy}UL!ZPq0<>@D$Gl`U#Bfmy%tcZQLs10 zQ(HqcuXi<_oAtSyfDaiBy<~tRP{5?X`ipE9$NpBnY+I4WrgXyA+|HSLL*0dz5GKV$ z!U$c6pL}8_TSB)AxbUSNV_kBcKfmkE6tbySSr@=Hx3A&Ha_@0F`cPeL0R)=h&GF$) z53!wW-^&KotLL_9VapQrm%j7RP!MR7PE~7<`S5x5$@eNi8yel7?`%u9R7-L-GXNAaIyp4_41TfGA7 zZ%lQ|%$nXNYe7C(OmoF3BG1RQSOrkBCyW)h83lRjfAMLQwJ3vcI_XOkHg{@v>=jy7&$2_Iaj*H_|wa4Cv3(TuT8 z4XeBD(9}BlZNqA!#GQpMqeVIlESaPLpf3sEZE@ts7qBp9DRmI|Y&9RWL9W4S#_`HP$)9bsn#MgFsPjK#= zXA2@5$q!CNs_r!d(OKP@x42o5zBLKVta=Xs{QA3fupjrh{+aFP6VYuE4dn|@mp zwsw*W^svPVajmu-T2A*O4vOan#wlD*W3}ailf%Vw;*$j2MJDPpP=i?^R}6wZKcU{) z=ULPGd6B3-lmsL(v<$1FzksMMCge_C>Z<7dsL$p{+}zZ$R1nUa%efC!!#8{m{P%v( zcZ|`FZ)uy+40u&GFJ%wUy$eQJ!%%x(Eb5kUS`-OSIYPb#Wu-7)5*X9O?-#Qk)(zWsTOl4$`5-uL#8WrI2V_1lrF&slQ{TNq(!LEx2yrp)$vAwD{c^Jz>W!aF$Tg4GX2o~2O~YF z*)GzObS4x4TajUrM(#j}((L)6=yQH+dkO{D5$`~;QRd=1fT2K>U(8dBRB`4e`)KAm zPmbzSctg_*H2a#%6{1_WWlE@0Aai}u-;|dzj2>K`@rG8sH{?N6=&fX;))zv7w^G_B zd8ZDog1fArJvZxo*%~G+K<6n^ja*GM&i@d~I`Jgtl&sAgNm2d0R^K!pq|O%}O=zZu{@9o zOu@Usu>ki+8pBLzH3sA!?kmr`aGnKu&2LIigC3l1!?RzD4OO7aK5xlNdPuhD>JcZ} zVTHY^nzunRTn&!k|5SbZ(gVgfkvz1o7HkwVU8B^0J;xBz!f0Mr{tTpayp=@DN)$}V zcDd^>0Wa#afZEtMvc>zEE15T3R8l-4Y46r_nVJ@OVTSdwk$hW_CYv*@Q&;a?cN+0pbOd+tv(}* z1z1m|g>7+aH+@>Jm3Q?@0I>C?Q!m~c^8@4{sU$FTu>1{~Z%&gKBzP$p=|K{qMvI5D zlx>|k>dY>FG<^%Q$%()=a(%5HUjY^hnCR-+oJuaWH<}1hSUyRiExPbItfe@Yg-dKB z_2+S={}$wZQ!{@&cHQ{FUrc%0=#k(25{Gn!lB+*{m!u zHGJCTslmE_8>62J#aXvUWNm(XAKdCTsbR}#u_HX!xO$7DuNh4i~&TC!wEw_uU4 zI=CD=M{7&BM7FN+jT4Bi6&I%4i_Ze-n!30~;g0w_=*Tk6-V5DRIb&TVfECnp;TPyB zvT|TM=#96dV1p?$&hg}RCZ9S#f%syi$zIj%(=>%gf!43}1kP z)(O=>T#cWWN*llH_`32ZE?+L^XMN_U;}^7bFBfY!Q!hGp`Ri2s{AC`k$2}&U{F6TOQ(CN#| zp&SbAv9!BL!p9@FFndB2AIg)SpE=hHvCahiQ^=J-m8#%(wV{X%Z$*QWCj^x`LFq}7KJnfDB=!t!^V1LaI7s%Y%^ zv72Km_b){`wCA|c#Z5bGOf5+oZ5XlhpHc~x%!5*iJB!JqfYQ|u$U`?7QN4C8reU`S zVl|ane~>Dl;u*VMA{FwfgVc?JV5r%z-c&gH@C_)jgcMHA1A6l&`4Nb`Cj@WRhCKSk zi#lwrMPG2#AR!!Hjz%tDY3V3TK%`yG(gpzgJwQtrzugTzSRa#DlQ(^t@b-}`9*5{} zXp#1>==B2u!yRAF*pxIciInnn0g}FLsNL8oeU){J{y@^X?%K#vs1ZUE7^OtCBS2Im zCY+3WRBJ@B8@AIreYS?`_``ae3|6UR=VwD{)DeX9zS>Uv_JDxhfYeJ?nN_pPsDGTc z4!II3&&vlWFsI_f>!w6~sr$rMs~mZFR7q+ilGD?4t#NzcuA|dB_pZ`|bTpjne)xd# z(7k9k;rmZHw?nzp=2!l6-))%XH0x<6W}mJ%jGohZG?}FnB?E^m2)(}X4$!M}nbkfr za>6J<#owFm5fJ;c^GMlt&vK=CXaIuAAeJZA*$Jne`f5|;oyOa&O~sHp8lxkpZrapjN_73tXcGJ06FRks28qiJXKsgr3aR+Z9-!;?8oLe1u$SSGlC`6R>aX zy#*&<$@kQ`zHoYFoCa>MnVNSla8D%_Y<`^0~kYefgE`*F2VX6kY=aJ1f3S)d|td*=;2= zos%qBTv?+`diPUu$?AdMP<1zAn0~uD?VLFs59#3yVJVZND_q9Y@a2Iw4Gidur%d`% zK5IAZ+v@D&3$(U=pxc%k!bGw}s^8hz@xtQ^TXeQDz;nkSPx_6)`M+d>cyu2(DdQX~ z_?qi;*7&8Ob;oPDT*zr5nc41YVD28vsGZzfop@*^8nU$pP$&nJn_87>sw$~{=#KDB|oQ5jM$ z!Th>7bXFILy0`*;l0>W{HdOL}KCsI9>j3Npoe5EH)e+n?8+7(9@TK|EH~viAD~;F@ zK|l#JHrZ3WBcFLhNW+Fl@QJRRkol-jSml>Z4wby=6$5Q2u2PAR9u9iyr`x!HXc+;5 zM_6XjBym5}_~L1h=y#MTHk-qw>8&F(pFlYOHjlf1nA82kP6zve8mXyD{q2d{kG_Jd zH5LhGFlr|}}2-Jn=?tg~Ud-vLRJv|)) zoy9FIJsy|Ur$7qC&z`T5ChzVABa##Lpgw=4B92e@A~^R2*C3%p6e5vDb9nj@<*SHi z>5^g^DB6>s8YLfdC225aH4Lt6l(g~WEs}cJF+qkNYEHursk=YhefO94^flq$+1}kn z9T!)NCC%`wEO_QEh7Vy=*0LGcI9_ZyD#&1=Nc4Hu*4jwt@AE=aDDEE+ijj6Kzsr;c z^`hTL$`4dlDvh-v8H#2b;?qHCixV5ZwPZ>#@4!I4^!8UGCXvGj4Ggst9E$FtV)Tc^ zzh)u6+f{O&6g|}UFPpOD-4Ev0pfGRKHODu;#+m^A# z^EV4Ft-9+0X8mBPJQDOH_CD>hAZ47G;RjTFa}f%4F0fDQ47O$zJuEG)Wx_u>gIjk) za@l1(QA(Kb%I;L49c7Gkw)3CZown=1Ub;f@5At*$1!^! z*l?5*?B-@6FH85uLe@+bBZ8+4$r;x0ft8{WGaf1uFGm-85trR)r5fe=@iw?VK4Vkd zsdc`w;+wN=+itn7sa|HxXU&VbY(ZvTVAo zu{vORcrl!B2)+7e0>$oCY-i_>OvRsDde+jsBZ1=xl zSinU#Ob&rkJT=erExBVy(f^4!Y^Aurnr5*_ned4~Y`N_v5-60;bqU{d;d8t|ax5lr zHoQ?Ee96Q=(*R|ykrR^xlri}ZvA-ox zh8N?!YtMVAsf);rimYVTh6Yvo?xD?*k2jwgyFsh3vNfvGV7%)g;G^yVQHe61mb00b zc>l+(KN6u*o|gfyI0$RsSYQ8wrBW>53LPn>92wUS2*YI&tR^R7{QL^3jy$1|36bya zw#GjqUld9V3|{9ACK~XLPK%GA3M}#j;shC?*^uNQXO}PJXoYxN(@O-8L=OwkuO@FF z%}-Y3FG$74CZ-t~j*)$lAXl5JZ3#|3zZ>(40A9pek!?iCwa5cAd3JvK(pZC;%ScAz$YYsXM?COzKMUSRL+&~CqhLEk>Nur#uvO|ZAt20f4z!K88F`AIMKhOD#A*)dk zj?Ky1tr7<5dFz|yImhWS;!U_SHe^K@yfy*?9mPYUE#JWgcFaJ6{B}A=BZ6yvVA!ZG zm}jBP5F+|qY;GB=_4V^iGxvH9xm;!XgfSnF?_#5`RTk6{JbCxa`Gm7rfmN9ih&A_n zjxdz&UsP}o0Sb>D!Q$GMf||L~e}!mDvjl8qm489UD)R*YpCy>)G9s`QXR7Ka(z(8GlFz+bLA#j zNqk2MLY!fxS4`M(=(jJZElw4BYvDwgsf3dj$-4j@O5ED<<||?cV$8#PSmSCBQ{MqS zjpf8U{3h?S$pgG`Yc?sesPTc%`Qo}*Weu|qiSa^W?ghfn|)IEK_-5pgNp7Xp) zLAb5}Z<;-3ov;y?RnsMi<-{l>J&?RTQi7Zt>ihH;YCqBhY4fy@#=GXE@ytXZwPo`} zi!Xa+oe>-Hn+;AX&?q|HFEpc1-AUXXNCqDr3BB4h9m@$H0f{%^&Ae$uw`!Mz1+bR7 zCy7z*UUh`g?0o1CKGW8Ux|ztBD?t>^Ux_x+5Z6L5>iBN9jLQg(4n-6lAU|0wBj?q? z=}TtvgjgFNGV&Uns$-j!#1wjO@KWHr4PGJMWvX&WZ93V0CHfs3qCUW1PDVl!s5NAbpg!v!uJw;sf%hl zA+?^d)(CSmz#1&{B6C7OEEm*{id^4)i_Mb(@vr%`%nO|vsRwaGM60t;;eFsg{(Y8y zAC*M75#Ss(%K11$i#Ox%T_{f}2paCV!S(Vo1ySQIhUVb#&~s635{&a*cUoAEX}szd zvL}pCb#>{O(n4;S&9aFmPW$hIEv$()mrk^$i@P_i{=ViypTxtjcf1g!HT5?_)dzzY z5OAxNUhaIAM#5wD)}-DSi<4b<6qtX?lj2*2%KcV+QRi;Uw%6*WE zE9E-$otT%8r^g1`Qx3Ry|7{iX;7L1#D}@))OSKGboB`TIZUR2HO`AAnh4Fhqo*Hh` zQ5J9ytF)n!TM7*!ET*O4ozzUIxC3WJt$}v~*GCvxG!goga z^Ugc?HHkBL37Od!vdFA;^Ox-T`O;Bt&*>%qf>l*0rR-yFwmJBoa;SU3zwZ`KD?E1@ z!2@l2aUH@eFeH;}#WC4hJHSUR%)Fs#n1C{N2zc4vnhOuxS?fsy;!Ez2EM^ zE4990{V+xyO4oq8$@$?YTc8^K(*23M`QZ`$8Sgim-N`8O^rb4&bSTJ6Bm zj4BsbjQ;3Q`zc3CFiCM}CdB4v>2!}L`PJE}+VVG@u2=+TZLWxiUhGo(`cBchH&Zhk z2;uT0?haHqi^OS`W>lml#Ks084*46~Q=!H%Zd4xIn055NXu$|Yhx}Jer|^d%##vL5 zkjo~)kQ{?)cQZzIBE%!uINjm5=L!_BVY`ZwUkNr!$T{I8g9g7696@&@DG^Py`3x1;iP`u#nSR$>NLd|Mna4MUs^9fngXNv0Xv?u8leU2gVzjUjs%UOiy*m1jc9N)cFe}>LVvRTLweYd{C!golGpg}mFL4{Ul=_}nwgmA!0Vr8J5$C~T3mm;WIJ8yA88lFY!p!n zIJXILxhF+6P+?h1*c3cih5hPf(T3bv*$QrbaAb(hN6gVx{e?k8Kc8T3&^BqTQOa#J zoyz&9+p%1$60$>avKk-Q6~z0A%OH>eM`@mD*66$0krL4y%uU-Mda~;cQ})uo=>=ms=V2efUO>_5PJp*XU;e zsm@b+5isA#CA0W!w6X zF+<7G>l7|ZIeCdbyhtm>YN`B>?$!%l+NH(O%om80q&MUeW+a`#ZCYAU`~%xluX)v7 z|JWI!kEdl#;N0)$AG zT!oGqZ*3zeT)w@Cq}+B@Ei~+ivFSO!CIKGYNxm} z`NGG$huS*!E;@X3KK^qJ-|FBo#HUY!JD6m)qJXBWJD}()%-lRtmdwhBkUC1xK(>V) z3(@VYCeL`F%oD6$b1fCRI$J>lU){fmZF?+o-8nhLV9<(d$|1W#kV8XDrIw( zB`z32SR;DX1KXzT&16(ik-U#DcN9ATZxQ_!YM}*oULlDPr;p?O81wo&jzjZ%ji5h`ms!} z>v6O0M9E9)&fkiceeJp$R@KAY5X_o{D`bkG{5_yAOW)0Ye_BTOt7d>h_-fg=s%vBL zn}+}tZ0%kdCXF*BTT|n^{A-E(QtnRmU-o8G2P0$o845|0@TODw(iG z1U(@eFQ_-=vt<@7jGqJYMCq2=mu|65u!Y3~FXJ9w6z1o~I3Fgw$+#H({X#ZbIIm?% zHtcoq(kEaCg$6A=^o)s1#a$W1hl9{Ph@mHMpi!!$V-)1Gs!0ALNm#h7g4#?#j%Tc? zAF4F)Yuj=u3}`o*#U(9g6oX{*sng$U6)91~x#gs`rTD|Ig&j`POta1)EOe&cmfR|4 zIen>=G#x}=Gmr75-^!`0pMF(&?b7pg*T}cq5#(a=o&})k8XR>!h?t>eqlC`pMNXUc zah2nKawjZ`YJtl;MG!-L*148hEYs8ue}YfSa}|`@kx>N<2LoSj`<6-&rSIR8AcTN< zyo-FNi)C^}qrc|#(W1^xwYL!r zAx6Bw;S(9{v8uO8`Q+q8>WS4|=**1xl~Bd(T4xoQG4?RH-7C{Ntjf$SJk zO&(C19R5|@J0gdlW63d~8YXQAQn_ZVrv|4z>cIj7BPDAL!68tB0T5C;9b$dt1G&N*m++?KvH68*SqV zS#14*m8-PdPXzw+au}fjolCy&ZLkOqNPV_P4rjss8!e3NlZfUh)Kv|n)pmRZbmg^T zJ6SA2U3lqZoc7w)vghNOEgsti(Ohrqrg+i$81hB_%<){3-66UxWo0be1xvts^n`l= z2fKh<`>;y_f%D!<>Q{C9$TbXh%tzHKMzVGRF0h|2cJ(_~LFG##6itM79lZO#MB=6f zpIh!PZeY~UHED@=-gM;DAIL(QP<)4=)5_>r?g7Re^*u~4IS&$TOQBKPSzBBf=V3*E zhM)((z-8$V!UF!Sf^sir%Dl7!Q^cGep`hoa>xua`qjD2;b(F{F>Ih0QE>*2|-H;cVH z6H3{)4OTjSLuYlH{|1@JRH7=ZZN$J%!T*It7hPZqqFb}d2n@C!N{VY#H61`WIe`7Y zsL09H#eI_#_F|CCoc3>@s~1nzn5;1g?;R%ENo@}0a3Bi}z2se9bzwxI&GpiwvYD^M zxizYUB1!|G-uYS!i@RGR(J?cH)7IPE-Bi<%^gl%7#6Crs?jyoJl#Q~pKN)UzC?OjC z5zet#?r7+iagtl?$~{1IEnq~ppvofpuVxqGpI4#}Ek3noNvbkXYoH|B5w&`*5Avn;sHB)PnZY`rLT+!}TKSFc|4ZOFD~TYIAkM*} zvmz+OCTeXq&(W)EkYoKJr+Zqa56v&GJkvwmKw&|}ham0gIu8gs{-!-TNa(W8%v=tY zH}TSkesee`#no$?PE}*=54(GyuIU{ygn^J&h9n4l-Bi~38DABA<&QqOY`)hpdAo;g zy^;7`i&BX7ZLxNy~TV-D(B3WsV#ke`V&H1$1g z|M&I)!gZs-I-O1I{!| zM4OKfLf8nQLJAdrN}~!?7`xWIdzb1~h!t1U>no9ab;qe= zE-lK1bo=0sKl_HugnY)`tMiJxFR;5ccBBL!L5Z|v)@s6w$Lq9p#gW{|jU3&s5fr|8 zrZmlOcwQlim_wy!Za$Dc+p6V_{uHwIyZhs6WC%g)l*t17GvL6SV@{?X{b{Z!oH2Q; ztq~TW&a!=^@S?vA0siHrNyY=)!8)o4rF6&94m?$Rw~rRA^j6h}*@zA!=1&~YP~zM= z7MC*Lbrz{T$usu=qTaZ9CB~-b`h&>pTXurVKUpLQq@1 zq(v`7S5?q)i*pj&DnqGK7_lL#>uBhYd3v4Po7|pGw}R{ul*=-*RUC@uK^&(6^)guJXFqo*<{j-%jWzOnmk45%qtceB$Ss z#Pn)J5L)5B9}E_3U|Fl@TjM3x(QvL^j&-whmS21CY2`w5DSF1UF1$`1Ill402aYS# zlt|iA{T6~2SM&ZW7F7ir^Y95wN_33k*j_y=*87kJwt<0#y{ z-TibE^wp9Ymi9a5C{AJCOYXb4rFw)o;?n|@vcK>qO+Q*`T$KDxuXJfwf|%2L6(r%C zH}$g2DXX}Dp!qBJD7+l&1`la+KhRMG+|tg19O}|IE)3>>qAS66fBjO__u&@#e4ZH?7j^3;Tk;#eFKg`z#%NhR+wJV51l`&OHt zcZ8EZ^V)i@d;n)lg{{{IZ|HP)v51;0nw+_Fn@Wq_m_X}_Ki+IJ3^Pk>WO4WAq;Q9; zS{-GM7L4Y&O7zqRjbd$0)91x>fA0oCllPO}2$(+dBK)@BOLY}H!C#{LxCF)l(*QY_ z404MBCfaiq48`SH`$pNvkx}Fc*YU0(wr>s%w?hw)2qK1~Uq^6q%M=0iF~vBarV@uN za|~*AHi~-tr)*6rE#}ve|#=CC_OIRz%JlTG9 z;yot*uWW@loFdtC$SoQ$&lR_>?hw3UQ(uD?&fX0$hRv+nJXbE#PW@xJIt6A+D-g4) zIBLfWlHVF&3SQQQbI8+6$uPC%?HsjAiKkrA{j8nJFGp!cxm7?h+Z4z17s(qg;tZPU z_A7dJ_*#~qk}(T)1!U7>yvrFp(&>a9-xoc(zh$@Xy7$}jo7j^RLXO#&Wi8(E1ixnj zZA!`*?yOqw@ax{xt8Via9%=s8wDb@*TG>My-qt{C?3_saYi1j(!xHVKu^eNA&v;#A zH^T`cA$Yl8B-XZGv*BHje-AMxgHxM;mFR;3&H9kz{*Re;=KyIy7!j zaR=SxO8nG9h|y7vpt|sE+$hD?A(9CV}=X{tdLKb=-$XKIiaEC}Hx zHd5yM>0n>A+?+dd~Di+y_)MS?S7y&I8MqWFhXyj#g@J#O*fRWt8L% zPjYkpev@|3&R#2hT1RHcq+B#WeattA=m7Dmxi*7NCPii!j_Z@mJ$0Q!DH({fKAvO? z7?C09r19z_vM2m&z&Jg_yX1m@cxwi%Bo>?Imb?a z-rK9Qo72}r2x;lB5MO+LcAhV`y&#=(9bkkYaUZ@VW#xG|D2aHN(ibI|TxAdJF`jk{ z5Lt8H3){0Wl@8Wv>N$hduV;43t89y;j>ar49;C-@A}fkk8f)A6gRMP3J z)ues>Jc~TZ>mx~lTz=PaaOm`p{>gH z_ig#~(;HWB0;$phGnQkq{RW>lx3fEIyxmKt?68R~91<0jl z!n9#zM53V*$}KZHYWPF{IOSnjW!=Z+kcut@VINMNq*EYkp3q3l<2<+IP7tuOmS#B#I|_6$%Y z#s0adwNZTZa*Yw$mW+@>c45MxS@LU7hZivluL(zu_C`ziSs_l6eM&=(>$k!X1Aw7g zItU?#uhtE}8Ct90t&0NMR7{%oG% zCDcf-VX7#!FhScUE}a+85&vF0K1jkWu%_nCf_C8CmNOaBFC;i^OV-|OkL@IaCgnmt z^!*v!y@@hr-Hd7$W8(S4@8Cc-zVbtmK}t8BUwJ#}I=qkd-HOSi^RqISa8AmOxB+5E z7KFYsVtGO?Cb{*?LzEfJK?#D$ZQU1~R7o!oU-VNTVw^9qM4-_ky#}6$8cA3&cNJ~t=uhLmAvFbd3RrR{k<9ooKL=c@ICnB5yMM+ z>sRR>IhnXV{y3(zsw8n9mPD^un&sV|FZqR z@J}|aoWj?YA0s%BT@H+}6m#$re?0GqT{))TB_lQ(FRlNaodwkMz0a|U*t{*)Yumkz| zz&Y|N+M2+l3uFeN_=L0NhOzxCCPOJPB>yl}3W{Tj)y9!vI*jQ%npe(H80lTW$L0)8 z2DXzAAG!ohy+nhu^c>jeFYFN7S7Xk!V&*S?#_VQVwqM+0_>q!LwDjQ|%6l?h+J0agoxZ-kh<&l@cd3nK6NH`Wy_ z?UMQJg^VLSnKYwgFse7eX39ALaANa0qW*~+T$2V_%Fem1ykTPMb%$`icpgkiM;*MB zhgD${8T9NkaEU2l$mEVzv0JfS2$ygl)86Z1%)srYNR^l?>iS+z=V$@01RDdNlCxbV zu~Ujx3rStq#3{UG^)!7oXUOh=*X9B=2k!&2!uc<@DfpZ$d%Qx${>rB7uWY-*Mz+z(Vr%OczqS&oC<*OVAeN)#GWW3AvH9pT;5g|Nodt)XO z-^t1(5<+DjFj@s-EJ09_N|#eVlg9G(Yf1IWxa*`;vI zqsv4!c_v09j<2lEAzh%=S+dxB9BOXTTDmk*-@JTPLMnyhB~eOE-q?f5XJ#E8T{oA! z2i(z}9)Q@9f%dT6znh&v5-H! zFB5SLI|DMmR)Y<*_zZPvL*ShVc7xyF(C+IHr4im0vcoRW{7gQe5)nrQuUh;yz3fbg z81BO4|7V50Ujvi=Z7X>-4!k*f5nwlY58M#593J4dShY+*bN_{+@nw14*$C0RQR$K`&Q=X zK!4536hh3W^=HKO%0Ex6NSmoTBP0m!w9p(%W&eVi9su(uR@pA#5d9C#fNqUi|ME&V zVqSTVo@_CRn(@FNRJzjWx%z&AeYvPlh>mR`nuyd{ERpHa8A0EQ>xncdvV=QR*T^o` zaxy#}*yMIx0f-v5ccy3m3sHAsU~RC84i>hr*Opm&a}fj4O(L;d%qA4ZF69wKKdo^-ySzstbpvvgi~WIAh~3-saTCe|BIUumcUI1A{GI#X3DrINI<_(!A7VWD^$hb@nJ^ujeu0D&1X2*UNf{ z*iz_15%dl0_eaGnKrIdYH;xY9SomRC*x|wn`h|&yVB81rzawDBKulCro!jA-thv*kh z($d!@fb5TnGq7{~rv(4{a(cj0pEcI1>eXK%<(i}jORSjkiRdl{oef>hy+QiVr}@8; z+Qb0o2EDY^Cjzd{v;w>^yMZRv02eHD_ME&CH4LlX2Qwwdv*LlZ0ePzP-&OsezY2U( zBLicRh$+s|y8XCPT=C+~MCdz`od>$!y;?J_KujqwThR;QsLWFMe^=puxA2Jp^i?Uv z&#Tke2qDI!PZc3MW5V<;g%&}~ep?V5i@V=u)ED9GTAC^R|L^Rc2RqULdU=-Aie%L4 zfl%>|FNSBu1DAQ=CeDg%$p>ME|M=qjy0KQ&JTip<9*+M1e_+nP-^Q3eAQkqFfNQa= zcsP@zbs-`Ih_)>0+5RzY9JXsLv~KiS74S1=7}r}X9{j&``ri+RN@7TDAw;P~`wbW7 zO^E2fE2)Co5D&DJSL;nE3oqgjJDj_sofmWrK9h~K4hIP1t*fN`|8DUb_7xDSpz9wn z)%l0J+mGQ?zm7D!TtV2W=CT|A&&2NYKe)*traq^L_rPI>cZ>zonX?~ClpkK3_an9P zjGWJnb*rHH5|TT?Bm*$fZ#=$W&k^yjx<|fkW4s;CedCtQ{_748Zd^07IPig5 zMhSu@EEt*?fu+WEV{p?R&Ht95=4Z3(1p)Nb1WHy+vFDL3fU%!60MiSfn4>eg+d!@b z7Te5dc7V7OK#9{bf`!)+JrMvc26~8R)I*~&G@9ffIhUL~^pC%}>D1|MhE`$>K;Y@> K=d#Wzp$P!J?*b42 literal 25777 zcmeEt_dDBN{C9{wTYGEOsM^#jf>v8wOYPW2jS$q{lv1nq9<9-;y<-QpM^xulMUjztVn5Nyb73001b}R8@2V0N^I>^92bJ z?p<6TvW5FY3RN}s0020;|9RoLjC+D`oy_l_8@<k;rn^K{KMM)qylb=>VBsLZ$ zyoa&ZerqRw0W$V>S*W!uyYuVV28O%;HU6saDBzv{`9iqK^z$G1djjAHWC@~#vM(}(Xtb}yW$t4a>3|I-cT0=fdjnnu#esm`C%HMY`%tpFro zmA`+CWH`HTzbK;9=G>ovT4#H`At~Y6fB#UOU#Uu5FKhNuWMFjw%PgT$1V__=i+k^tr{HuATZ^*_4P+E~ABQJQFhE<$qiJ8nWV)Wl<;MX;4X|Qxt6AQHB z8pr4VG?M3ahIB-VW)6LI9#Ow9eS%Idn{##J?? z-LXRtPrJv4Y7Hyo(HN6lNi~Aqyr_fwzA_g5J0yiF?l~oYS|sZ*KWyqyd5uG-*~o_h zCc`nLUm@YsiCX<1W8op(T(cI{W0{-j=4aqd@t2`rXD3gee13&4*IrNY7yZvriVFbk z6z;~S_$ByqDgJ`=E_n@CT$nX^o>ba@#z{2!25A2U6R)A>W_91V*%vfok>i39g_DIoE8`=pcQ+hZvNf@-|BELj*qpPxv1<6ZHI97cQ~-6=V9Z ziTsGfKjwNLy9!|SpcmzeSkDtrYh2`TacTI;UvH@IZsQ<&*WRS(S5fl84fX{3huLgI z>xqXNQFcys7v8~Msx25TIb=Z46K8+mp1d54Tn46}Fd5+sHYbd>WV-sm#%{GpD zZvUk4U)8%y`V!CY=|78YQw9^V>Auz518kJH5-EZ(uhWV|>$Aj*6DuG;&!=m>6Sdme zO{wHsvs?9;GyYrZJ+g7yj3@D)hrU4{t!B8gqT|*M7yWpf#{R@j5^a2e2oTvah|QZ! zCr~)`i#ahf4FyLvT=pAz6qts)==j!5F;O{FdIJa1Eo*TE;%8z-^+rdD^}%^CbLh(1AacG zt-jj}VGU;1_!gZ(_}*kV)-QwOA_rT4!rTF6s$l9Me@o`vXDw6c%fMJ9EA^k`>K`g$ zKRdmZk5mM*GR~@S$S87pXgg7p>#?x_{!uci8d|aZXiIN$IlP|V7<-;WI4uK4BGUhe znS-Q3qM$uAH~OqNbd(Iz!&1^PvQq!JpXXmaN+KfMyncr`ybJUO?$En;;!O3mLsgcA ze04%RnB}WMr&=p-J2IlNXP0aVm zIY^=mS1@FjqhT!7U=*sSiN5IHh^<=uDx`y^S#UMPKa`RT7Dx&;@nBTcv=M}uiapFi zq{l8J;RX)%#zaT4%R`G4jE!SOa&bh++Soz(D4dE=(()|mNKjkLM({tA>D3MCz-Nzh zwx+}a3&~;El1`bk6QkF?@r95FkyDgwOir>5@R~PY<)LM9nbQ+JJ-t%XhNl>DbPS=C zSZQ-iX>a`h1PLRu@$}zPS(1gynj#PHW-c@P%~knIDLjv}VtcS{-C?!-)^fpm{s$C0 zm+|{I$)tzz7@#ES3f$5VNo!)~=1^nvUwGojS&}UnCu$ENZ>zZn2cv17Q){xDBOAW0 zXgns(%Frrnne=W%yCEw%I5|P4tRXr5lf!p~VMYq@VDOA5HZg(nGG8G^?$egVf0Ady z%}YE3{J;?JB?A=e`tk)^kS<>g*=|RBA|!VC8@!(kQaCg!0S(%FTGG%la`lvT&~}a- zMrfq4T(S2ea)Ec$`o)*P|FR{zXLx@39YPZu4A33VbFqL@oC_x3YJ#vm_?jUFF1R!N-pJ8Gu9U2f6aTwD(ZaN$2*>@6m6H|IY(Vm z1(1^EbK|l&{xtmeAS2Iq+5zsCUx#G}3IUL>JQk{*yOe*exGM3a9{mZ@X(zJl7yZqY z*RY=>CZ8UCt^@9GTkIV%?YNpNmV0$O&1PUqCsxbxqBoM4VwK{t@jLuRq8Eg09?q1C zOt$ZY-y1%C`9RLNCl^Ls#eMUnBU^{@LYn?=;wM9DwkQF^7HU!J=Gz-_C@b^<9mp7` zBf3#|oCVm_?F&9NM1ZLa(^3y3Iq9Eyiq~doYSV`bN!nz`B7KJhcZ1^uv6E9COIwIT z^DOpy=C9uqg=%oq_Uf;di%2cpK_n+G$3(cu?U7I1YEJB8BIDPMA2UvD0Bl3V;3a{I zt0A5t%Yll8Ay%-Ni_1W48)+NWNLPiCCmsTS0j4i_H}MYD&)qcSnIH*{T7?J><| zC(jqvYak-*aewJpsU)BetKB|ni4b%kQN0MVX4#pac}6z?3N@%H#or}VFDLny2f&E^ zjG+4E5@MSB^15<}2Cas^kKThTz=L%iO;w00{m}hzH$qe^!5ITo?Z*RL7GxZX$6nB6 z?;rv7>PB*m^as>7e2h@L9FAMPjcQIiHptEwh3dPGG~gmvmTvZQ2MLwez}Dx`B1twv z3avsjn=`k)ciO^#%7g!Pt*#*tyC5E1!!LO~k4z?R94C#y_`TPmyARDe2x7=*_kQx+ z`WLBOe^T9LBDmr>dy}}UNytmD$d>rSLqhZmD%>dAH&Ag8=n*;-k{PI&3-|bp(VkG^ z#a9%;-ZNM+vlMgbdMh;!l*hXQdULDlN?8Jr<9R`LLfAyhnC6=I*cX3IRB39!Y2hJG zs`kZheiRk<-UsP)^JMF7f)CQ+zhJwe4mowVdZ6{{HN?o}fTTNFq=OnpEV7l?yM{U~ z^qkji1Ivg160bF}tuKHi7w!qChacH;#6-RMXaTU$Dy^tm1_i4SCAEg=r2F3?{NzS^ z9o(}!)JmPC_%aR77;5oPuMa5<2xC8!s4i&hDWwF8E9EzWdKf8J)tugGLXoC$;Zd{q zyt^VZKh=zcx#q|)ht}jxc3dTz+vSyYP{stCeL3HF@koPDOE))kzNfBvS~?S~9>&6W z@jCC5*I?@|!W#32sjTEVpZp87Y>8C>zF?YfF>V$xE-rzj(`S2JsG7VDU1zGdcJ*%|#d?f{R{tZl;N34Lr;oBXPDzMbee6M0|KyzLvrL z>v-N5e7j}p5Ik^%|Na$kDr-(S^4$HNaOm&%jvHOdDiaa}o2t#DU9buamu{{&O#Vqa zUKQGFjo$z2dCpz9bV%?rucw#r9`wgdh^exxtED}yST6V6Z%TV<#8(A1MM&hs zeqQ!far3$~JySneNm`?EQ=Cz#ucTt2PllJz*-(0OF}bAf!`nn$24fuk1D*%RT! zA;a%Ci0}nBi|X{`dgrfElV8Kj>LGXe+u1{xRvPVQRwOK%*`l&Xu82|-hrd61a|!h2 zs_Gn#hQCZg(HM7RJyyaDxHo!={CqkRZF362to(eM^hH0p&L1YwnNbnCL+%K#y}v($ zlBrd5aoSoo7u2IUJuP(Dk&s_jNOg8XTcN+f-a;F=x_R9E$P|=fDBkvvWPAE`hCpIe z&jSe38*a1DgXuB?dpO!Ft9uk)IW&>o{^S0FXTU1M!g zI)ggap7!(JfG;NzP|y5tE@LJKG=2SPqg=xm)1vrHp|&!2pJ|OCr>o)JVI<}~6Sg*X zapm7Gi?kUcSPcEsM3|cB#oX>VTYAye{5H-VIcVR&Nj>-)QnG5N*={8BtvF`7XthkK zAIPXXXFu}WT?mRl_1gNm%0wMMw;X=zc=o$*@f2)8tC-7Af#e9I#sTFGi?G*WDq3QExtlvJ{A%$Y5l@3RrRid6b9^HLjDDyfgb>*AS_M*MbUI@LAbvmJ844L}< z&YS0k5T*JvZZvl*;9Y}cbr5e)g^`zqcN9c@5K?SffhSc<_N^7QS`?7Jr*=&;CW zoEC{K2j1G?Walrm^fDl_q4zT>46QOVj?JB9ON+BP z{jrj`ph_-J=Dg~w0ivxrG0v6GeeY~rPb)0Lxrr3BLDQkXx>KvX1+ab~&r^&cA2B>_ zH+BwLBp}r&{KKmHQ&7+gWG$f1C66L9-8kaI)?tMo`~lrK7aVqn^c z>Y#nGdsi*YBjn-AA%56Vlk-pWUZRcNIOk-iKNs@yHWW)=1E?uDB_dhHMaHLo4U|Gy z#mdJcP5g~AI7b<)QJ;^^ZD7^yp$cu6Eg#KUJ>+kAB*o9=?fKXZnRnyL=t1M*9?*Pm z#;1orD7bV}8bS-9q+#m-Z{U*MT9+@2IoIFQBV#-yL*mfe5Iwih_ko%Hy5x=h?CmLQ zlK~+dtn>S~-ojEY5u>L55BReF%za&o7Nj+K;`?wpkaHmJ0djG`HE!9B;^S-C;9;?8 zsInc21TtHuWt2{;a@v1$)U`X28?kS0&lNN!6@Rxwy8d`yG_9=NUsN+^;->*=zdqGR z{3bo6m39{COzwUbm7+VQXoE})u{mL7ckc4>jkEB?Q?v*kQYS^pK*e$BF3!;d@6Wls zvjE&iOoo2d4Qshq5^qC}2JEp?FiLx$yGCb=$M42knRHtTRi`1ew$}$gVDVfoy@fRd z#3W*u0gswW(0zUGwk3)EPgs)j`95~s3#52SU!ByUW;PO`<|2Bc*{yj0HJvFhADdq3 zE+sh5&c|?F^ZSq7J6x$Qcd8jNBx;P6lgrKUeziOqNuf^C$;#eTE1JYmCq*#7R4S#w zqRtmI_q1eb3Gg^z#QT8Sa_L7?{bgg|7S-g2O(n0fF3glS^HQ<42n7aKba2n#mT+x?T2=P1(eXiKzQOTc;Hd21X| zs;R7M|IDU_smYXAi@adP`&G2+)?eUNIV$`+9v?nys{ftTuXe!Wwq@3{DsjXh^_ulk zBkq`1BXw{(oB!DRDCPrJ)RS#z()Rp;XvB8@7o>Qje1_A?k(#S>?jWnwdL3FZ8sGVj z$f*VWUUtDj$0A%zZLB37PJ+tx;Q4T5NDL-+24g~_RPRWK-(vk01^4v}_=-$^j#kAkGfNT)t!9%PY(UWMy3zzXOl7gBpe%9m4ISy<&JLRU}fgf01>9~ZqXt^{hsVd~%@yP*rSq}DPpU>sgf1Fiav75Ngy|DQzfjIL z;pF4~Qtvt@vGn^OK;cOK(t2+E=n%UtsECEgrQ2EniFujET77%0_&jSLb}3ZKgCn=# zFm7x`Hx~t#bQ46mbPjmXirxt#!W_H!ZjZ~+5F2;Cfj>3tdVwq-dy5d&ll_ekGBW}O+|JuN} z);K;y5&R-&`%q>9j*5VZyRnzXsj4`};Z+h$<9yqXSt7XokV2HYW%-GDY&lD_U{kn7 zm?$}8!+W<{L<0_%kYM#CW7?BraFQ)A&Snj)BHnyw$;j$PEc$3LP@tOW>_MajrNvJ( zzRrO#E@WXKo9PQNVpBtgu1jn+S?7Sa2e^#qwEYY2bZ#5DZ`6H*J*tphnucFwM$P%~ z0%ENjb0>Q`X<~UGra)>v=4jv&;3fhYAx7>)o6a6A>~c8~)sg=iHu$Xq+RXOo%bP(i zlu^?9lRM0tr`*>`b`L_m^;~GdCFkTJ34Z3mtiry(>hSuwbcHbZATx~en`k*Vfh{#) zxyZ_P8TETbV8?5Wa*F9F;ig_+t#Z!t^Q*?P^&>H@3WbqVht|UAp7N8WMZ4wow92;) z*Yo`~#nH{wMiPU8!@fOCK}=Ieb^ic|T=T}V;e(R<{1kl4>RNPqnBAdumyXDjThX0& zZ~2@u;%^NyzV&(6jjCuYmR$=Pnlw$7SWF)lreSJ%Vq>6P=Tx@aiqt3|Dol>+;PGH* zia2~7CQ^SLvheWQ>qr~@W@3&=Kz#g-UI-P&<46Lge}dRwijF>n4fZbKN#i9Dv4pEi zlHM)oM`N$r$;y@)^jove;Oe$DTDo(_jWL!u#_SjuHTlCF zr@$e3PAlAkiige9-JB*Z@)~Z;nw5$Dp%D3xhaFHlv7^XpM?fW!xmNS8WM~!I?$xt&r!4Y6A)L!zIZ>+aQw#+nuA3^6P&?#>w!L9oGa<_WSp!flhq z5Sq=>T{_{h_kkMPl8k!VM24&k;ik#UEIGK|av40s}dTjj21)doLG zpF3ytE@8n=mbBocfqr%Qw5iZi9!wpb9op+jWY8%}9(qY%Rf2!q=;j=&ar$R9_3B;F z95eTqco;6+`Vz0fG>#o0R-w%ZLwu3J(z+EgB8q%WL>XWL25A~<_FsUU@`AkGb`+0?&F zr@8~{TY@6%QIg1zq&GgUtsL{4Me;zKsucl{+^lvt5i z=zIl>q`uQw27x`7_3g;^^kE!9;*5>Gx7)Y9UR;J|L!r3u+L5r_ml!F+BeZ1jXmsJK zs6CM{ZdocRSBxdy7@hz{e!rCbm8AGBRHATT&)bFKa-%O1g8AF|7FFb8zBAyS5TWDy zxLw(wmJ<@%AJ~@|Cc!^SfMkcoUhkz8RNb_Z{-xH$@Aw$Z)r<|{hlvb*3g|3ymcb}U zlVQF6n+XD4Lga+UC$V)@!lt)SuxBV!`e-m{U&>`-L)MbFGN5oz#i+uBy{S8S_mVmM zvOVr+J0cy;@nNxxN8gXAX{wrhDCzR)qx7x{5F|8v()nk5$BUJ`^F!FoFFDw~HGb6T z(zx7y-$|U4yeNYaE7LUACX^6{h)Rd5ea!=Me34J?zE9D8O_dh(BEC291@&4F_I#pq zR33$?+_(tJ>(W45KWsibw zQsH-0yECw8n!Sa4Ja<-;T1LC$T=kXxD6VKSG|xUF+~@Jqqy0=R`l@ST($LPj){v#-MX@`j!r>vn zi=b1kIjj&2a4v>&+9X5yURhIi@qC;A=Bmj`H~zwP35Fr4?A6-TkLiNn;uEvfZY|9ypYik}31sjX(Gg?sJ13dYP-g=KVclNDj}Mk^PZyy1v}F46--SM*p8t>HIDR@y0& zZpowlpXDD~q;>70XE~(5_zFO&T+(0YvaW*sJ1NTbe`dfH3Ne#d52<}`u=~W4Zl>Hz zahS*0OEfRv#KoREYkRh5WI`ALk;DSVw z<@=#PVa`q45L!3ooh<%SAf_40B@DlEv*6tGMK!Th{@wB>W!(H5MqZTHS24f9{yo1a za@8w4m*R0sLFau~=PwnM_u@HB&9jP-@|@Fqg||OgCj&NsUS#Yide@3tCvcqEkoZdj5ui`OzNvbt&7Pm^;dimD{+4kk8n0nGf1#CJrCA^T$OX#+yd0TN+3a z7e-^wtUZySbPnu84N>r4WWAe(bCC6(^X&<_p$6;$SMxKHI+KP;J-F2yL?)UOZ9ub* zCj>LzW-YxMj3}bOgudr>IV_^ADFw zhLL#KabFRa21CdG3z+64V`31rqx=cVeyj(UiqxN@@_t;LiOcC~8{yYfl{|m<3KX9~ zX-n!tIbUa?1AMaqid{U)3ZX21l_E8*A+AKux39WEt*IK{v90*W?$q}vaV6d^!u0D} zXo9P?mJE@l< zQXcPq%$ENPXD8-wq72Dj-VRA$#l9vN<^gY2-wp7IElWH5Ya*n=pM*51&mtKwq(H7V zVo!#RFei{1ULT3xnp@_k0e5jyyWH5dY#bwUoweTnQbudXiG?v&1- zESX*2NTN%m>nwjSeKJ#14>k`*{S4XvX39y&M9&BxNYcger+?Y3Y=U`V#?Dv@3bez zgTR(dzklQ>b=3ohRa?`DGdL2sT<#q%W%e)g9;u7D!Ne(Hq+z;&FXFp*CoUA$GE+7Y zV=&TGhN0Uzy*O>IT<&NAccTKYVgAXtR+2m)D8BM8w!IEhagyo@2hB7{ijp<;)>-sT zEKKwOJ9b&lA`uM$q8ssMx#ifruJ@1KTB3abog?dNl)dgZBQHvt;&e_6Q1d-cS^1tU zwV4Eo@trri3eEUh?8xH%_M>h`V_RzSbT-TMxhbz#lVPe5XskzJy9KC0a1vX(a_8@=;o-5HV zKc1)M)mKye(Z43TcAI0h)r3=InZdHgzICx|p>K5j(x|SGfA*37!0zj%=b4rQgRfA` zD>q|7C00Nb!JAFLY6q4G|9gt(sx}XNJ|P)=a`1dKEGsME9D~p|bQu z|7fdN9^TirgK6fVu57J4ZF6n6%(gb?NqY@OL5rx}&e2ugXS_%F(w1kn-kiNiIJ|v# z^~#G?(6)g7R>`k0lQM|Y5yucoJ6Kxput6FGZ5&jT^=|WNHxNwe_CSm6p89-Psa4Rp zD7kBg9X_mu2_4@zulvG`mPavMQHm*X_W&2z%{$wXo6MQN9maYXLB1N0G}{5u%#~)6um;>GLeS>)7U+8)i*D@1r(j zH?tXf4bd&j>mv$^itIhcjEpB7SPbVLJG~! zH=5nHM~uOquzT&gh8M-P`tAiScU?Rch4i4bp{ezz_ev>WJ^xsQA$S6;XMK_g<^LPu z??vLthB;N*HAwwIJuZPr?v+rEj^GQdbi$Fzj2e%cr{Uop#>;N@QSG2VlvjR-g67B0 z>8HH$byBbG&gv`FBA1SBC$ZKKf&f9DtD%U?M#wT~PmHNEI%816m)U2bZcZ!kQpfUX zh!mcA*Hmii&@bgP#Cn_?^}RLu-&rNemq9-*h+vQS(Om+tR}2&%7+jgOlRWW~c{KD-C@ zLV6VOlXk7k@eAF$ba>b^zrVfDAhHej=YyCC7oD<*7#ev~haeq$$gO&84o_605p z?hMN{w*R{QTP}8kL{-?M-EnM%>O4|V>@AM;(Ye5{@-*$$M!yTmWF;N%+gyLBoKGcD zBN<9ZRs}Bd`e{*-)KA;Js*4T0MB8%WfHJth;wN%C#{-2Lq5a=Dv`_t5>1$z4hkEY5 zn*!=}gcgn=1*&F{fSYctk|aL9oxR*~o9ujF5RZ#>Y-yTCDSQ zS8?X2J#<4kDJy<0WI4w8+rP-#S@$28c-(m!y)G;zkwygGMS0USvOd4Nac(Z}!{NQs zJP)C{lB-y@2x1gxSoQu3H?i~e>I_1@xbeg<0hV<@e|k2@Xq%^>!b^&s90oRavWl zQhDRENygA%=lxzq$q`Y?ZPCUQnO#kmdQ?oq7LlZKak#{@;|dOua_XXni)gcw`~WEt z_nm>?F8eO&zU%X;8bUY(CN{NI)?ek@Xa8iz?6)%vzuj}n^^RK`Ul#@~qDCQnov;LM!K=BtEr>ANF4op*A1Q=K0b$)uKbSJw ztEG;d(p5mRu4u~PoNR|Ipy=^aUaonTD?Y@JSz*{uw6KJxwwFEnIagnH+i$uu{Q*f9xq=<-gea{MKW4k!QU8_Q#d!fncYQWZ+4E z`OFGi z#)Poo>0|doEn14vijUp zTyNo54_89cwDZ)V2{tvOW;v0|j<=>V1JXM!(bx75D^3JxlMY6t(JUuOv0qL;Q%!>; z(lse&)oN|Pguk3R>=<>_I2zY=@NSNig@@ETz%da z{+2e$7@9k*5JiT?ua;SplK#T5QgxgY8E<-%u&wyb{wj&lsD~9av+#Ccd^2!yM!;dV zca_PRGW!g?V@gH^=;RPfN{OYpVml#jttQe}Lsh+6o+5~OPN-;RC)8?{j#!*4ylsY)ykb!C=A2d@kvV%1XsZ`okw`G2@6*kob z6`;C>^UV4Zr=RfvV*xBs(hRSjBOl$nyekGDIB`Z_|W;V8FrkogV(p0VUJi%@sP_PzGrz6(r~O=Bdtfl zAW%g4$CU3jbnh2d&$-UznhjQsL&{aAI1BuF5uXHIFTmL2lJdl@lkB*2Gr=8XmcPaL#G98f#!l{RDe60~3pj)8bdY?R8L=&$3+70s1= zd&ToQ05PfIdHKi+!D52;hSS3iA?+MkCe-Co89bfz$n0&11hm0aV2&J40d~)kg|os6 zO?S{>7{|{HX%gMf87pb}bK!!Z05oSj@}Q2|1sfY|oAq9(Ypwd6z0zdF zDFDXOOeIWmEm&!H9u=~BMo~sJF7W|jdZ50<6cl~yz4s8jHLyohnKbMI<4(j9J9aX( zskY!ffo-(ls-d@JXLilRuS*^8D`C}H7+5=<|77x%ueEmZfAT@>w8%RdFG(pNw6WdO ziMIyj7WHGQfidwZV>r}hBz$Sd@P`S}nKt~Cvp=wl*Q~e^#zsQDvYl?Bl2I{rxjJ@v zo~A08`Ypr)z5TZX?H2d|O0qB6%@YTn_J$%YipASMAsPHA1%#uYP&cvesey&|L{=AV z?uVn-u9*$j|7NCMVP)ZFfr`5@Gtf(7mSOGdD=d)I`-20i%g&W7_w869vGl>Y0BUa? zA2oMSAIM0!Sw^#B>`*ZCwUb<0xk{y1;Pn*V1&-2{;QHC|8vjkURmMR;Gt-Xrr7cC7 zKO;R>UVxLwM|D``Y7A>qs(K$ZutE4v?-2s;ga3XT+u4&Kf%UROL6_b}`giviBns{d z&z7ecU)Z~aPH`H!SvH6-z$!L=Lj@c+5^p|@)t>%rcM(2m3wkyovV_%`P`orI_k+sK z`N>x$xJkC(#vzw=KV|jdV31|ru)sE|0K)W^#qsb3 z4tc;zqYd_ldz0ZC&^>QhY2{d9UaJB}ywa31u{^ z93vLQK{?i~fRC$T$2|N^*&ljy>#_+1AAQYbI-?YZcYc1${3BTu2JddV#&KPgZl%rw z`>F1EgpzWXsvmuMi7R9Tt9q*3?ihf1kGW}IX#`RmZtsmhCZ4nxFkmZi4HX#8Xq*XE zfF`>;Y<1@{cILlN5K@6BDw%$QOea}`>J0mwu&$I6u;?06n9VzpG;f*IZ!L8sV<#Gz zseS>a1g%@OUeM8;tL>23go*LP2K&4fUn2t@^sK=%3J zxtts z2sbpY@4|$OYsv>qVP=xO@*X*=8PS=kJlhKLN*3*byCuvyA}7Kw;v=np%f1zztPm&N z>8fW?%h`nv76p1ZQZh}k7sOC2Sn7A)iB^duey69tN@)2=xCXHe8|lgKsff>|$caY^ z`$0lFZhz0{45)SfR=w?N1XJfTG)-aCL+3l0u+r34f@^A@_ytY{ukE?8iKhg!wZd?o zFo8Zc)Z9gVocZAGZ8ad!6M!Xn=UvJe>RN6=w@m`nyP~TLcu^goQgw}?d z*3Pym{h0S_rFvVy=D-ffLWAo6WRDJ(geO|e6BJd zqK7j{qGvSDjIq~YiLkNG(b$9GsH~bBJOi_Kte4M{@wT#F_AII?xK#BEXy8OSL$kT# z6(hRwrE84;!=t$gw&kIXqS9<=DEbfsxAoB(_JcWNiXYR;l&J&;#)#UVz2R(j2j^Rx zI&?IpU9bx1bf-<)6b08iJ#Xy;CAxD0HgsFK%Z2!xzF{vv#gUHVsvWltc*3gGubaPb zzwrzSLKO;jdB2!DBi!WO3l{$Amj`Sj#eYw+a|UMcPqP6=9no758yN^qezN`kcd_9I zTjricmx9U-;5K#uO~VzZ?khO+YTk^%q3Ubu#q!bPZf-`CMBfjkt8Ehcd76zc+9=vm zto;*;XQy?kzs*itqvF3ju6UR2KA)VSLE=&W&h|z-%wCzg^@~m7a2c_bdh_V>O^S;l zXUc$1T&yB!*zc9&OgO4fJn=UgojJ1WJG%S$RP%T5_MPhybncvVZrFqw_$S&X-gi>+jSLAag98)c=XmCRiO6WOOF}%gw3wxdCN0QxN zJkJh90j`R1#G`K}B;!ln50skvs!lf;5C^9tFRbOg7@2xZK#)ozGmnLjUY)Lovq$o8 zb_C54w!UOdx{d17;~B>A8Gdouk@M`DmWG`Ep}H^XoY|Oj%wzk3^ag>g^!gwA#pVio zy;Gth+%m*rY)+XR1*5uT`tB~zg})tBTeiaf z*?IOXI@@pWiXt~mk_#G(J$dv3v~~v=5))@E8AO)tAIe~Dd?k9ParMwAuIsKY(U)?J zO@TG9v>YglB2%_iKN7m&N=?J$zg%rXqCak?WlZ;vZq}4V7LPj>lQtd#ix(!uoO;&8)U?3y4b`T%s>kvhO=KT=RtuF;&J3 zee*w6oa}Z**8TlGk-G1k-t~v2`Oca3*GnQ%)?$`|&QEe5F{vq{p<*z0)-SvOM>QSN zOFzmC-M6fM0~Z#oE8BCCLHm1L*Yr9TI;)LDmjS`&6HcX9PYtzIpOx~7#T2WJ(Eq-? z?hJgQlwC|U9!zv4P~12T*K!H`w4dB$kAC!wd$3ba;)kM=x1>biS=AMboz2{@RrNi< zLN~>#VYl_@1uy8+OPX}Gx{SnOcg~wkRcUe2_6NfU$*s*DdlZ+Cv>e{H!xD#IH|MIX z93##Yo@HdBdY~Hywo^~n@Qe~;9)hCYMBd!mLI;DMDzX&^GBka@`7M}U@Ae6zX7ytk zuYsixCK&jl(rBLa*N-`a519jMa(LFkHy_FX$&Cwct=CWgwBa>r_$s0sm39s5O=cI5 z!Vu4i`ut{qO^ej4JN0pPT8Nvy6X#?Z1ZZ)lFWPY4pa}}84Bcwx1qCm^i>%!IiZWuE z&)$rp%zY@M^T_;nk@?|fM+(w(yIN^?I;)cA#OBN9pD<>lgG-aEN3L|GZ>mytPpXon zbz{^Ij9QJV@!_(p2?D)1h|OVSwGegfjT6A4$dZ zh1-VXt8EsCk^rZxq!wGW+Mbk<{~6f(pS=01XTt=T?fLk^KE6IJ`C(hhiYU0Yx~GQC z39VFCXWUzJQ#DAyaSu%Cm^!*+Q98EuO{}hxxI9&$_nB|=4zk3|O(`^XlB>PaP;-fM zOa;7{R39-e9M^Cj= z@Qlfskty(*iT}k?ZIU!Nxu$-l>H2t}QEZtO)6k8K8pGzo$29FKbz zjHavm{4d91m3m;|qR}f9l^hq9@jS+fA`xwR$a=RUi&DE4_sT_aSRb1m>W(WEGPgr0R1@8X4$@GE4KF4Deqi74;hHT_;f z5z1Wu`vuOq_IaCrw?fABi_O-JZY=_%XeZSkgVzFu*s^gf(+) zqoF2~*6;u&Ax}?u<42&hZmycOHg6ZVI{qGzn(&=;{pYGKN8316<5lpuad%lpM3Ag?QcVS4vvc# zvk~4LQNvCdga>w_-UBEeG&ff5lCm3j{J~Mkv>KbU_gK%E()wu^cI?H*8!;?@=jUm; zZG4|+;%)+jLINur_C=*OWv^ICTc0~>priz=a(U^kB?=U98-z!W-@vZh?Br(nHT~ge z<1~?FGR<)v7bo7dcJt^RHIe5-TKS0rQgyhgCRYJQ@sORLCe~{Czujr2j?^_n6qSCD z1_Y)ozm^YYaJm}$dU9U47cqTC{3|nxT&x^Ra#s<6v&Q=q$hYL+?S;snq~U;zoFJ+noFPVp;&M1&YV${m38DhU1z271-{ zWo!8;IQ6irZqipN?TL(1wx5ui(Ds*$gKXJ?%rlNmF2|5yV_Kne++~O11S>tX zrPDb(siCEi{Kw)2CYG#LnOa}3&n6#JhlBm)W|N#hg7WtzYKZZ@@eA>>da%E*=g3jW{WOAvxoKl3D@btV~QV zA58-r$;Mss@G=a)ZP&qq;o|J84S|7YNM9Vud0{Cc(0JyS)O=(58O~>Nn1>B0z!8>1O+!qD*+$2G!)GaHf5A%_q==?BTi=63 z8HXjLR<>k%0xKkluNe9fe&$UwijUvXG!?hF3ADZ>!r0^Q-xIy-^9X98RslrCZ~O|! zUx<|(dmYC?F}^MYcfgw$N^R1Zo$*26Lre4gJsO4~4j;F_gcYT8tx4q`CqKeyc)HRC zci>i-c;Y4-cuIy!AS@xo6za06Wo(N9#oerS2{!lbepE^=sRl_KMOpHd>L z29UY4Q!(Dcw$~+&CNQWxk%a?WJRYI6zhMCrZJ@UeeBt85fWf#|<`pg={pba;nYbQ0wH%(Y z3o`U+bvVjG9c7fJxXa!a9VzDoBb)n^5nf0=QZ4@II&>(7oR=9FJd8uzQqh zCyH_Pc*Kp^dQ;FFcs=zb+@HftxR? z1czg*rI#=3sP*NBNrW`1&n}I@1L1oiIw{pl%5YL8K1`bte^Nw_VHoJ?rCEqnr2Zjr0RjHMJRGsQxk4ZjTvF~J7m^h?P*Xdo^TQ%wH@ACEDf8soHa=X^tWD9 zw=AQZeK8Kh;=Eh!#p8KvA4o8O+f)buShuqvWg>#R0VJ+@f`Pjm9yxawy=2#l?W?V` z9~S91Q&yj^7-B=0(_l&WWzptnH&pw~e7qD6T+7?1nz>d3*408~row8=YX@XM-n*5m(0W(Ly zy(=96-TOerhdDeW0KIZ+nchyZj205?_U^+xT9# z3tX$;HvdqLa`3XIfQTA^oG%Jqs@0Hc7~q7@`~oeFBMX&u{io7aY$IDi05sX%Z+?giW3 z7$aB&0TxJ(UFBF7MDr~9>7UDkU@QG=SqlPd9tc!(1eSR8N$XAtrReS>QqiRYV%SgO zk}~W3X0nWKODB|k@h?}FsGP?8JmJkR_%%b?JqdqbcddMzfcF}jd+FWfc5_g5{!S~^ zH;YxZ_(NTF2j{|MKZoexS~S6Tv<;hb7@#5}kJpQwx#JM87SA18`Fa;UC`WL-ra+Lf zy>mNE_iTMj3rVD04a@ya_y*F>&efk+`^47`LI~dTezV>0}ly$@lFB3Q_bs&a}w77Owb_b<9f!e)#HqNFC);gP^+j&tq=%sM>3>`G$@_zP@ zADq2UwKH{dkAU)QCrT0DV!U4GFCquhG+jOeQ5^KIalF`#Gu!S7se>V+6uw#)>Gmp6 zAr3cB+F226ZM;0_|9~5Z8mITlIpAO=+iZbJLNEl4ow#1$x*AgJI7PJ?2X(yZ%jU9b z>&;HsJ?Lz7LG7Ynu^8KP$?gV=gn=MKWJZUrjA)N^4k8X+hj<^P<|p?+03-!+n=p3m zeOG}TY^Y)5{AxFR#7u@h+tMA;_0i-1tz8>CWVo^jNOlwU>U`T3c8hna@(p=5X)Davs<4Aqkx8)I3TqmEG6E-h?`eSjJ7IzuoX{vl zy1@wDjm-z$g?OW&%c7{;(Tf{U+@6;f3rM zfyi@)OQ!Q`=xnsQngVvcBSIKEPYy)n4S>Ssm&I8!-eV({k{Z{T(sP>WTP{yY{obfb zm3ao;;_H9roKJjo`vTV!NIll^d&U*WeU64{7Tuw;{6&N2RN22g3ErjKF}Vnh>Ka&x zr`-&J>%y#W>3JuPJz&n)xW+!->emk;lY&*$QR5cThk!tOftsIBG^;+}Br>!`&JwUydRplut`vE~i+?w2kc;Jb6 zn~V{!>!NJLxf6>sRly*J7ImCMQDtp%ij;l;VIeC zb@cTVFeKv`zP4w4VB!P4{Rn-%16UsEtPE(gYEThH`t6b-{OQpp$BXrI%E4-_Jo0;( zuF1;7Im5H#N3~HX+OM$t$Hy2W&mjPpGif7##v8_V(hO$fYTy_MXt1a*(eZ6-VL@byx3n zvfUHk+oYy_eJV!&`?M%pC-B=s8`t_8kuiOMa2$xrn2J&&{llC%H%J&l z%(4{+5+uPIZYASI#naAv`}m+p2aM+_MW zuZ(L!-^krX1EICt9YUTp!Fh3(`lv4=GV0KpZk_zTvGm$^@`*(9?7g#*We_ptvTp)u z!+m*X)oItuW#D)6G@vRexHa7X4}C@qB+93WJ^1{}H`#uRZzh==Gal!9P*%pcCa(gT zDC9A84pAzJDPgfC^6zZQ4Ltpf8d`~8a_6j3d}1-^=I{G_uL8?W6zPopPGK!O(-t|d zg#p2QAT$_BKqm9i>L!NuS($YyRJDXaGTpu5TMxrbs(J*fv)`BM!YI?rYLC}Nb$=E{ zsQF$HX1>(4{={)37fCxjU(mid%9Gt4Y3l8&dHA!Z8aO{p-RK<52PM@DE=9i>SDt;A zEYcQr(4I6wC@S>q=_Xs zoi8wW-n41vkj?)2;Eb`P307GAe3iPGLNVT>v$M8>;de~CH#zLVk1eWPsp{bRM=w{V zdVWd!P0R)lVvI7R>O4Ph;mLjsZ5qpK)U)tJ0-x&(Gl`LONtWjL=g50kWa7J#h&3fH zk(npDc48mj`VJ*1c!A8iu&5QCWRfVA?ToGzqlKM6Of!wqRuS6tG@^Eb(AWgV$tOy* z(e!dESLg{tcf)Py;^qm#oL`K2pgxTAtL3If8uMt=7!L%e2Jt2dHGxa4e4@~A0vCN9 z1o4@{+Yo9L>nEb?AkN9Ha9^1Hkh>!PiI#K0j$gvpYF!S-gSKNErpde^_t>|c@ZAWM zfB(p~6sqHqVzH#R3)%FFpEAzNC2X)8CoPE?H4RFL$ghjbi=S`7Wga6j`Bu$85wD4B zh?j^Bt}Eb!9l=}ZvuWJ6cy|Nlcs;wowsi}+p{6e&mST1JV!2Gfv8Jf8VVg&D-yJEv zUaas^wS-ibtHnyNM*;7Gu%;_PWlIk187zaL=GClFdQ>nq1b+PcT550LNR2zl5WvGgXw;yu#Twzv=IJ^(O%P%Es42;S55+>l;=&x~pcQgF`Z`KJ zk?b4I1t86h=#G*TR6Zr(d1?Jb@mVrkx$ZPncx?6{>l53F&|VDgT&2~9grnj~mZRm; z9e;yRkks5jm05uwc*df9D#UM`PAoG+peesPoT*lDiR?{~a}~O)`QA0xL0iJ-@0*{$ zpChAIhR)+OvOwS=U3- z4#8}nPLMmyo(2~c7aCB?!*701YCm{mcndd?VjqSRfyX*l{0mQ^iaYaz9O?^0 zi&sZD(iysT#4BOt(H=D$54q<99F4#gN~&Mz85~&ZSM6J=iGM;@L$z;q7oi^qtnut< zqjG+%%ljisbe#b^`guJ3ehr0t0uD?h8Noo-?f8roIhDSCfF%0)*!-OCE8<69@`g;7 zXgvG$0L1#V3QD8ZaA#+vB*-#97!>@Rg6p<~_tDVzA)EV!-Zwj|f;*odDgZdZ{cBHM zHlP&JC(Uz5_i%LguI@^Ttq^8<9atK?6}}8>@R(iq217eWCbgE+t9m&1WmAT?tvi{H z&6Sc~>Bu$_c=mJl^8OmDeF4cutw zpvhG}Wo~o);)i;MqOWKfR36fLLME(Qk^lNbBqn$#v}3tf&^hixXnKI~(e4-I&}Fr* zA7WvWn3qyqt|>9ptw>I3iB1<()%9y7@aK6e2a@8g8E5;c;2_%w0^&bFYRwvpkT!n7 zHgbQ(x|LMBdSZ*|enhF)W^VjJwN8_aTj3#$ZlUenh`j!x-(jwI4<<(`AD_wP7IFn{ z_%*zPOQ}QU`~h+pZWJtBb`elkOTcPDa)zSWQbXPiVuf)srJ-z)az8gP)UODYAVMy} zn^|`epFXT$_^r;iSW*XcgG2}YuV?y0f()7lV2hj^=r)zt46d@l#;6CufT%!Ez2sB4 z&U?=JOSQsdQEcYgF?lxzwmzd160onZZ12irCo)gm^%?TZrf9)wn1G42v$!N>b4Pdk zm9ySPD$SEqg_ZR6?>b!ypu|rxUJPEPy8<4qBoU?h7qop*s?_x`ixYiYF4wSD`>&wy zL5j-YgJHNKx_QDV^Bk)BEWXD-$w(`tc^L$wxpFESHAEF>M#|R)Qq(M7B37yFQsz+c z_KnG-Opl1c&~iC=6VWcM9`!ETl;BceU&ques?r?-sCf~s!3TOdvc?52gmSnf@kOn# z-lsDBSrHg$Om;y8jL-UCtsURrF0hWD8ny5xV$g4hk9%JNIV0to^v11=cg@|nSrR;eYF3LU5T)?0Z4fi= ztw@|V$R@ykuvVzqy>QjM68aqjGaNR9NUms&Yi|(BU>qeIld4ulm^|nFfZ|!N_&3ma&z9->sJbrmucV12pu(f5igL)I_+J7YB{X6aCz&1lq>AA2mMKa@`r-=FuiJI$*lsT(ghE(hF6 zKKs(dRS?3BmGtsNVPMu82`Lw<@3am`YPyC@;2Oy9zjb*7c@&dkn>4e3Z*f;U1kiqx z&D`zLdwzfhY&Dh+v1*9|k4U$mPqftw$4w&Aoh&SdF<|8DwM6;|)7%4Dh1XJtNP(~X zQi&hQTblgC%dHdEPcM%w@?JS0Lk}%s1g}HS_LoN>h97qkH=4A2P1-miiY-OIb&<{e zyb^Fb=uT@_{5gT>M{}fJt~`W6V(lzBFmg&ML1~*8DR&k3mBP~x^#=J$Idz#BJE%yj z$iBa@_UT5=ogZE(Nf@u4QFNv1C;FDh)1OyrZLyI{L$oCm^{1LW=YjzT-qtlZTyos- zGR#au?|!a*C92qRYVfDGx|k(-W=|M-ov2Q;JS=$1j@`Y{Cw-%3gY}3XnhlnmK@>m;fX~j_V z@$X+0`%#m%TLC7s*X0NEu7zz7Ap?6(nCWF?TG{NV$gq^Sc-rb4Ag(|))4JP&cbz-d zAif>$>|l#}u|m)gkpjBkD!JY1_>dEud)DLspnyOP|Bx>BTc|wXl978xyR4*ZCk>9` zZ=)X6o#O~Qzs#`W#2#O;xOk3XV5Bu}TN6b7%Iz4gRRI*g5(yKu)+i|QJit~@QTRq* z&{MoVXn`Q;S5NnZ5Yn!nO-W-oqo`0hSz&3ljlb$T7MU?6(7XcIg)1%|{~{P#G#BHc zD;px>#7O~ORwGiv?xflQ%{sYAU&)&=odv|{G)AzhNaoxhF<#7R_u>huAGZB_sXO&_ zI-${eI(m1n{l2WNo08vBO!K{z&B|BYy<(EyU2I3@QowMKq4Upk9PO#cN8SRma?Kr* zgEo)xg_%cbC$`C)cyl_-SE?vOD>LF??*0XEL%C(93^3K;SVwJdwNh6o=+c`!BU9{V zzO)0B?+5!L#mPOFDcLxXzuC1+11EN6$4u!2bou~%8Sc0xbW49sAJ)A?g53D7fqYVR zSdgF$S(;kut3K|?}+L+$QFJkp};k-%O z?cH6ZF42%a1waL?|D=$y`>4Fo8U2cq} zlmUx8oeHs>u;RB?DK6x_^@thACdPMEcvRq^G3M&mo%zIqDFEX-HXp6Y^fAs>eMO?r zG@jCT$Cfc@#bkss5Rb-Yu3Clh62ODAp8jFc@fs~! zSbcP6No?&sATx?EY|3?|`qgQko_YU8>0JFmp^=g)`ruFSROIjc)ZdOq2vl!LDpL2E z8y~^*GK81`I^3}6Uokt1%ZqTkI7Crc>@@s#ra1S;+Fo&Llt; zdYDJ_^6Qg6+09rx6QfaocVBVYrU>nNKW3I^qbzhLQI&-qH)`shgV)(dSW6h;PTKCW zHDkI9nql#eY0mAK_igKsnUbQ5h|F-xi1fcrJR;}?dc=+b{I6{U6|X9uhR zOAr$A_IFMRK1+5laHS=)?|Hg{yM!5sBjrk1b_It^aQps=fi|;93GarpL)SMWz6Y<} zmbcFRl!sdXs*?6=lAn=$a^Dm@;jlIf{931!QL#~uWMPY;hFj&xw-(h&Iar@^)fNA! z7wRj@N1W*7Ge~fL*Jdpf(Pe_=@9jD`QGPU=8tW@T{zv-JU~#pJZbvs3eh?O}rwYE3 zQHURVv*%R~{m#acDUlnEerz$XkI*LL_JW(%xNNQt>65lw0hN7xT8ZbEMHKqNv&IV@ zY2InPon-RFZEa0Tz8^W0(%JXY1?v0?-;MazgHt!x9f5q;gipCAf=XPWg~7W7zG?T( zbzb1NFjLOk%5TP6!AG(e`dW~HSiyB4fKlW3EDlv5kAD4s_VJxZE3pnmrv^P6PzwZ4 zB&-GH?$exeLvLUv+xF#zTeg%AQG}|LOV$Y-Gbi3BgUr}L<&}}RxPDi0I=QR*BDV$d zRy+S!QM9^>P&)ma|4b4zc|ZGG+{A4-B8J22Lw&BX)>2*E(9WaFpxKKQ*Q|GFZN>*t zv`ywH2g+_y7VNmko?~k0%O=UtIhVfKIfxp0A2?;m-Tc*~C=I^nA%8T(qdpKR1~uO+ z3RV}q;_$+URiNA*1PzB}`0Vz6vYk>!h#8OEg5#^4!&~K;yjU~iEMN49%MYg=P&MIt zt}e43AHpoV3WnVE4DLp04E)#R`y(~Avj`Is4`B~VfEdPIO`!Mc&rp8r(P#(YGA6&3cw>1+-4wiTZfYyVF%mA^?P~- z#qBShVtdZ&dPAE|I*i0SCP`j9%Jx%Xks$aslB^g@NEFHRvBMmh3?X12YhI}3q+?n#vcsl=W?upOvrOp-K(uV3^hwUSMZQFIemw;yyx9&& z&Uu(;^7(_{Nn1EWkn`=X0g*$(%r|K54)%Q*kN&s++AwyZl*hGr;@0{lC|3ZZ&FNVI zV%K?QpSNsa>aWf`m^i5Ys7CZyA~Mj@+CkHj-dw=G1s?s&e+`JGlv8nJ9h}sM>)CXS z0pQBUw)6A9-XJRxxTb_R;41z8K0& z+8^GOAmPUcrJZ^ZTTt(j0>xNHa+e>CiWq%p#r3-Cyohga`3;_OowLh&CfOR};SsU@ zc}D=*NEcjs4YsmUui%OrT^GGFIP#6sX!4rmdsA}_tMpFI$IKbF)*XC zzMGzY_Z!kYruv6UF_4KM^ft$?4l17vfm4!BVk;!-bIS_uc$s+vNC%rT)vXp18}=^7sq@x{vE-H%X4%|*~LVPA53#( zKZjzf*0$i|Py|CgLApRS;@e?*!PD_y9!UIuB+dJE$$fgq?iH_>oaWkP*`s2Z)CM@8 zC;0>k055oKS_8G1wK3Ehf9J@nnfl%+(}5~h#LJIvgpP%ps5wr)RJV@B{F72bF=fL~ zqkR|-{q=XEue0fYQ%FxI`jiJNQgXyv23m;W+Q>H}AXm1i+POW-bUXb+1%KS-_uVb( zKNH5O>=qMs4(s5KiZxe>8jFQKmZ?FB3RjRHVI6~b+2;@b7IJlDUIV=Zz;AQn6Kv~$ z%FwaE;D1)YSwF1WU{7sCw5gI&yi;F+g(!~3li~i_AlfVCaqT|N#;{OelKel*- z{vIrU9)RfC-^BALbK(6BfIAO35U;Qx*O@#z2Ar`I&HhsT=f%-=3>uN%SBRDGgS J^VlNfe*mJ{2nYZG diff --git a/docs/logo-text-padding.png b/docs/logo-text-padding.png index ebe53a285856a1e3b302b1ee3cc106582923a07b..f99fdd3c5d3fa28f6af5a300d630fb267c357d3f 100755 GIT binary patch literal 25396 zcmdqI^vjDH-(R>rZ)1Br&*Pl)xS#upN0`P-g)3Aysm`4{cST9@xz@RJ z7jVGyA;m@D?*mCV&M7^ArsDLloWemJIhP|_y^m$0G9J);up^U^A~Iy7eEwyH{C$Ge_!d( zqHLHO7yfsho3t{B5yZ7T&|hEte=-F0UH{)z2H&3pp)RJ$F#bi(e?L;Vwf#-}e|?O& zNJfhj#gP^KcU%AYr3~WX#s572_eG*??`2A#@o13$M+^V`3MftFe^<#nf1b?yig*Cu z|LTtx_v!!X^8c+Oa|O_Z9fnfpf0g%M#^&<>u3{^5E&vkr5<&Gp$^%Sb@jpiU|FH=i zxVDZZ5S zyCxg(njx{zvuOZwx|29zXLe-t=g>Uf88@u79>X8^;YaZ&SUh!s(m7D4k4nE%crG1)%O!YrK zcw-bhTs;q~p`(&;`DBPHZ9?1Q}SKfzyzd$4T%E{~3t@ZS?gLIhCQiWUd zL9bKISf$Ox`AfGk*UDY|6Z1|np;A?)Zwxd32$wuLV0Gi{l+^4reY_;L(IN_OISS3Ey>DE$j5 zMuDZgc|W4LOw?{z_28_a9lD%mvz+Ss52B~}fyf#UChG>X!KL1Z}2&Ttiw?-P5 zL{Z=g7m>t|@pi~yPi}L-Y<43iuAfnBrucFF^?n6b2eYH`5$}UIOVnw9l;7aBRnJLh zh0>z4!_u>gp+fzMOHx|_z9TF6^uxhiZ=sE#6<&2{_{nPNX~D_H^r?gTb66}BWz+U& zr_n;QlT|Fo_gCJhyXj8pfP29f6O}0T9_-aI{SyUqxY!CwojmE2u7YQC%*}7m&^}jU zcPYZ|8`>WR~tyD_ODLnFe!S zaci~ui0)+HvlG&Up-q~Vu-|sX#%f_HR-at{^I5z9aXVIOdv(7e!Sm!{1E==n;jys&yah%apIrZ- z^kIJ#k?ag%*GP7l*nGSQAM$Jabc4v>yu(in5%$UOG+vCCI5ayMG9v_gzwzGdSAZSO zR{NVCEnVu?xk0~FJL#g5@onhh+ItzA4Z59Y|McCI_hgmCaCGWZ*L5tO0|b_cG8x|c z(Xk!`GBrP@Ih~>L*2Gr2VV+$Y%g=?6OK=N5ik<+r>c3Y+pg(NC?fMC|;o!8kk&UJY zO@RgK{;~WyvQSu1#{dV+=hiCLSae6^vE7*9Vpz*Nr zD@^^GOQHY#t?7?4=%Ds#_J0Hkgdv$0l^c0%!_j7g!^-dMBZTDq4K(f!0YDd|iAIAb zezP9OFOwQvUxW+iI7?|hNTJv>@wWVLO!_~K2fUzDm7w4+wu)B-Rxh59y5i~D@jT<3p0RF)!(0cTw6SZxEGX<`!E zrwzN8Beqf4S7cb}4eJy};xfy1kMU=xPDeABq~9c(`9X!xr50_EkN6F!iLqnk#1ka{ z=>`}9P(C^GIia?RAZXm=Q|~ZQQ+#Rp5B{9|6h0301t6)Cxp9X4=)5*9veb`Ih?*bY z!!z>h?GY>fpv4P>n4Q;9ZB5jM2+S>N%mUWySC?ysR`5_O#uxo!J;PtHNxpOc*ZRy; zg2Oe@pudIsWHq*GnZ$EvSW=VC2n6PURs*a23yy@=#x0~$S^V~AvGp5x|Fa_)=yV|c zv!xLLDRkH|q9tr0JC)~W7i?m6W?BGp~Dx}C)X?u?Y!e4<_Dd8awYZQ7SaC|Z*6+^o_^ zt?GhF#GtF<6ajr0(b#*5Y-e3<-qbsdP2%POW8Dc!ck0ird9(1JUi}L-3J{{pQ$|GO zb5xr7Y_}45{C9YcXuFk9ZTyKgsE&2%{U0>mbkk_bT~*j{C47kbGO10h$=M7dfHS?B zB<*Bp{QRJ7o!d;BSgHMYkO%ABBnk&6Eaj=H9Q7hkW6!JZ@ zg38zK&!}#tXNxa=B!sq1SuRxF`R}rb6tV&)+#mE@A2xQHV%}F9q>#|*8x9D`^`(Smk_B*xjP|*$BXcQU&_du9u0C0hmoN>{t6y4N0d5(7wQ}{y$Nic&=Ny#A5IdOCtF-1JSaA)VK`RE?Vvim2n#Ldj2)kGjDR5ZgsY?gYvI- z;--(5IqOi6GG9*f#5S$6`tJ1;t4n>0Yo#Y+tJ;sd%}y)Qe0DmCnarrX+Qj3JcOyqj zKeC=~g%S_Y{@4V7x=c^IO))p`%S-HZ+$9~QkR2}drsR@EYWo>ZnC<^$_AWr0VR?>K zdCFq>*x%n5jx^*jHNO+#?fk*yZ-OA{ue(bfsF)pUHOxm!*aM-Z#J1blcWt$f+Xbr0_nU{-G4+9a*i zu_ooZ+~+>z{^TJD#n4n1!3xFL?;gZDTSI4}+fduJ1emPpXbL~hdeYB27Am!u;rrQp zJHpJqcFH3KjEACKMRgLkQyh_qoG95c)qg~K zy?Y3&B^+TadalM?_dkC0KC?9v^>PFD#>Rj4S)ecJVfV{}M69NGH$$<39KodV_PifPkt#O5=k_>N@P5;wn`lCOV-T;Vk&B>o2mFC*_ z^lmo>bw)5dtM)uS09@v+y8q$8My=mkt^E|z-NJN@@%-r>yjQ62PHg(5eomhXfY8&@ z@>LCdC%ZicmP;7Y-)|vC!M;86j>jK#GHF}nI32Z!+9=EmtbWUSSicOPh{zNAA!VyxCxmea zpldxo8@FEZuG;}3wCdt&R&5)9@>}@iOD`Ne5h=c!$9uo(?BpBkNgir94gcs)wFdx& zV_`h-CtOhi8)g7g+X%SCQYP#(OHIsI_wD+Vo@{h~JIZrx29Rc=a+sgK(PZyZlnwO? zkS-k(6ov*}K(Cr10vT}v;!aP`Tn>1jqc$0OnfxEtA( zc%PMFIf#@bYL@^860Ef;1xKQ|%2X^Zj>YIHl`GHm$i;Bk(RRFeJVP{Esy0*A2@RL4iSO0N)2#T`_ z&Sm;+k8{+5>rSV~Tz$Fnx7&a7`*WZnphl56BcKW09e*OU;bM_krt`g4C}&jG6n-f} zcwur+D3WXZ@?ZCxLob71UVm!n>(tOV(|RG1Zo!`K*H>9ndvx8^7>Hk-*5At=8sz86 zkg*lSlx|zdFqg!Hg;3$sxBj5=;KP82y)PS(4RFd&f`1bV@f0X)L^geS)94pJws|Kq z;)SK|mKddL*j~ZFFPdaV>nhir!lonWg_AOTnLujG=ai0%` zJV^&OdTJ?0|A_Y#JOJmmWW0Ml)ubd5qW4+U6Cg%d^wo!9E+@v#-JF8=Za>Hs9;Zne zacW<$AEF>#H>RRs(a#~(|D9(lssUvpnI|8fs5^*`+?W@yP;F)%zrpj)^jtE8Nl@~3 z{r7cp>ceKjG_TetKbfpt+>Nf_zgf&2(EGB606&$sVQ)1vMAk%ThGd_w7rlH%(r`G+ z$kL-H^N~5KNc2t&d?e`jFF_Pw^Iyr%lN~-9><-_xn0{5`rDnqX&aeYNWloI$OrT$! zzuyP9A$G-Y3$7W(X_Pf(!UpJ8V!fu`ssByOuf&0BmWW$d_--ryVM_CZzM0r+lR%T7 zCLmP)%Ba=Jj90nIOgE8^1C@U_%=@ot7$Oa@&)vi7Pg@ZyzlC1^n@W@=gi3L^1W0dB zdi~h3RGMy*+S}CLW&a)FJkR&H7qdwL($fx9%pHGP>oWd0!PgTS>ZFe{GC*lvYQO1HUCId2R$o}rsydVbeYy!m?dz}fHhypYzC z#lOGo97qHxNnN^!J$_cXaqHNC@v>uQ_UYQiQViCm-ga$rr-ZvcW$g*_LBwsEjr&KV zE@z;2z%Q}Kl$F1hJmRG`KK#vU+<@6ncUkVOa+vC+c5C^qEw(aUG{3?^ITvJ2Q~{{H z*<@QmXjearAEmqDP()59hbbwtTt4Fss{ZP~cu)b)zX~E)6u{%*;K!buCMi#Mmswj@ z3xD4EcW(7TruOvdoLK4x>lLfk`l6*2w6LF!!JE%Ca0_A}x%FXjx?Gep(J5o#jAIwqq?z9COBG36waT23*R=Pn!dAaP>?#*}B;;GuQ^w%sXQ zv&{be+(6oOsb}RRfe=mxfC(GpdCJ{tyAdK*n_UT?9R&a*%9G*W3{-&_odZ1Og!$Ml(Q)RqBs&Ipuf9}wlTj=EwCtq_(isdMeffuI&zssNTr@5s@Z4E5+LIN+?vPeTZiW01b&|z zg&r}u&2iAXVYS7mo?XdL?tT31Le%wsYq?!RB z3d}R>Db7mp3L{y1te5M9hZlbpMg7Q>kFakIXEYe@meMex35cYv!(~R;|M~z@sQNVH zIvc`F#7c8gxP{52k$WpjKl<^ov=8Jf-e%ZywYS$&k-e22qXg6q_G7rD`VC&BGM5iV ztzr$9L)veMH)&b6MzRlf?*&DcPc&imoyZGP{B!WhIm-#x6YnMR#j6Fct&%wx zXqRd*Qw&DPge&0zmK>I-S?ZYcIE?afbk-blzo?j5<*Zu2Iu&qFFW5Xu`=(0zRGhIQZCBsygmz8+n6P~=JoRmJXST@0A&gVb?@z?ioIvTuWRCn*! zi=T!8lUH3Yo(R!bH*b_|QtfcRE{T$hjg=}JZ*P9@?R2k9>hr}V1VOJ+U^X&AB9XD%#O$6@g z4EFFPhyR%COhmZb6v^VOePnLntb#SV)IsT|&&eU7+QW+UVA*1Z>xCahRjiD%COdN< zP=l-=jc6m~f5 zhR1FdHD>ziSPC=}a)<(7?j12{c@!oeEJLwa~${XoHBhj z8GgH4PbUjT`o=u9I&(u>cR}}&g}msd?9XIu2qfn-?P&UYb6Z87EY<1#HZd(G^)cGK zD*-}Odzbxm+xb170r~jld*d9z-A8*-o;GAiPwBegdNL#rw74S3Dwo!uzJX@rQG;hh zQXlt7a|Fdq5nO!9et6beLGXcu%d_(NzMOeOOFFAEN2jK~+K-4M-emuKu{oDM#mcXw zy7s*_p?!jW_=HcGHK>BDo(14o7zsX!;qBil!h-Gtd}N1rJSY1lT$9KGYOdibWJ;BF zJmeAjt(7ZiOCB<$*nJba2>Te7T1U z;GYk3Ckzp6QZ}$jo}`mANNc8Y3~b++{)~z|aNv>$kP>fGJIRD4KeuB{-Ho_~G1Mv7 zCfjrTat|F09p0}PeoN+^`+@GWnkr%qWa_1jzKi6<(5)QM*aK3GcyGEp2-_aSKarkC zK55V8-AWQzG#k6_NY}^8GW(Qs(5yxPD;rdcu1Mx9gE!JicCM|K;I?g@w7(FOgw2{7 zbpvL?D@wa`i{G5oT?Lu4v2CY@xlrzD<`>CKms^CUd9UUxIVTib{O;0xLdF4bO6jW) zha@xYK;UaDDPm4-*WR{Nj(Z1eR>PUgqxm@M$SQak%YR4lnUPQ%E@vlTcfx{h1Lv10 z!v$9{R`;mPqXQIZ{MHakslCK~J0x8UqU~$4yV*U>j?o1Lm8==p*i#w_2=Mz+(vJ-F zzi7|~_?LQi!7Cf_VbLX~#FD-d2L8PDgd7G7zu#6h5ClxHywHhIZ^uj*BhpgO#PT@Z zjPOh~G*5e0P-SPh}*CK@IPc+awDj?;q>V3Yu|s z7n{y$f@&0hv$&Ffi+$%Ht0ke{JicC5-)gsJYT2LGsOKnO-Mo{vwQ(N16H(hV8gJ^? zA4)Hm75asPmFy=)a9v4gyd{$yB3H6}?oSm#_@)RYU;uzdO?8XNdnYbC7%hw<-R-Jc zBOgn~7Bk^kUhB6M2tMP9*;KNb((yspo0MCLlCT5X%{tKPefAMgK@A2atnVB$G!Wk_ z{qWckVuu=;;ArGtbg@Mr{LWVCX8avfXjr|Ws4&ozMW)qK|B)TDxa7Cf$#bcG5O!|% z%Swhk_A#LGD155{2dW!n>b9+%EZ%0s4{Y86FJc1`8M*cezosB5{KfGU0c1e(B}Hga zd8TI-mbul+N$2=@Y=U~zY}NyuDin9YEY%JT|%`oA-*_^>uQI6s|HO~<{JFe zlofgeZEfx%2%WQZk5Oreyt8{TgW5r5=zufXN{M7pQz`C^794TMBK+YY%y^KD?Xan| za`8R0dOe5ns?Mbz-a331?lVB~1lcug0)$%r1a9@bcEv*d(#<<#ih(kT014fNHLTY> zB|{Rh#op~-dVsxyHa!jng)_|J7Ec9Q3(8uj)bKi#w@Y8jHZPUN2_AnN7XPIay}! zT=UrAH71b~(ubg=tCef|2PVi|yVB?T>reps9!Yq;lzx-qr=DnuI!nLQ>w$j+dzCWg zJ25w;KoVT#*xZi9_%@!OlGJl^8e_MpP=xg`JW!c8|pZ2uQl8Bd@UD#kQ?~;t^aX} zX@lyS-aFnC+_bymx?r)KutsPO-J5V3W_`85YD7NCT2ae~D$Er+7R?6W3&q|Rd_+Qy zV<3eV=W8xP+=Y#ei1@Dw;IJpNaK%Z;8kBC^&@!E{_5I8JM0f8WT~9K7Ccnow9GKqf zZOK;U`?->F$NJ1uY2Os(d{I*GFcW@mwmMxZHm}cc1!tnr3By!-8l}50;?Y>C zcs}+Iqxdmba1niC^bDZ{0$(Wc>;+1Y_CR&xNRW2bHySlS!8R7ou6$RSUXcqE(Rbrw z$~W>;q`;01fzL%&l`lXD|Gh#%);2PqqSiEr|7DYxo?CR_+hVCM;$7^jmg!HX5>3RknDU zsDo4+2=uTxL=2T!wJ#l}(=r4GK3#cGa*$M`-A09cTH>^2Z9mK676of)sekURxI^Jz zdR@nFgLL|F_j>sc??kYTWbiWxi^!KZLaT?H;Mz&pUh+uz<_(qDcdqFzF}6khmGlahzpCS2UtEEJO-`W9d+ora+Y>o2n`iASVYg#uD+4k;1W-k+6vbEb zi$qU~(1Z)4A>2{f##Y|Jdo=`WmBQaZ*2d)5r!0^ zJ_&`;y{NGjBk^7M)oI%VwCchxDeXnnhZf<~uHnX&5!V%|j&DhHUeB6IHGNr3k7LPLDj!_!sKJ~Jv7FBfSS~-L zxjOi#K(X-Ko5qyfr6J?HrZaZ*9G09RigD^;j^tv(MtwoiGF6-#&^RCNx-z+D6z>8g zS$?X{zBTE3tZMR@D7s*tQn=B5B+y=jU4skAHg8JbuH%VVEfweGs{xJB-kHcreG|QX zi*aAGakb-W0Gt%8b3_ur<+Me^J2laBGX~;IUoE~g^XSlQP{T`JdQ0a)#$eCzA8PEM z{cg~3(4V%v{D`j&ynKUNB=17#W6Hhtw|jkD;Zm&`dV4vPZrh5Cm%Ln%RRgS zql3oze@m9^j&;wrxb`yW6jSEEnUN@E4pdNtP4?rF{r0=i*I9+DiNMC(AwMpkM6Ym_ z>Whq%(iBJpT`K;v3@ZD>+95fLj28-t9W3Ud)`gHtZ=$TLpnpj z?+U4z>D9|8v%!#-?RBqTWwpAKCKP*hlBjH@V=Ie%+-~0TOEeR0o z8jHPR`27Jv<Mt#665?QIlX-?7Gok0<+v*k+RlZf0(#<*JEK7Q%pwhmVVG`GBJwf?1f zh|4#2lRpX>RozS0=xA8x%}xE1*h=seq)@sarF`R8qo3>4J2uxg!52Q{`29#QcX)V9 zv2pFoAGwqZ4bvx^nTiS!F`uYeRd3-I$-@T{A{P5n`E9Y>?S~f`R)R2|+A&vSWJZ2E z>poMC7!;XU77$xyC}}oK+xrwDrzL{3{)zcRe_SqD0*AMWneTnzM0 z?}*o)Opc6 z1=)YaF6qeHnuJt!WHbpsgsO`)1)WRidE{fSO+~Ga?bk5c3#)%6vifwmL0t&RE01Wu zkl!W3V&Ri`+ge+wv|{5Iq0bJxTjMHZCFxV{VYGeZbh$w&c@S9V zIW!Oo1zddC72V`mseY3Bx*;m+?P2YvUCS%{E_iijsWJ73+6+J)oGE}1A znnE*;5{vq!s4bd$(mS}0@YcE%eO=092irK6I*zce~~cj4A&P_!pH-V z@0XvA${BDYM8vm7h`EuD-1ME~Mmtr;^5C57(-{Pz*is-@{AtE=URUsI$R~C~HvM8F z-Gl3;=I5P|l6rDRyAS*}{}PJsm1zmwVkNHne*2J4~nDjZWH>d(q^d zD+nb=gklQ%WoJ1e8QU(!!>EZyg?*bpg~#v+i)y*KAqy0X*t?;=2eMQ1D+qBfVuao8 zH|qY15@2dk77|q5H=^-&)z89Cjqog*cA!iFJMWzwS0Q7mi+`xj{_2IVVz+8X1ch5^ zd{bAsRu&jaP~pXBlVdvV2DYXn-l>Q*p$I}>ntMT}Xlr}Y0d1Ri?dsGcTN*EA&1+g> zzNHJ>x`b0VsK{GqFM4XH0cT1>4a!;Z)Ld$isRMkW|Fs$ePQqioOj{jX-4UHkOy-y_ z!o`1m7f=i|Db82#`&#enf6%04$Da= zsP87I70e0np8*>dI}&R%hw6UrWGH+bWm+<8RLvPer^x>51G)DYWj*Pxe#nIV^{MT8 zb9EVQN;ku&VvdsrHTC@^Bo|;e-$bpG1A&udMnaCLJx{gCQwZBb3drO;9{Ochu+Ur; z6Qt2yVomHhKezZJ|Hc6AMSp&e0|E2)j%fbzK`l$a~PIc0xOylOLfQ*1VKA3 zlFbKMwftIYM+H-r-|KM+WSEuRM8XWGmKI)HZ-TFsJ(DQ1**nR=AGw7 z-$C7cf#5^D9rBWFTE;fe3KLYloaI7~aWk~{p~PBF z*)WVr(hsj!8MGW6zt$PQDxW8w*)9{dG;lNbWzUzCs%sC+* zQ=u+5p6Gct3Nq!Wvu|D;s$ZX$3sw!^Hm$+3kJM@+LaQ1IB$HI2UGI&o#=qE_o3)dN z(Y}PBlbo$`lKmIRO)kH+2$#$smzZj(eG3BX`c~8Cg?uW>)uY(M;HFqe7E$jnS9jT! zLHoeM=HODTR2!Nova0J|nJ8Fr)#m)`b!S(a&&W-~GQ*vA>-Hw8%|fmrO{M}X_=N6c zLRD@R1Bm+?E_|?r^7U98&npXqmG0s4bpq6(hhMrz2Hjw0dg}M5+6dy0RN-y&WOPtK z=s&A(B7Njiyku;^tbdw0Xt6$^&SkL1htW3d&!zTDP~OV9|2->J0#~O2^>YJP%OXmB zoejji1*S4M&=ylekavRI!Q7>U<0YD82Vv=4nqq96&m5awRr@$T43N}h+n_7)%kBU`P|y&eM|6{jXB5P^`>gAo_p() zBM;voZKPZGOUzB!Fog&cJgmjU%@O$o?QQ!GCL4oYOO|2&#^5kaIsuaI-yK3gwxEt0 z3Idv4UXOr!!t+ud8IwF5>3ImrU=hIBsC$AtE?4Vc%pDhtg`{8Dq(i}9Ip?>vq^A!` zi%AJXBj0uN^B=(kB36Oy55_Wn;b>Wf{S_z10{&F$WnGO@sts6MC(F|i5i#{7aR%_| z{Q)LFE)Y0XH^p@c`Z2f)B{(N(XLkc1$Y8EZC@3U1byuk_emyC*Cr<4KbzUW77g4&w z7V~sv-YKVz+D-j?xrW$^4oG;A(QInI91gpH4)OUKzO?Pd3otu4RoAU?|E^m%-KQ$Q z)8AbCUReu{b1jJa7D)59`pW`OwCwjhVFvamdN;X@Chqk;gvje*8-y_JhQnH)@CNhV zNEce1u13ijShg<>Uc#8)GM3e;lx3&c2NQT(@Kh0~G#3a(e*UfeSg57C|PMJEq!RDx1rz> zFOe)57q&WY_-gy(nZ_(W>^tk~`Pnfd?O~hbX5G9!9*fEt3);;dO=0la{a~ZY{;JCR z_O9F$Wkjh_HQx%=K4(Mq$6GcdfU)i3RNet+sdZH5k6Yd#-CNa-JyaY}8sO&$MBc*; zsq>D5`x@@X%cSpl?M|9)o*6yosQ%sx4tcY){`n>QA-H58o3}NFke&-ZHC!5>0Q0?l zHd`o`oPUgY*iDK@A)Gd@Dgh?vZK+zlJ|4=;6eN3nW3>F6r;yG1^Q<11=P7KjKJer@ zxQ^W(3KCD2XUfC`%iv>gvhb^2pZ<1uJ%B}gXCl4yBO`d2@}_(4+7u{SHZNQ=O%PXL z7QPWO&!mVH!swSmrjD-09FpC{&%u9gJdsbzpr%}ntCV79p#u+UrC#we>`8<$F6rh~{A4O`fb_bqo_L@otJ!U-!mQT2AfPp$|k`(T62 z=cc^-mzJxx0m6as&U~+4p&AuGTB#2SGO$W_P80s^{dgY?AxF0Tny^T^%6Q81I3s2f zZ1rLI;lknd#3!5wq?aq84FZ@k$NDqlOhAT5Tgf~X9&g@@py4>o1MstM z$pjn2KE~Hmu9wHR-ferG$Bk>))hg_NK%T)kT0-H*n_+ipjjTIDCE|@;hJ;&k{Cz%< zUS#jT3;lFS9j=NHOGD-M8e%b>JRo1bsdW@4c>A-A@Q$S$J{jX%Y--#xs7R4^$xRm# zp&5Q%=9|e;!+si*naPAgbL=f)j9gUHPO91+?-nDfa$LR*ZY~%pRaG+Ji{$clGGIJ) zKC2$GR)Etcm_ex7SFB3K2I$<_c|DTgMA$fdB;n6kWR3GT*QjTZjDg#w4Da4%2&kRy zKQctbHfl)okAQ{%P4{=6=cuihph3C%8x%Epz}YHx@y7wsBlXcz}Feu{;}4_dHab+0834$MEE&hMX& zZp#Trf;R>FbPSSDx8D}XjBwsx>Ua3o7Rn7ZA|JtH;?nYyU7Bq=7F>UWOam9cT12kY z&(pjdp0o7rr@mzb=-eJw}m|M_x2D>1O;t&gxtT^Z?NW11W~(QhGuao3Up?S=8` zTg^pYjdB`NNBggmH$;2!=QE285rF~CuVhL%DPzlHHU{T^gU!|Rf`nEG|t|Y&PeFDYa==0s~ z*ocNkVaph$m~e{SS?@eYeG_N-itnn>STrzDy%Ic>=hr1*o6oa4hSHTzRh|l~%vqdV6MzRNzj>xFolC!5ZaTpEn~TFdx6xaV7lZm)uo`wa zb5ep_<$3tqU=dv{GaCMzW5M;yo22q$n(qnXlVgYj&a2r*4ux&{Lgo4jg- ztF)mj#0+6mWYw|*tweq)a5`;V-bTd?hVJC4XpU$GI+ykFF0u1T;|;@q6x-k6B|FRu zna@GTG^P?9h8s8rzqJ_q4eW{;App{s!Q>VNxK9&HfZwdxuMawJbO^w-ENZQ<>_Mll zI*3>56NQ_TPkDI z6P%cc@ZC6p4@VhA6Sf7;#pFEo6EwMZBs@lef$E#$se!xk8->ndUnUxs$jartvH3!P zv=n>!#y_=u&H^v32S2bCN_NOk)?l=S8Iv7qjChfGhwoMbN`6D;NlYIhj!~Bwgwxyw zfv=x6eR5~O(WkGf7{1xV+jAMkK2e*|BD=9eAtr5A$#89N1vpf+I2Q#-jlg|m_5~y5 z2n)FJeUPt@Lbi)@qNj`|IhmyVVd zXRpMN8gsvfvSH|OH!pnL>vsY!(11`KqyRfU&gi*cUr8b_`x|pgua3wgY<_&G3bjKb z?0LQHKuvE4au$T%mbV|f>7fQergXN|c`opf(%PV3~h={DD&84`S*!AlkA#>CX4O{ng8BXA?tIGv?yb(u( z`f*F5=Ja+FiYv@kSEzuABU)Tt-N-94PXraO)YVvn`j_n23)fGTsNI}oY8b8+7+l+G z^otPW$T5yq=EW3#&5N+2D!mYYYgLf0q1n7Co%@d{;HAn$TeQ1Cp>w#@t!l-@pD6IK ztTI+Y%RlNQGQF=COhhGfVzjH$?rk1Ly%Si`PI_d7AaQdGUxj{73fX){x*awg$L5?- zvj3X1Q&%*A;k@$T2ZPvr4Hk+$4XvY60UOQtV6NN1tqRSl=Y^?WIbVGusRQGb5&E4f zjz7i~Zb_Uoc#}275XhHcQ$;ZBtIv<=pik~B!hR7lx${}bz-&3Ed2RH)wXk4y% zhh#6(FuPvX{(AeUXDN5L>~bMlJ=HZukt8iE*(rT$eIMLU-~@BOekOdA{0Xr|&D3zQ zMVq8ewhas@@(gcPI>R)8WH@KH>w8qP;#E)O@a;cQdk5&$dK;^lXEWgEoYDII;_XcG z-}+?LXVaK=8X2q}tRlbugo>+w`%1cDuTZos$vrSzq2+V;MYTk$Ytk2$f{N~o$*Q`g z1>FETN^hG~UYmwTjoifm^73{a&$3j~pg5YrU33pMCV|nllV<*(n-hyBr6zA{W_q5G zX%+1hT9@*y{fH*#V0vb@IP_}E(Hc9z+#nClvm;tIijKR4KG$j0#%CLj7IDIFZ2{+EnVZxnC%u-_B#L^2JP5L$n0GYhH;NxOV@95^XR+rCRO_-< z4=+Ta@9E_q%3EaGzbn^?&5zzT8j+lz6w?N-_(bqq?Pp()WuLnrG83_QC?1AvtwAiO zX>Qzk$osX0WjCMPo9ZIgFVEuC*+OtgjBD5x9&Q!hy6M~&!;Oy7+KM{Cs%W;ySv|%E zo|Z3KlK@BPxa(7DXrA-R8QQ6B{RYgHHSKPLO*`@@tvl%Zh3!5p;phJC)bMk?ru)~b zyig_PTwq3#`K6xD&08=nQBzp*@%D}Cxis0!VP=;}xKR+hTk0@)I`1T-oZNf;Jj z{79;0fIDO~Ow>93`II^7*?O2Q6?b_v*B@EC^GJrpfUszoKJax1+SOg>32N=$n7z~J z81@%mmcy2$yhGO~>mR;|vFQHV;{09d)`CDI;0!sty;Z{E!cgMt+oS= zSe-GDV~fN^y0tS?y|dos6yAGxcvEHGlr>}MwS73<@T1q7YpC`#VL-WLk2Z0x(fOLO z+tO4iuRkjgNGropW~1GYJ!T4{ag)4cte}%^c+?5}Wxp9vLg$@x^GY8QUt%Slu4c-N z#5&o=VLvmulutuE~O|V+P~u3n9@Y94`+PRWIY-!GLPUDt{ugPM;G7!Blewu zCTENH1g@>~H%^DMKaS8E#2eY%663)E<(%q|4~2gF7d+Lnje)f?>VJjjGURtXwGBT^ zkyV_cOl;1i7f;C$* z2>kRdXvC$uy2KklPVBTB>e~HYcGcFgaD9Sd8}4D;l_Q#RMF!lA5n8=51z+cjO&f;) zaH_9)-d3Hxelw#LRW*0_9>P;faU0^90Gtv zIQ}|ZI*6nsh_Qer2j91qWg+u!PBd>;sBB6X%26nQ>9J#mnntN(Z&Vhnn$<1?iUYUa zm|x9e&>R5=3`^3@>6%6HJe#`sdSZ2+k_Gn`R={XKlp!emouQ;=?qQdCf-FV*lN2d= zl;nI>ACwMvihcIyy)UkSz` z%Sk%l+-lv>1cU68;m}7OGnGn2yB?T6J7$83Q)pqdjsc%zaE+uiC|FZyvn~=JNC_~C z6k8W7cd%mW5}nbfk0_D}IGN{BiV)CtW8k?OB0#Uc8K&g93>3&TG*`<#V8j}r5(czG z2}6N_kuP>R(Q|>hlbX;5w!R@pG%*-sa7lhN*XzKFpm9GExJgRrn{_k~_ifZ^dzGqk z6NG|(v)oPF94OUrxvD*5U7@yOl{`*cFDj8 z3mLcK6_|kOsvTiA2GlfXDK)`oO8~_9R^7H%GK?A3Ly)!CQYO5<7UV4S)}0AgF6gCp zlX_stUB|4A2^S05h(dWjK(F4#0wnmG5OFsYkn18~7?Inx-><`Fg$vb+Lh8zf)zbMF zkCi{r$lpyD%1&+vhWR24)7q3il{0C*;3q>K=-lf4&n!SYFcqXZf|`SFhgk1YujBR3 zp_(Dss2vuGPPl&4t^M&0u&8Zn3fd1f8ii7&QM=$R8a?u86OF~@Y4$?sJGj5hR| zPJep-ride>RE0a6*vvSSn8MN=&27t4EKlanaDz08u$8zZHT>kV((X;&3e zU)AjWb~NaH=m)nTNd{0vL>|Qn8Nj2I`VZS1Us5x<9I2O^sS;BAxQhEg;E)u<9>0|5 zhVv$WVysfDt0JAeCo=Dsp)?1;&;MY5lu%mBk>D`69gKHi!sQ2Vze*35WX)gI%$9N5 zeG*3a6p}DNMUA|f@{@ebH8@3V6*&?v1%r>Jl=9n@CulUUy{J5LqPfFz;$s+0v@n|% zeBI=_J zm)5qv%ohGSmHcjq49!?=8c5%s5@2?#?Wp4APhM>2k~=jhgjYwsJ3n9?i9m) zD8J3Q<>@&vLU^afE3uc$;rvbm5O+omP zUD6&&V_&*|N_d*|wG+RBs|Hk!FR+ZiCXPZL4k?_Onfm~ujocwJlw-hr0qruBbx53gk6|-quH2ijieWcu@Nscw zWi)07cEVUA!k(Qx8KU}$n>|o&OfA!~JjATyagYW}S7M;j#}JHmJ0F?%IxNqZz78li zZau2jgSPvwY4<*r#R~JDdwZ4LsRfvn=91sm?z(KaU>_u-PPjs!qVzd-A;=>6R@Dmx zjF>*w0WRti4?tPGFyp3d8TUo zH`5ds^ua5xU8w@#gxR(`x8GqVNNRn)KwkgW=-iLSfJ9}4owG_TfzK;@e3fpMGJ4LJ z!A|yZ7?K^c3r8L|w=<%{6$Jzdq=(5)sXEHt|--Q0!Q zHCrOYU2%h@Rzbw-ELT7xTSgzHed&`*edq%>$sF1r8Kk~jw|G|*c=%{>P}G+*5+51T zmad_6obO&TABJ%3G~F~cTxupAQ~za&J|jOL1kQfK+<*z0)96#{9N`PtbiP5NFeyE^ zM#1#5|GG~lop}UI|nqr zR|lIIL}<(n)o5`?&7Io!w+*Ovn*vi2*9QrSq_pOJQ7tytbsF^c?ib>3!E_;ZE9VAcJ$OyO>+bw`Cj_j zRhflMBk2wvPKa!evSIoUF3`lG`m0KT$HU#dHOzZ8l55<+#fp&HmifREEk;Y>xcFTA z(jn=_kb|f0JqD}HmJq>5NyAYsaRO`d#HrP91s*tsYmt0Yn2Qt4w)KS#bf2P}abah9NC8nS$TGMy8gfR&io(h_U-?g%dQ&Rkjo`S2pJ&~ z1}Wl_gc#c(8k8-2_NGFXEFn8XwyZJsG3hE(*>|#yl4Z&akugQXeY&deqf&EAJ|E@_5G{w_0oYAn+^?C|9qaq8*6o3mRtA{GqZ z)4y1^jEedULk;q3`Azer5;gmP455aC7BX$RSmZX86n2Oe`4Bhi$rzir^B6FU7`i@G zc+AiHNNS2^Vb=3v{U?(5J`^s03eWods*&GZVqla_VG?VQ1Np0%wDLZazj%F&8&_v@ z!<79bb8rZAtMxt zx#1EJG7%9geRmdk?vi{5&7awpH@Mie#}ye^4NLR@gI9#R&ES<|&$FanJo~gzn`f9| zh>bp0^BF2VoGmpP?&O8Y%CWbIpw=ApH!!@a94|L8Dq8!v1xoj>%lo3{|5j2%EW+5pkZN4^!_8sOeZ;a^cxR|2M7{ir6u2eM6b+6TNa#cMp3JKda zNxl=~a#reEj>W=Ub!(A;o`S%lqVp&9p#J%w475}Y|C+P#s22N1l8GDQ?!EB_gzOob z>VCJg1RvPi`7$I|?05-Kvwc{EyXW~<5G?$1k5=Ob5e8>J%(J|k!6R-ZUMv%BIZ~yI zh&qWRAl8`b0)jWAm;bD}Z!sT03)Igl3fXpGlqY;T{G!Al=u~aGE2ohW4_TTn4Xu;m zO*t|(hLt-YeA?452-9JGDB~Wu@{}ydzpOMe@E31i=WAs1#gIuJDJT zEWr-3U3b;%*y`4topd0=-0%Y7Be`LPS%N6d_?A}emiZWf=w-6uYkhM+#ofgO~9nAy5Y zT-AXIi|C%oOb?HwuRV>*sS~dymy-m{Eo$sg&0Q%15 zYT4{pn?A8x>20+pN*dKFMZss)JbQdew@6HI$1})GyYT>^958#obmR9V}n(T|24p2mik#_!TEB^hgx)X?xNACiOg=5&s zJwu|s0`cojR9g>jxj`R`>qKq#A^In55TBa`-7wUN9R9 zJTZi~biir_FgsuQ8eM_Npve7O4EM|}HR7d)!8>6T}~>}t5`)d*z`mh{O`3^0q1dyx4n5>I@K zy(t9B00>AhYo-_IIt8V;i=+hCPbQ=xqd?=CD+8RyM`EGmI zkO#u>-a(%qJJx6#|K}O@jqgj+k=$}h0eYjicp)g7B|gAI-V!~PGQP}?(F{J{!T*qb zL-RwS-2;yhXB!~v{%5XD0M!rCR&i~+P;=m5qZqwn${|ht>qq3bzBB!kM*_DKRTB$_ z8GX(0&=AbH(?7C16)Vv5TN@qSAZ13_I6c&KQpGKSQ~MBKdXzkXVQH9j5JP%xwi$F; z$Umb)eBetyoO$b(9x863kg>X+)v3-}eXu`Z=S{WBg^S+RS>glR45VMSc>^JT$FmAX z)>(?_pf(f=BAVKIf+o=YLdIG-uV&94W>zaj32cZNcv5}5=*McdTLO8mflG+FS+muQ zAc-OWgDN3IrKS&#>SY}t&@C_=+&QOljGH_bccSVsrmIBl1QWPA3*o_NA|Atogd?qG zdB|q3wsCxX8WffuKz%8n<+d!=mlfgdwIVti{r{aU2DqI-hxQS2Zj%=Dp*?8-trApy~&&NSnh4Cq#cl`S*Dc@NgPM79Y9n}Az!Yy zpry4w+Ty;wfBko2+mrF~DxKKRv|Bdh!$wc@ppdkT)P*a2IC$xyM_=OY%~*mPn+FzM zcew=N^%d2vGOlGnq2MM~#9)U{A8bzf5eYf zmJoeDiaZHK>+4|YbUj)2CrkUcxf`wW;9MIST*SkR{(Vz>eh?qQnC2BL<~OY8cz7n5 z3*cAbftxI{ad?~y^@LcTe^sC`<}}$prE=+|lOCUo5f=aG*!lVzX+K5Y7p0vLLa*2i zo^uwBsqS2?LwD;$h1|?DSbeXHT?&MOshB)^Ts3>K2G$H5F@?6i_qo09ejZd-c0sbJ z&4q5Ar9BqNB7|B6U9I*Z z2v95GBoEHkKHti$_0zq$xUoR)=Oi$89QMue? zJFX{{Q(lBlsZ*kpgZAv~GcPsX!qA2I#NexMsHlCT^N7nung#Mla|?Fr&ruc{I+wWFV7)KH&QhVLj!o3w%bcpB5XOr^cfLt&@Gb5iYLK@$)==XvTwF@N5-0 zng;;7K`fPr!aFZ-IRL#WrBJz>klFS+=Ku-+rZ6_}y&m1Il@{9EKKZ9MoMxxs?H;Gy z|Ei;i**rc_Kl3f5mZf$2h{Wwr&F>19bjD&cLi$RV+YRESnS*!uK!jaR$y`WjPe}-#uSb`H?sKiK%hX zs&dG+^of_y%^;0dmtDNbsbTlU_s(p+is~DOaOW1kYH+sE#^<60Of(oNGmpvP;Wcg{ zJoF&>XoH3Hi~&F;!V|i6u)Z}7pK{e2jIb_o2aPxn3w}74uY>JguM40~S_hR~sg3p- zQ9?H8U%)Q41Io&`_)m?Wuavw}(CBPEG9V?)FaXrpd_;`S%y66Mcnhwx6dSX{Y(@D%=3B;=rTDWXm%3-S=C{=4Hln#92K|-UBvj z%gAQ5n*7_248L;16Xw>gYU`YYklEC=llRvSRw%BpKoIS(_olG!LuS-0;S=%{t!A7~ zI5?vl%&~4OSQc}Zq^e0r%^N$#!vT|zBo(yW{&;zb!_WV%GT5!4s_r!DUkRKkbxp## zU+yH@d9KFBlE#kFw|kcap*N&dVqx!Ss&5G|DOnPl<%V#j;hitl$BfiA%;-)NlHcaM zaN6Ul$Zc?MH{uVD?f-d@RM+%g{kp^hvm9O#K7l(~~X$O)+`@6oE`L$?Z5?do~Ja zQLSu7``mfe4C1zc$2NFM&=Jg!Ya`wXHs?1BS-rP;IqI2{DV#|RoG|ub^7`Loi^r)e z7{mp0Ud{7%t<&yb+z)KZt6^G>5$D>R(5);zk~RZMeE`ap%*OUa71PJzSFp|+Kdh{~ z`FL5aMwV7qovNLBnQk2$2aWSXk^)))(ucU2<_|89hmg5GdZ;h1?A79zo=VqW|kKp#Uz?C!j;Y8z>mt6R%NZ;#BFc`2OSKs#Ly-`;Z zC7Ej#Zm}=$Mw;lkqKX_zE`Jsb9~<cu}3eh18u? z2z@}30Jp_6r%R%Uo+?7fZCs3_hyc~GK38$_oc3x5g<-jPM)cSazuV1_;N(Zz4vNfb z(gTsL_bWm&wXc$x{D%*)anQcGY$mE#_&$`yPol;g^BPQ6D5du1Jt^4xs`5V7!MWSw z{xhe9I`6fHyWPo+*?i9tpz_2bw+?2>XqT)CC|#VWU&rYR(;C7zW+yTZxp!l+U6mrA z=#*LWrVvv|SaWXH8yCvP^!?ntf)nV%Q%*#jDNX-pu+@N;}`+5b1xwY@b|wIfr0aTF8WVT6Xj0h zcLVIl(>VULIHqr38#-TFalG4UD!!GX{58?>b+SCn*_z4O~&#&W|`fh3mfP zxdo*HAOYKy$>v{Q3+HU(`S&LQz=7XgKynpkT$KIEbTB%4TYERi7}nsr*I|}gn>G!M z6-6{|+Fo@R^f$&|Q??Jbn%%NL1%oY7)aC!qSw1pg3tVN3nH9vAD|r|va(zn~8!9N=K`j!OIju5u@W58q$d z>%xR#X!Aus`agK0n*8+aolaB%V;$I%141WHs>{{!8DJHDOD-ymM{r%GDTcJov=aXnsBG&qDG8)43k}v{E1WxtS>Vh$~pd79qkB zIIf4o^W`EAhiG?OhmX8?rkbvwnAY)r3l}F!k5Ww> zL0DT9y`D_hEYDKAz#z#v(>*H2?fheWtUB#CgK__Sm~zz6K%*wj**b#a^RWBb>-S1S zgrP6{lQyxpLMXYjPK+SMDUNT0B5ec~!nB&$PRTe5#C#S!aZ-Jq8DM~3HKal<%&1eL z;Vne=j-ZZ&(&Oy)d%R$lTz$TuL&=GOp(s6v#1O^GVa-BNDp^#Jo}23orBdQ1qeb_- zQ!?CrOwtb_nKXd0@d8|5>J9Ej$I~pjfI%NsJ~e4?m}_87@&Atx{OLRsW`+E~?s6S2 zE;mKaX?H?P!9b0mD)%0!GoIo!TK|f%!$HnXUFda2ULgb zj|+O0>~K?pO*#D>_8bVH$S3zK&L4-3D-*EZ7F~KM5w^XhA8Dm^J4dX@SSG`^?GeTP zVda7v9@W;hq9AHFZ5b@I-D|cffuCEA?OB~dg7CHjAtCJU#GG<-{p(M%T^Ma7sC@j;a6RA8c<%?{d&A&Y zdNWa0E&|YlWxBmQ%bCa9T@1hZzP+$^IW@qlml(N+XL$~AQY(7^iJuL`6xj2n3bVVi zG%Zn9V=n#6T$`4Q?T`dY;LcjoBsHv6Qol^}R^lOIY81RGD=BBv5b#=t`-|MVm*(=i?x4+^xv>xUx8xbqx{7|13C`; z?k0ytVN{b|#`sgSGTz@tOUeOdcV_Op@EaPV`hhbq1$AjsDuSYDJAv0=byfe*_qJXH zz5g)@dGd^|PCGT$lWMl7K=;1xe@3QS@7$g8U+07$AXgF4GHR~7xLEje1aQzBC1Te9 zy?YhH%ubFq)wBJmoF8xHKCTBwj<+dBHt1H0cgc#VOz;2g|E`C?l3%J(fR~w*M z#j$OBJD!vl_I2JvE&RWI75OCis%|xV0a|oG*4`;;C0DDdZk4*~6wy3y7Z3xeg;^qM zv)7PGl2C-z}>$KlR@0j%;XYybcN literal 18102 zcmeEt_dna;7k`y1t<|bhvr1{HU3*hi)ZUvKu~+O}lxnS3?Zj3)s6A7Zv_*oF*s)^= zL2Tc6f4<-U;`@WRkL12y=iGbFy=Ok-b+l9|?=arEapMN1+M8E;H*OHG5yq!vq=Zjl zL-%#UKXR`(robCF?$TcW5#7iJF%Z5a0_v$M-GIQEuV;_|6g3rZ+^9>wdv1H{#*Nn+ zYOfRx0*SU~wW~nmS=h5QBMmO%T9No_R}Y)}KUo#f4O5P<6cSrgn{<_JXtHfc;|1^k zS)+LX(gi*AQjLsh)zt83{QdZgVy1}mTV72#$yu4-P1PUW9P^P$V6|z%8?Pk~tM@+* zEW&KlH6okrE2r95PUrH=CvUDt#UNrOp{ zB2r(%4{Befen#?N5A<#`3_radY}_DjyM9KE#YQ3HdQeVGv35O^(=U?9@_JBACHVhe z^8fbar!{Hf-;BK2-IL|b0ff@=;=jqO?Q`yoOK_sOz$Ko}_(Sc&Um7t>mjCvY?$Pk= z*c;&<_gfIT+2<#FTbs$Hj<%pHt6#BdCRhFA^*_5DLdcBW!53;D(H(sM$?V~7qr4`` zAuaF#rZZsnZ~YZCB7%>R3XK!lC8)1 zMPDmjae;7t^ZYrCojJ@`vODx2=WsaCoXprl^vttM%tG=E&D0FJ@I*uipW!}hD@%w6 zQRQh};v=*!&-d#mO}E@V?defvwEw-^*@5fKiXI=hVewRo-y~5*&&2<_r zW~x4#UjcFqJ%{bFt;kcbnoa~hP0cAn`oYrFe!s&b{g z#lx*o)kffh`YE40`SoJPcu5a#(eR)pbiS8>#r&h5GIb^DuIFoEx>?TLgRZ4G#zuM& zOO^_aka{tw3FkKyE(5wp?EYZCUZI{m!?1`hU#Ux%Q)GhcAUWo=LQnDH(;vSs%*mcV z*R^_XP!k7}DlK^)eegQ6H+e&`I$&ZK?y=7dCkUwrgW*o4$_qQhy0_{0&9(SF^bvQr zgfBhKryZlWT+(hoOS&CRV1d&wok&(UerVpQaSmr7Jw8)k${z{_eek(Fn~7r2h`da! z1QdkfjwW#vKdQm=R`1l_=fDV$sci@o2S?ps=cp@I2nPmpH_}@oD z8-YiFYEkr4tJ})|Z7)PPzs7Dv$E4Cl>jwA7e>X2L>->pNf&(F_?K=yxmVP~_h2y^e zo61j^YCGZ;{tvIzvb6WZ5XiG!%KiSFTD6VtuK-}+i00DK^W?8o>U}pP>Y!1m_*(Bb z7>Q3EUNFM3$3bM$N3l=H7D)(agQ&AcLOsdGf~QR2l=0E%wch*16NS-zCV_tC+a|#+ z#c^53o$2yv_cgLJWVoD_ehpvLar%{VeJhvqQmpY3-WajFzeu-OK#EIKCp}a_mV}Vx zTx%4+V1CVR&yFJo1l(d+2pu_GB?0!VrbM5pmzbd^1+j;!${hI1D|U;3kd*8BB|`a# zpe1^CkOcFOx<@lR+`URB=315hsej%VJT3+@&pt_I25_OUeP{dJ5|GUE!Ltb2ET_qb z*CKL$PO(P+(*29oOAsw5I=F4}m%?+B#Tsj}zPyOGnGIP(-zX`=!|&O#2YMP0IMpup z_F!yr2)VNtp{MJmw$n}S3l)`jSUaDn>2VPUXR=HlPRA)Vsq{)3;);^R&>v*RzpKrk zBjJ;Ip<%dK)4oidE~CcNnPu-VMVl+Phr#j}_WnmwSC-PhXopRi<{Gl+UKZE0=Bl@G zI%kqCdUl;S2Ck$T_xwraE?o$nLspyj!okwLmI1S(7y4g|#S-Q6#b+OQjCjp8faUr` z#Su|l*5*xVvT?FcZsdVu4wJc3DcG26ME=w|skA%4DIE$&UFK65gjkNl_y2Hb zoVQNR(;Ji@dv-y(PB+%`3~D&bA4Z+LC%*k1U;1Ds!zy^aWL;LBKrBHnU;b>?zgJRz zQ~c~NM3g(yb5gES$~qzhtkA{UvwIc#GzOoPM!or&`2+H0vAznlzvoZIyTkRda{Yzl zy`=~bYQ7ICaQd`L7HNM8!Sn{f9rLT>jOFA*#t_iw3_> zQftKeYQxXAo2NZ^o--|+vt2H*eNX-LS+Ll(>iEL>tX_UR&DiRZn8%F7g{PF&YCCQ* zCW7rK&F0ULcEstwVU;?a`!${&5PvzO$W)}(tsIVVpBJM(Lwn9Js+cEas|IQ9Rtrx- zw4+&M&vgw;HRf}pw=J*gYE%I_1Gh=X_aUKzINMrR6lC`&F|~>3}Y~e4GcwEp#z>q zojzd)AFi==fiLl}u?=_}wjLoj-A--m=;}%uIp;WHU>C+OQFST z83eRaPUi->*MhtdsX?|mcB@H_5Nj-xSkRJkK@Hnaims(4OzWHb47=%67?XZ0=C2zP z{#zy!&7Npq;pC9mzw@krS`zlM?CL{{?yfPHr$^gpnFsM#9AIBgzwrO0ITC(wDj0B| z`!P1$MeJm_L0Ho(eDCap9h}#UDes8rT@5cqQ$)Y&3q|D0Mjz`e_Rcp=d*FH%Mh%9} z7zZLbDOEvR;$<0qjrMLwmDU$L&t}VRN-=#{fk;D(y5S2AUgEM#GDbs_bsZYg^{AiA z#M?`ns6g7$Njmvv%iO%2c8+uCW#LTHf#pw+ei9rYH|YDn&B2`c-bhn(d7F~Gcc}x} zh8|jtDKBM&%0yoK{D6XpZ&Ss@v4u~#X0HkKzIo~5?pez|{lpb+>KcyiZtFguS(ZBA z>ojdyfg&td@%I0ggFIV=V?58%_43VbmADp@r6JDWtM2uhGd{|KHN2MhT3tuib@F*~ z+gQPQz;my9k53R=aXWue)*r`212@2;Iy=Ywi35eI71fyKKdFkNUf$FN`1K>68z@VPB(*h`t7T}z%98oKXf@u0Kg&ZyQ&kiB9Dqda$?ds1n;i+&Y%#0sX5kb z7D5A=HT7S52@yr7)`HFNVX#;JXt(NcIOomHk;x(X!|Yn(bIpLmfL;k6^+Bs7=WR|y z-`pBDQa!wP@+X*-+{tok!De7bn1r>d9?+N3ry_Cbqwur*;gz*Ri|J=(Gvr>9o89C# zt+rXf@f;9PE8kw$zuZe}A{3uCdAbGYl0Hj~Cr0WDOciSWj<}j|!?gpr(Pw!9h+JU2 z2>f0E06jBrPs%#Yl!cPV`+eVxjL(%TEcxhPE#%?$E3O+HFt>inT^`*cJ~G}Lxm|O0 zT2l%Z;Me{#_B4s(?Jg7t4e)19?ztw-*K z_f=Lx2NyzFt6jD%1j=P8oR%Iz(I@bL@6o~)tD=?Mjpp3Vo@~D-OTYdQOtNWxj2GpL zO_4j0{gTZXdaG0IPzHa7A?=y?+xgqfk&7gg;kee0Wk3e9euAZ+g3*_k!B9`cX@HiQ z=Wi^NwM0SSWmCJ{C|9^Ks&dU+p-N+K;SS(uNiZ3@PQ5FkMAX3Ee`m&Y8-Sgt<9Nf! zEhfFh3X^iFgnqWI`Wu5Grk0`c9m(fiYaItpfq~}Jz+O8yS5HQG`CFVwxj_IQ2=Q4F z3=8sqKjfPebH4sHI_}5ip99RYn;`0tr)?qB-DtFApRCtFdEz8f;4&oZ%xZ>U5(?5g zL-3?(OMUx_ZSSj!1(JlaPmB zkLL88a{z{{joG4d?30AVjys?~Xe;yzci1LE-xC+R5E>TxxMksUgT#)cdm{wXvI%Lg zs8Ye4dCaznzF`zs4t``I|CuX;(M`eg$B_)T1cr4wRyuqxbm)W$nprJ^b$Mn%y}6WS z;dFY8m7LgmdkRt`;&Sq-SmUOOgWK8+#(Jsu#tPWYOj_~8cN@~8-!Tk|R)5FBE8SF$Ele0w zX!9i>@?q8*XLy3+hj!JUfJ9O^W4(eVw)%?Xifs$0hK!$YS=t_2jLM|bnGsomEEDM| z#qxnx0TnZe3zwp}4)=;xO}HfYu$~{J5eceoU*DKqD*7N3z6TXIn8H1YUFC9Z$$Ha@ zKdFikrW}z(36y?<1A|JHVTZ!e14)^GdlUHBEk^}q38!8KSwsu3&ebrZ99W%rpg&}) z0(DAhZ=wCGXI@SohH~8MAHvIl?_HJ_?A!eIKU&^$JJjRbj)C)~&WFuz(<_}cP3E~4 zNS)`gA;f~tobLO`KWo_uo(h+Hez5a*OKx!1JM6TcE0qX;CAE$1=Ly~*={fU!xtZ`R zG0R+@A-hN=(RB^JGb+d;6X?}eT;lv;mEhq8*h#3lzPrjpS%p-e3MDJ6NC0H|$(ilM ze@9+=yf#er%56WAxt)faY5W%h6w5bn2+P(9zkp1hp7KraS1t;T;phVNxbvX_Zb%=f zlL^35%Fs#d{eU-&b1dT6|8!^9U^vS}ZI&|kCph?fD1DMek*qW;OOspG$?`F~9LR}y zC8AzUUnZ6lVpuZD=J_ks&$j=Y>)NXJs8<~;cque9LBkT+qwvM6%YC}(v6`#x3*3ZV zvSzZizQ_xD!F+1Ld|F0w&vG9#1ds>!m_|(UiQ9-**MbsVAZBjAdgHCwTPQxd-sW`Y zOUKZ$(Wfgh@nRhQDk$I-4S4tTurdD=9IaVB`cJoRFcRK|b69+>e)h~-6Od@9gX!-` z(3}pO(U#eZS+HPiY2gBgPy!z1Nq8^v(#FhCFUk(RFEr4OR~LaoEnPl4QYe>d8N&}9 zP2q)hOrLmUYQ*{h{?Tv>{``oPy}5!mi^|~K_C4l@lo-+I6Bn|Q&^?(}G0kYHD~~1Q z|)&c0~h)JYkx^)~kgY;LDT;zv`>xY01rikco;pm%>vsx8!muK!S9{vfW?vDuMRWO%Ca3!sb#by(A(w zyNt+gN8-#9W%H?B=<%Q{&yW1&Xw-*Z z;fUaIT$}1F-!#6yeCyqFwKh`p4{;#|6L4>g5g@jV6E#1U{o-UpBwN}k3&0#viv$Q< zMMSXvpt(eQ((IP)-v3y$K77B`MALI>{~{-D-|XD3MA{I(=OOOkZ_*I|$!F+cQ4oop z+)N9*0PpaALEBm!FCrS-ow2F@RE?2Z4xhCZvD1n%bCqS=pIl2IpCoCCof=>|v(e$( zk=)*vtRBNU#t7WB(^^TJM=!tK&FK;+@6nEh7p~;G?oAICEgrY~($1uM&q-Ea2yppp zEpN4Hedwxm-*KNknB#3DGR{9(3a8axNbQtYI)U zGcDXZ8wk612qG(u02oyU$cd%~?LtQa$v|msWNw-d67tcf1W0^e4!t>_6Yxx7K zVZSck!|2e{p_D7RED@y`YkG?vyo}Z70?Ro5{0p!EQV+$d(Z_R2^Z8~6H(Tg#H%}>9 z(8JTno2M@yJ2W(VYB8gJ4fW}*xcQPI!I(wBvWG%5h1>G$5Ap%-va=+m=(l-mzfA>_ zK$fWT!ida!m(yfx?uH2RhKBb6l=V#%T#}x?vIaA8bARD+^2BI~XL)%+t7hX8)8M>r zt;+M6W^{PSLdio4t3~GM%vCme{Jz{Q(_%3b0{Y9*A?7eJ?;xNvVS?^>!1_{n>=)-xo%W+Zfr_MV zk^@lVz_pazaU@ru$bQr5)=l`?eK-e1Ej8t`2AXLxz_Px%0jyYi+fwXNmhq2Lh;pB= zL(JU(LGEWcH!0K9X3bxWv;7S$J1KaxQgm=Hpkc0rLW$hBh7gnbrv7X?Vz>GEXGG{N zRhozVXG2oQ#YZ3YW}neLKmw=Tn?HQbqhO3vhZmhgOJ;ZR z^9Foa4eN8%d|KU%dx@%EX8b!r)S&kVuMe8H6XDybnUT*qzf9{AACl+aNwkO)L*2tH zUOY_R(`n~m#KG!KdQf-_=82$@PWkPX2)aB|l!K9OEbv-tF4$h}@(T;TXj^$ip=@l zW{25Hg-cwV2n+oP+BZSuT3lhny%$b0Jb;+s{c{%&BIp{+tKHx#K{?fyxIcdX5aWo_Qj zxZG;WgN5D&l2>*DjxTJKD*Ml#lXiosipA!ZNc^}Qg65vip-UPL&tyHr^-^7J4Lc>3 zRtE&2%tu?3v@Q!GA!Y766?VRK2hQ{A=Pc};-$~?_cV==!m?wR4jki!Bk@)V(IoECb z@SPsJ>?`T>E!TimHWu;Z{0P`3ja9_U3-qoD5bg^wz8ph+CfMP#NSSD;;BZdq0WaT!Ugm(?3^r2e^PKk@Iik(ChV`@squk zDplTn^%fj5{(!Ab^!Ag%9pSrZ@tNOo!6Kd^3aq!M-J6CnHi9!5M9&Q2jP286Bc+?) z$NNIMg`)%)-ZefR{H483*@UFUt2=E@=t6GMX1e=REPLsB2zk>^F)muMEBCDPiMBL_ zJtmUneoz7_-T4dIH2?$6FdP-D0^#zVE7J6-6eNc;E>i5l2>tblJkJO{lbG?oSZRwb#y1kDogAg5a*aDtnJd3@1@n9!jHN z?{uA`$=28$62$e)XMkqio)c-80rJ0@nM5ZaB#HS}tTx_*Pl9E|zwI+F`!d%)l%?D3 z)32PBtm$jMj~0LX3{-g#WCGOD*(6pQ`qk~lM7%F?j5W)*&|ppps8j#-v!xRCW80)(RmVu2e5og0!MLwb zF}|CSPOK_+^=ipWlfmbPXc(KF+8t2Af)q2Anvq4T%|KCP zF4J&($@2WZPb|UsCM-6?-QT4CV0ETcecyB^CGkKWv@`TdQM{RvY+&~1FWk`(V)5qHcxp@ z8o8dhfGoqgIPl-QxJ1onT*0Ewnd|9d6+bB?Vjsdea&zbkWxQ`kIKK(B7OyS@Qm=1j z;|{&BY#Tt{M70D)g3lCj=_7m>yROnT)gu&;y&k53aMsnD{q~~tW;Wv~FvKO7QF}_qzXZz}h&28F+j{XtjM!C>2i*0Hpt{brppSRR0{<4K1FXYSky0^SiP*$ds zX$DI)MNvNr(QZ+*&t`0oNAa{AurT3*I;K=EsMq3`CPDdG`s-Sh8Ei%pcc`$AJA1uL zotP8TXV#k-ug)@$tNE34;6xBavK>;&s-Hpo9VkX}F0&UBE?cC$hKnNVM7EB_;=IdS zH(fGnnSa%0CF5f^M?^VewvDaBL^shCt-VgHP^>>wbym_k9zB?u|91P; zM5yu`|E3r1?8Y-vOyNZE1?Ib6Emhd?@d0N{HVH`m6 z5prvK*HPEmld9n>%|*TGR1uyv01xwY`njx#us$5JZ2&Ki^>k~YrWyBz9L6g4qcPLKFT8vvQny$PjYvu9sjjZlQInrLG6qFSxuunb^Q=)oohYpFWSV zUtH~OAB=wZI;K`sP*gpexL`!MW~@#J*Fh{T`lviiD1x!QhjtJMXNEE>3AmzM%l5)F zA`sb#2)@Rtf^#p1I5E;(id`Q-H~EI!qCtI019i0a%>hdB8Nmx+c)nJo?fvM(Pwq{d zFt7PK*I8Id&QkWV*-7x;86Dov1nEOPhuGWOGcoRh28EtqqH=4OE;_6fYGmX805L)| zZ}hlet$99)nHFUyj8b1(<@(k6vv`vAV^|~3_wvUN6XOl+&_p8FYN6XKi4)Fa@! zUduuC9&*{{081L-R&*g#4&zyUg`2$Rv3fg{<9$6_sZ|&H_ohlr90P9_lTOTY(rJBTC zF1Bm`)AZd@4*nhUZ`4E>_>O@wY2Nd07g38GG185D(Yh5m=N%8cZx$4f<;jY5#7&s9 zV_4yh%d)W7U)*DSZ#rvD^B^3qt0f9%ORO{*D2tu!kw5yxHRC{ptg*c^@|baFy1C}K zzitmdn`k%vZ^hx6`@Agdd+`kKi-*+^6w$@`S7|Ls*>0|Sb$&!vUEQQ3wsrK`_#I3u z@$Q3AdD4awahE5<{WnXXtlaDpbE)?%-ECCFIK`!DBx@FFoO1yL;>t|A^b~U%LRTbI^gt#elZI>#{tVL1K^S zi0x{R?VnB4oI*wahz0uSg4W9hUGt3oST@mY$XeU#YrnTWlK-AgmYDKSdFPLs( zNwrZyQD^6YaQ=NulIO^reD|o?gF{2nJB&@f)}Mr)pYfWgLCa7?o&E4FV|)7o0UmqM znB9`z^kz^3CKQTi1b88bky$ln6z?;6UM$D}6#D1ui}mO_9+ce98*l=8y5=XfghLw~ z1Sp49o<^{_7^p3^&Xsf-C&sU*xsZ%bvAh*>Fa{9;9qDi%{%SVZWB5c zwB%&iC{)JG2KuF}Q8^{9XHIP@{zMR={CV#N$?{$tz^7vDmGXL0RAa-WJI%e6a5>DaG5-6grG04bK5z;&_D)ivJU;d?sfekFF^|a# z-B>3hz{)G(*h#%+@KvM93)3)E3c?cxu$YS@Bo?_k&fl+~khX5_w{mw=D z>Gz1m;c3*Nxi%AA-%__$LGft;>GfWO^oNXCM!FlGSBO9x2-O~SHiU*T6d$=i z-aZTV+tCD;GinqOgq)1Gt>&OPE5y?sB*q&qgJuWydwsRm&_87PLL)N2WI@oPl?kOb zo#)UQ`W|X4kET{uqN`sE`LB#0Fjsns2uyA;c@sB)`80Dx=AMet15*wS>kb?IQ=WBN zNf6>c-<)>WVbNDzHO82^ml+khzsn4LTbcW5OXN})g~=cq+iH;BIBJ1vrs}(==ta38 zL|Eo79&X=}+~jt5!A%M83CG)82B|9~6Py`?J=3`Gp`mrdfC_ZXfjO7mRnGn76^D0A zYenoG5ig$g(tQxQ5xqkwK_E{PPDnE=hBkEKQN4=Djm+hW3p4(Zp3jXxpIS$7hrGOI z$av{ZUDu+Vnca+GFf{hCR$DO*XkkMJ-^szx z7o@r0>Z@o^>Rz-Zq>6v1+xs;SHdndAAtdDPu3Jz`k*-e9$xH1_u7CbL34Cl=%_Kz> zHYoC~e5y_EDg1tuaD&AFB)@&Bt@t}ArTyc>%~2;$wkg@{hrwL_E>(|8)zST<4g4En zw`4IgyTJG*;}c(~a<0icgy zKzD>a4Ah@Om0OhN3TZV;H;#s#nw*+q44}?_^8Sy@h#_TyBd{&@@o`4Jb$7#&N!zxg zJwBOaM%epCLq+^Co!Zy*$^-AT5s2j@n84v6p=Ks%nQFzTLD@gCBKgsBM*reVBKkg# zDDC81;-U}FEXuwEIwvW7ceXrj4@@2Fb(wl#Ijgbcy<@QpI^?&Df3?v#YvCLXHzQ&( z7>V;Dwnw%%h}8_}08rzs84_#ac8kaMp&F0X*CJi`ley!#UnFm!s=UpvW(uEyI(p=PYa z0@ts#3%H;W$q(zBYGowq^mNBGNeG~R+@>ut4`8X)SQ)j(0x3?I%xP==T+sIute;{Wx%6MHsdvZux zdeVEt(5y_;GWAlAjr8fH+)p@|{xlNW;zb(7&$Hfq&S{|f>}q7Qmm2eia;-`pZBaLG z2E4eR)=a47w`AmCbR=v1IMvzAM=b`r+Qdj#2ISgm4@llu{~QfW2wIHtvMRbu9m<88 ziw2bSw-qcsCL|Na=K()V?>!FO<5?c_jXJR-KfAy!zr^}mDa{leR zxS`d*v%?Mfs46Gju#l}fm{Q)v*CRuS!xfJeOw3B7LSTSVXZVgu+v(Dun~YDeUn-e4 zm-)LpkPgp`?}!mwbnu2>wkl2OJ2<{2;*}xp835S5X(pjqlqmLGuC{G%$4VMh zR6{hoF>Ks8+4hGMCCn@rD)DWV(U($pW>`Ig(fqTt`*vfmq+OGcg>k?AfYqWXXY@oL zLAmtl-*H>DJJPJczdrn?AyT~^w6nn+xqGW$RgED+8G=mD}EcUgyvg`A(G z$(NY8@n1aO=AIdv0}z0JzvX+i059Cy&k? zdj(Ud!)DQ`^%mA-nf*6wm15Q!;o>JQ!02u>RpM5)!q`uE|8-fWBv#p}O||N}w`PK| zk26tI?T&LM$lPXcKA62A5%_Udo6*@p-GS!(amaHbmh;1YG|REQX)a>@!KeJS$c9)Jel6oX*r(NTQ_y z3_Uf}VVo;! z&~7R~jwlRRB)4thmPJEFkAMpz6$n@w)g#OIZ+C#4uI8>KS!1lx*|2|!ep%x-JZlEU zlli{d2|6K}O+L z1N6|mOT4K@o>Uf4%1*d+JBg%XDs;Z#P*GX>gy4LLff**+13W9E1KMtty=fA4VpGy! z+{9=EmSC)Q!x>XUQWw% z(LwFMa8j#VgAsQ9et(@kq5f)kYGj+PN=>Wj)XJ~KS{n17Nl?NhoNLqLBxNONw@<3e zVVm^y7p-;-`@CpbUDjjTrxd|1o~#mVa*x&yf-Jca(fELJf7LGIp}&$oVvJSZRcu)E zbOT=p>|IJ)LX3db?Fbm}@Vuu^UKr}QkIN0+|1YS94310t*yXK+vnxBKA<<(1&9gK_rZb zv$dk3%F}>znV6xh0mq0uU75wH<+t|BZ-r-Cev#jo`*vL}D#?|XWJBIl4t8Vzbsk4Y z6b&25BatIzuq^wg=f^tg#<%dj|CK7mx+rN(Sw*!~7VzH?qCIBoe+JVAx)x<6bDI|K`3&^B4woRx z<42TS3v#{eTnka#wJc#c}sQ{indxNR6i_(Mtp6-)RS1 z)7kiUPx|2pl1%&Tn8VD=gai)`Bm$TA?$P!(j-f?3(~(8T_(f46^cmNKSmf7!t~VtC zz+`o5vL!Cx40GP!2s?$p`OTkIuSzl_sA9CD6xDYVET6hp7cG0QP<%R`EkBgA^ZG-d z%uR({M0^HI2^D|a(i;x+qOKgT>kG-iTDtfMH6g5b_WIEof%~6IJsMmos<+)4fw#x! zZw71SNWT}(y#Pl=sy0g9q6@D0gO_3ZdhcLIlA-4_CDMb#=pQ*m2I(4rtS-wSTg~s2 z515rihyujVvd3|*(mGxawQMNs@+|(w;*X)LCt>iw?T2brhy z_xcwdKZI33*kfLdVN?_$MVw%iRuVV&j;1{Y9Dqd@4Xvn9kVXAB2oZ6c&t5q90diw8 zJ1_Gy{tpOV)B=|QtfOe!l1|rJF}XnCKvaL|7>2<64rCKXT^iRr!z0C%01g-N#7T)I zwqLERva<-4MZ-cbSb~ym`Iz;OcD|es?eb&3V@0PqleMCmC24P1$V=VV+jfe`YcdoP zbO3)*?O0w?j#Nuxe@EJu_;>SMIF3PVNiWW1J3)hKF;-3}M(p10st%7Zt2{W)8X11K z(z#9T2|zsrt848|&Hf7YN++2%dmb2f_HL^%zB>sD>-<`yBKI$D*l&!xBhKQJKHS8= z8w%>a-uVe`wg=+qOuEdV2|o_DdMX`zfZj0=&G0Am9LqrYrX5r1 z|A7a&u_v0FGT+x9t>z7i*%r;_kjDXU!K@A7IWdGDildp06X{1PPF?ubq?zQCe2j8t zlilsV>kJLG>A_`z*4M7`={(S?${$IUOJP&}AZ^_DM|7{tS?eip{&(*ulWhs^d0+>!U+=<^IW6k-fUfC0YA)jN{yZP6)Rzq1e0y3TzME5zBu zmOhbJ;ON`P5Qc++Rg%z>ZY*mmiDdP5i@04DkxvDt*>~%*+l_Yf4z67z9gD?gy zWn^y9hBz3WG+*~A_tX5_TN1E6<@)$Yi}M## zv%m-|`abeFuJ%EZ_g|+q1;fC?s9X&5wEqQUh1IqBq4Mcz*<%X7*%Oz{MAm^gR$}e+ z;9A~}(H8_2(s(|xrL`fl^TlrMXS}6q$kCR!a6Nl(5R39`tvJetSZaKz%CcRAV8m`L z$BI-yHN}lMoJP6O;|eXTQjFmT!IzR4-BcVQiT44p9aL>Moz5QXI~@jfsoLzzOko(1 zawp9vv|#5&Ayrpue_?yKo9U+#4a~l_WDJ;aHJjeSH`avlJ)_PmiN)>rzkF?qAdvJJ zBwK4W^yRuSA;^#NWXpDMLy~NwC%qurz9e81 z9Rf6d_(uKnSv8m4;aL9xKA_1dI*uoqAU zlT3%T{OY{N&*-a=%+yv=_$gOkaperNqXd8T)05SuYmbSR*uaA=sM54D(41^3;5<~U zz-#E{W`naNLNb-PwR(FGv+S$m+ZimEOl>U}0}IWyU)HLuSDfsQzI|KPS@|i7XzVml z;M)x67c720nnf}|&+G;h7u5x84`1qB># z1GmRB5r7i2ljH?%O~}T4Ey2>BtEJ}sBO(*Z0?7cF|L|mr{w!P;-ytxL3lb_$_VX8x z)e~pUipRzgWZ<~qF8dZ&pLbCeD1Ix)ZL+S2{qaZ2Ly^=!w=;^1i+@Yl&AtVDipd{~kcLb*$d!PNF6yDa~@*Te8IX4L@v>D^(fTSne2Fw+3YD8 ztzG%kxg~G!luX!@WsB9%a~AN$0IVW)N(yPbBuF(`G1$DTkyVRaAq0?rgOMU*E(y z;BOR>&0q&f^^&s!6*wj^w_GUvn?Y@SQQz_pi z5Mj$WDxV_@C$y_TV1)Lx)EaHB21xdUsID%d7|%JH9&rf7{0pYhwy^LIiIR-9EPl_S zx#_C-Y8;nF<^VmL#eTPqdhd&lma@6jfg2{V6qzuFHP`l8+Fz$3y zUGdBs{-j6#TN8U2l*UTm=G$kOfh93|k+w*!%|LjpcRowD-7lYMc!0G;a=Ti)G+C;f z6n;3F*glec}iNS$BUBp(#)(ctC#@wxl39O7Cq5@C2hy%1;rkG3R3&vP!=BkS((2#Y9W3{p`Cf#0 z;}n)7gs9;n#H`KkJx1sB%7!$9Mo~{$t6#*>=3rCO>1la9hS2Vzw1(Ed$q7Rq{L1Hb$*kl zO4nSY2-b?4@k3Xda)QKzSk_tyId`WIX~eE_EgGa$9tShKCteQ23-J0`T5h-F#ju|t z?{xS4Jb_PEr_q=G&z?7RLgF@IkL9djTy0X`jn8(T=7k+i=Ard@qZOGGaG^^IBcBS| zV)33G^8EWDatN2&msA9hPQIUP={QSF&y42M6Eh@GtjzqbBA7pNhz0S&;7`!pXQRQQ zp=sq7yiLdgsAzd7NbcFa=r0UoC#gjK6x+PFC%knMC5Q*Es5MK86A zDA(pqYCuK-wRcZ11jO&IGkYIRhkZ8wf?*p#Z&4D!g|kyXU*vXP#&I}ANqU^l6=z=r z)??HTv z?15UJPXD^x@A$O+7Yp>qO=me zgftyqfr?*4b*-T;+qkn! z4D_9Tay)5}8pk+>i9UIYmfD;<&lg(q?h1wg;P>suI1O(AK4YFqAZn@9_H1twKH#!doso1 z2>T9}6FYl^$!Iw{RgdvuT}lkF3=TCXC)sb#^{^6+Td*KJt4IH67**%=27{I%H2gj0 zWJv$j@8}H_;rSbJ87$rKiRQ9e?J?B<{$LHX)2$?w6|ucm^jY&Y1N^^DOe2!*E@WqE z=z!wXJL{Y59Aw`a2Z>dlEiG>naM(*$2MNF^aa&N)+VLW^X4X_7MfGXLsvzh;P%Q!C zTgWhKil6b#I5N=$@JTAxt$z13Z$fk#Jt?jiyZo%ScClYiBPTC^1}vG#?ppZ!XK9A< z+77e*GS)A2Ns!}Q(7J)MY*Nl~loO>^1LY* zzI?c&&wx{yb$TEq-yrAw z^Y?Kv`}SH>mZDY9V#Gly`;{?sRboq;S^w{A=Zda=seB*Lv3tR$bCJ(kzOCC*DtyS^ zO7IYS{&~6dmtH^j%;i5fjUmDmWQ_?6+ZV4D%Yv`k7FKMtuH@agsqW9ATi+#a?BALD zOspIWl%U)cqjW^mRt1RaF1_WgUgmHXZH zzK&2o|E!7)?DXUm;OPlb_il35=DdA>{f^CTpYyRdL0O`ACga}4NuhHO*d5v6r?LMX za6|d4ywy)Z366!iil0Yyjr1vyx|g4LUf32td!n;peqHgm`?>R7=QSH&Uw>cp(6hIb z?eoon+Yo>U5kvtMxkvOb{QE~;agMFzw5AVxQ-zLf&oQ0=k`5AP`_i~@QQfl;j(DGc zHr213rvBX=nf+%c>+Q#!cjAFNu8V~Z?KfQ67r(VMO!fYr+;7W)&QMut$^7etzytgL zC1JV!9G_c$WE`HV7*n|L*WSX{Z~U2efYe+~g`E*X^8-fKflJpYJ7&;U{*iP#$q)Mk YiMSP<@2pM(kC|ccboFyt=akR{01B1@?*IS* From 47a3277d121a175abf67a2575e49dc5327a6b58f Mon Sep 17 00:00:00 2001 From: JohnsonLee <0xjohnsonlee@gmail.com> Date: Fri, 4 Oct 2024 17:26:57 +0800 Subject: [PATCH 053/128] feat: customize channel information for sqlness tests (#4729) * feat: add pg and mysql server_address options * feat: start pg and mysql server(standalone) * feat: start pg and mysql in distribute * feat: finally get there, specify postgres sqlness * feat: support mysql sqlness * fix: license * fix: remove unused import * fix: toml * fix: clippy * refactor: BeginProtocolInterceptorFactory to ProtocolInterceptorFactory * fix: sqlness pg connect * fix: clippy * Apply suggestions from code review Co-authored-by: Yingwen * fix: rustfmt * fix: reconnect pg and mysql when restart * test: add mysql related sqlness * fix: wait for start while restarting * fix: clippy * fix: cargo lock conflict fix: Cargo.lock conflict * fix: usage of '@@tx_isolation' in sqlness * fix: typos * feat: retry with backoff when create client * fix: use millisecond rather than microseconds in backoff --------- Co-authored-by: Yingwen --- Cargo.lock | 797 ++++++++++++------ src/log-store/src/kafka/client_manager.rs | 2 +- .../src/python/rspython/builtins/test.rs | 2 +- tests/cases/standalone/common/mysql.result | 29 + tests/cases/standalone/common/mysql.sql | 8 + .../common/system/pg_catalog.result | 140 ++- .../standalone/common/system/pg_catalog.sql | 28 +- tests/conf/standalone-test.toml.template | 13 + tests/runner/Cargo.toml | 3 + tests/runner/src/env.rs | 381 +++++++-- tests/runner/src/main.rs | 40 +- tests/runner/src/protocol_interceptor.rs | 48 ++ 12 files changed, 1125 insertions(+), 366 deletions(-) create mode 100644 tests/cases/standalone/common/mysql.result create mode 100644 tests/cases/standalone/common/mysql.sql create mode 100644 tests/runner/src/protocol_interceptor.rs diff --git a/Cargo.lock b/Cargo.lock index ab5043939a..418e68a88a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -226,7 +226,7 @@ dependencies = [ "paste", "prost 0.12.6", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "tonic-build", ] @@ -459,7 +459,7 @@ dependencies = [ "arrow-schema", "chrono", "half 2.4.1", - "indexmap 2.5.0", + "indexmap 2.6.0", "lexical-core", "num", "serde", @@ -534,7 +534,7 @@ dependencies = [ "memchr", "num", "regex", - "regex-syntax 0.8.4", + "regex-syntax 0.8.5", ] [[package]] @@ -607,11 +607,11 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fec134f64e2bc57411226dfc4e52dec859ddfc7e711fc5e07b612584f000e4aa" +checksum = "7e614738943d3f68c628ae3dbce7c3daffb196665f82f8c8ea6b65de73c79429" dependencies = [ - "brotli 6.0.0", + "brotli 7.0.0", "bzip2", "flate2", "futures-core", @@ -654,14 +654,14 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] name = "async-stream" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd56dd203fef61ac097dd65721a419ddccb106b2d2b70ba60a6b529f03961a51" +checksum = "0b5a71a6f37880a80d1d7f19efd781e4b5de42c88f0722cc13bcb6cc2cfe8476" dependencies = [ "async-stream-impl", "futures-core", @@ -670,13 +670,13 @@ dependencies = [ [[package]] name = "async-stream-impl" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" +checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -687,13 +687,13 @@ checksum = "8b75356056920673b02621b35afd0f7dda9306d03c79a30f5c56c44cf256e3de" [[package]] name = "async-trait" -version = "0.1.82" +version = "0.1.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" +checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -778,7 +778,7 @@ dependencies = [ "digest", "notify", "sha1", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "tokio", ] @@ -791,7 +791,7 @@ checksum = "3c87f3f15e7794432337fc718554eaa4dc8f04c9677a950ffe366f20a162ae42" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -802,9 +802,9 @@ checksum = "7460f7dd8e100147b82a63afca1a20eb6c231ee36b90ba7272e14951cb58af59" [[package]] name = "autocfg" -version = "1.3.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" +checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "autotools" @@ -890,7 +890,7 @@ dependencies = [ "heck 0.4.1", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -976,6 +976,17 @@ dependencies = [ "smallvec", ] +[[package]] +name = "bigdecimal" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6773ddc0eafc0e509fb60e48dff7f450f8e674a0686ae8605e8d9901bd5eefa" +dependencies = [ + "num-bigint", + "num-integer", + "num-traits", +] + [[package]] name = "bigdecimal" version = "0.4.5" @@ -998,6 +1009,25 @@ dependencies = [ "serde", ] +[[package]] +name = "bindgen" +version = "0.59.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2bd2a9a458e8f4304c52c43ebb0cfbd520289f8379a52e329a38afda99bf8eb8" +dependencies = [ + "bitflags 1.3.2", + "cexpr", + "clang-sys", + "lazy_static", + "lazycell", + "peeking_take_while", + "proc-macro2", + "quote", + "regex", + "rustc-hash 1.1.0", + "shlex", +] + [[package]] name = "bindgen" version = "0.69.4" @@ -1017,7 +1047,7 @@ dependencies = [ "regex", "rustc-hash 1.1.0", "shlex", - "syn 2.0.77", + "syn 2.0.79", "which", ] @@ -1036,7 +1066,7 @@ dependencies = [ "regex", "rustc-hash 1.1.0", "shlex", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -1160,7 +1190,7 @@ dependencies = [ "proc-macro-crate 3.2.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", "syn_derive", ] @@ -1177,9 +1207,9 @@ dependencies = [ [[package]] name = "brotli" -version = "6.0.0" +version = "7.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74f7971dbd9326d58187408ab83117d8ac1bb9c17b085fdacd1cf2f598719b6b" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -1224,7 +1254,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "40723b8fb387abc38f4f4a37c09073622e41dd12327033091ef8950659e6dc0c" dependencies = [ "memchr", - "regex-automata 0.4.7", + "regex-automata 0.4.8", "serde", ] @@ -1237,6 +1267,12 @@ dependencies = [ "num-traits", ] +[[package]] +name = "bufstream" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40e38929add23cdf8a366df9b0e088953150724bcbe5fc330b0d8eb3b328eec8" + [[package]] name = "build-data" version = "0.2.1" @@ -1346,7 +1382,7 @@ dependencies = [ "common-macro", "common-meta", "moka", - "snafu 0.8.4", + "snafu 0.8.5", "substrait 0.9.3", ] @@ -1416,7 +1452,7 @@ dependencies = [ "rustc-hash 2.0.0", "serde_json", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "store-api", "table", @@ -1435,9 +1471,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.21" +version = "1.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07b1695e2c7e8fc85310cde85aeaab7e3097f593c91d209d3f9df76c928100f0" +checksum = "812acba72f0a070b003d3697490d2b55b837230ae7c6c6497f05cc2ddbb8d938" dependencies = [ "jobserver", "libc", @@ -1504,7 +1540,7 @@ version = "0.13.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6026d8cd82ada8bbcfe337805dd1eb6afdc9e80fa4d57e977b3a36315e0c5525" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", "lazy_static", "num-traits", "regex", @@ -1654,9 +1690,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.18" +version = "4.5.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0956a43b323ac1afaffc053ed5c4b7c1f1800bacd1683c353aabbb752515dd3" +checksum = "7be5744db7978a28d9df86a214130d106a89ce49644cbc4e3f0c22c3fba30615" dependencies = [ "clap_builder", "clap_derive", @@ -1664,9 +1700,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.18" +version = "4.5.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d72166dd41634086d5803a47eb71ae740e61d84709c36f3c34110173db3961b" +checksum = "a5fbc17d3ef8278f55b282b2a2e75ae6f6c7d4bb70ed3d0382375104bfafdb4b" dependencies = [ "anstream", "anstyle", @@ -1683,7 +1719,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -1731,7 +1767,7 @@ dependencies = [ "query", "rand", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "substrait 0.37.3", "substrait 0.9.3", "tokio", @@ -1771,7 +1807,7 @@ dependencies = [ "cache", "catalog", "chrono", - "clap 4.5.18", + "clap 4.5.19", "client", "common-base", "common-catalog", @@ -1818,7 +1854,7 @@ dependencies = [ "servers", "session", "similar-asserts", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "substrait 0.9.3", "table", @@ -1877,7 +1913,7 @@ dependencies = [ "futures", "paste", "serde", - "snafu 0.8.4", + "snafu 0.8.5", "toml 0.8.19", "zeroize", ] @@ -1889,7 +1925,7 @@ dependencies = [ "chrono", "common-error", "common-macro", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", ] @@ -1909,7 +1945,7 @@ dependencies = [ "num_cpus", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "sysinfo", "temp-env", "tempfile", @@ -1945,7 +1981,7 @@ dependencies = [ "rand", "regex", "serde", - "snafu 0.8.4", + "snafu 0.8.5", "strum 0.25.0", "tokio", "tokio-util", @@ -1957,20 +1993,20 @@ dependencies = [ name = "common-decimal" version = "0.9.3" dependencies = [ - "bigdecimal", + "bigdecimal 0.4.5", "common-error", "common-macro", "rust_decimal", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", ] [[package]] name = "common-error" version = "0.9.3" dependencies = [ - "snafu 0.8.4", + "snafu 0.8.5", "strum 0.25.0", "tonic 0.11.0", ] @@ -1986,7 +2022,7 @@ dependencies = [ "common-macro", "common-query", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", ] @@ -2021,7 +2057,7 @@ dependencies = [ "serde", "serde_json", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "statrs", "store-api", @@ -2066,7 +2102,7 @@ dependencies = [ "lazy_static", "prost 0.12.6", "rand", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", "tonic 0.11.0", "tower", @@ -2086,7 +2122,7 @@ dependencies = [ "datatypes", "paste", "prost 0.12.6", - "snafu 0.8.4", + "snafu 0.8.5", "table", ] @@ -2099,9 +2135,9 @@ dependencies = [ "datatypes", "proc-macro2", "quote", - "snafu 0.8.4", + "snafu 0.8.5", "static_assertions", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -2110,7 +2146,7 @@ version = "0.9.3" dependencies = [ "common-error", "common-macro", - "snafu 0.8.4", + "snafu 0.8.5", "tempfile", "tikv-jemalloc-ctl", "tikv-jemalloc-sys", @@ -2163,7 +2199,7 @@ dependencies = [ "serde_json", "serde_with", "session", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "strum 0.25.0", "table", @@ -2200,7 +2236,7 @@ dependencies = [ "serde", "serde_json", "smallvec", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", "uuid", ] @@ -2231,7 +2267,7 @@ dependencies = [ "datatypes", "futures-util", "serde", - "snafu 0.8.4", + "snafu 0.8.5", "sqlparser 0.45.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09b11c0c88934690530807185d3e7)", "sqlparser_derive 0.1.1", "statrs", @@ -2254,7 +2290,7 @@ dependencies = [ "pin-project", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", ] @@ -2272,7 +2308,7 @@ dependencies = [ "paste", "prometheus", "serde", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", "tokio-metrics", "tokio-metrics-collector", @@ -2333,7 +2369,7 @@ dependencies = [ "rand", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", ] [[package]] @@ -2361,11 +2397,11 @@ dependencies = [ "rskafka", "rustls 0.23.13", "rustls-native-certs 0.7.3", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "serde", "serde_json", "serde_with", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", "toml 0.8.19", ] @@ -2624,7 +2660,7 @@ dependencies = [ "anes", "cast", "ciborium", - "clap 4.5.18", + "clap 4.5.19", "criterion-plot", "is-terminal", "itertools 0.10.5", @@ -2809,7 +2845,7 @@ dependencies = [ "proc-macro2", "quote", "strsim 0.11.1", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -2831,7 +2867,7 @@ checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core 0.20.10", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -2863,7 +2899,7 @@ dependencies = [ "arrow-array", "arrow-ipc", "arrow-schema", - "async-compression 0.4.12", + "async-compression 0.4.13", "async-trait", "bytes", "bzip2", @@ -2886,7 +2922,7 @@ dependencies = [ "glob", "half 2.4.1", "hashbrown 0.14.5", - "indexmap 2.5.0", + "indexmap 2.6.0", "itertools 0.12.1", "log", "num_cpus", @@ -3045,10 +3081,10 @@ dependencies = [ "datafusion-expr", "datafusion-physical-expr", "hashbrown 0.14.5", - "indexmap 2.5.0", + "indexmap 2.6.0", "itertools 0.12.1", "log", - "regex-syntax 0.8.4", + "regex-syntax 0.8.5", ] [[package]] @@ -3073,7 +3109,7 @@ dependencies = [ "half 2.4.1", "hashbrown 0.14.5", "hex", - "indexmap 2.5.0", + "indexmap 2.6.0", "itertools 0.12.1", "log", "paste", @@ -3115,7 +3151,7 @@ dependencies = [ "futures", "half 2.4.1", "hashbrown 0.14.5", - "indexmap 2.5.0", + "indexmap 2.6.0", "itertools 0.12.1", "log", "once_cell", @@ -3206,7 +3242,7 @@ dependencies = [ "serde_json", "servers", "session", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "substrait 0.9.3", "table", @@ -3239,7 +3275,7 @@ dependencies = [ "paste", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", ] [[package]] @@ -3322,7 +3358,7 @@ checksum = "2cdc8d50f426189eef89dac62fabfa0abb27d5cc008f25bf4156a0203325becc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -3333,7 +3369,7 @@ checksum = "67e77553c4162a157adbf834ebae5b415acbecbeafc7a74b0e886657506a7611" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -3396,7 +3432,7 @@ dependencies = [ "darling 0.20.10", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -3426,7 +3462,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4abae7035bf79b9877b779505d8cf3749285b80c43941eda66604841889451dc" dependencies = [ "derive_builder_core 0.20.1", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -3446,10 +3482,21 @@ checksum = "cb7330aeadfbe296029522e6c40f315320aba36fc43a5b3632f3795348f3bd22" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", "unicode-xid", ] +[[package]] +name = "derive_utils" +version = "0.14.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "65f152f4b8559c4da5d574bafc7af85454d706b4c5fe8b530d508cacbb6807ea" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "diff" version = "0.1.13" @@ -3604,7 +3651,7 @@ dependencies = [ "serde", "thiserror", "time", - "winnow 0.6.18", + "winnow 0.6.20", ] [[package]] @@ -3658,7 +3705,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -3670,7 +3717,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -3834,7 +3881,7 @@ dependencies = [ "object-store", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "table", "tokio", @@ -3888,9 +3935,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.33" +version = "1.0.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "324a1be68054ef05ad64b861cc9eaf1d623d2d8cb25b4bf2cb9cdd902b4bf253" +checksum = "a1b589b4dc103969ad3cf85c950899926ec64300a1a46d76c03a6072957036f0" dependencies = [ "crc32fast", "libz-sys", @@ -3968,7 +4015,7 @@ dependencies = [ "servers", "session", "smallvec", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "strum 0.25.0", "substrait 0.9.3", @@ -3992,6 +4039,21 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" +[[package]] +name = "foreign-types" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6f339eb8adc052cd2ca78910fda869aefa38d22d5cb648e6485e4d3fc06f3b1" +dependencies = [ + "foreign-types-shared", +] + +[[package]] +name = "foreign-types-shared" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" + [[package]] name = "form_urlencoded" version = "1.2.1" @@ -4067,7 +4129,7 @@ dependencies = [ "serde_json", "servers", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "sqlparser 0.45.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09b11c0c88934690530807185d3e7)", "store-api", @@ -4109,7 +4171,7 @@ checksum = "e99b8b3c28ae0e84b604c75f721c21dc77afb3706076af5e8216d15fd1deaae3" dependencies = [ "frunk_proc_macro_helpers", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -4121,7 +4183,7 @@ dependencies = [ "frunk_core", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -4133,7 +4195,7 @@ dependencies = [ "frunk_core", "frunk_proc_macro_helpers", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -4263,7 +4325,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -4437,7 +4499,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.5.0", + "indexmap 2.6.0", "slab", "tokio", "tokio-util", @@ -4499,6 +4561,12 @@ dependencies = [ "allocator-api2", ] +[[package]] +name = "hashbrown" +version = "0.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e087f84d4f86bf4b218b927129862374b72199ae7d8657835f1e89000eea4fb" + [[package]] name = "hashlink" version = "0.8.4" @@ -4734,9 +4802,9 @@ checksum = "add0ab9360ddbd88cfeb3bd9574a1d85cfdfa14db10b3e21d3700dbc4328758f" [[package]] name = "httparse" -version = "1.9.4" +version = "1.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fcc0b4a115bf80b728eb8ea024ad5bd707b615bfed49e0665b6e0f86fd082d9" +checksum = "7d71d3574edd2771538b901e6549113b4006ece66150fb69c0fb6d9a2adae946" [[package]] name = "httpdate" @@ -4817,7 +4885,7 @@ dependencies = [ "proc-macro-crate 1.3.1", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -4832,7 +4900,7 @@ dependencies = [ "rust-sitter", "rust-sitter-tool", "slotmap", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -4841,7 +4909,7 @@ version = "0.6.2" source = "git+https://github.com/GreptimeTeam/hydroflow.git?branch=main#b072ee026f97f8537165e1fb247101e0ab2fb320" dependencies = [ "auto_impl", - "clap 4.5.18", + "clap 4.5.19", "data-encoding", "itertools 0.10.5", "prettyplease", @@ -4851,7 +4919,7 @@ dependencies = [ "serde", "serde_json", "slotmap", - "syn 2.0.77", + "syn 2.0.79", "webbrowser", ] @@ -4865,7 +4933,7 @@ dependencies = [ "proc-macro-crate 1.3.1", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -4977,9 +5045,9 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.8" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da62f120a8a37763efb0cf8fdf264b884c7b8b9ac8660b900c8661030c00e6ba" +checksum = "41296eb09f183ac68eec06e03cdbea2e759633d4067b2f6552fc2e009bcad08b" dependencies = [ "bytes", "futures-channel", @@ -4990,7 +5058,6 @@ dependencies = [ "pin-project-lite", "socket2 0.5.7", "tokio", - "tower", "tower-service", "tracing", ] @@ -5081,9 +5148,9 @@ dependencies = [ "prost 0.12.6", "rand", "regex", - "regex-automata 0.4.7", + "regex-automata 0.4.8", "serde", - "snafu 0.8.4", + "snafu 0.8.5", "tantivy", "tantivy-jieba", "tempfile", @@ -5105,12 +5172,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" +checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" dependencies = [ "equivalent", - "hashbrown 0.14.5", + "hashbrown 0.15.0", "serde", ] @@ -5127,7 +5194,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" dependencies = [ "ahash 0.8.11", - "indexmap 2.5.0", + "indexmap 2.6.0", "is-terminal", "itoa", "log", @@ -5209,6 +5276,15 @@ version = "0.3.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f958d3d68f4167080a18141e10381e7634563984a537f2a49a30fd8e53ac5767" +[[package]] +name = "io-enum" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53b53d712d99a73eec59ee5e4fe6057f8052142d38eeafbbffcb06b36d738a6e" +dependencies = [ + "derive_utils", +] + [[package]] name = "ipnet" version = "2.10.0" @@ -5217,9 +5293,9 @@ checksum = "187674a687eed5fe42285b40c6291f9a01517d415fad1c3cbc6a9f778af7fcd4" [[package]] name = "iri-string" -version = "0.7.5" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c25163201be6ded9e686703e85532f8f852ea1f92ba625cb3c51f7fe6d07a4a" +checksum = "44bd7eced44cfe2cebc674adb2a7124a754a4b5269288d22e9f39f8fada3562d" dependencies = [ "memchr", "serde", @@ -5399,7 +5475,7 @@ dependencies = [ "fast-float", "itoa", "nom", - "ordered-float 4.2.2", + "ordered-float 4.3.0", "rand", "ryu", "serde_json", @@ -5439,7 +5515,7 @@ checksum = "b9ae10193d25051e74945f1ea2d0b42e03cc3b890f7e4cc5faa44997d808193f" dependencies = [ "base64 0.21.7", "js-sys", - "pem", + "pem 3.0.4", "ring 0.17.8", "serde", "serde_json", @@ -5474,7 +5550,7 @@ version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ee7893dab2e44ae5f9d0173f26ff4aa327c10b01b06a72b52dd9405b628640d" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", ] [[package]] @@ -5533,9 +5609,9 @@ dependencies = [ "jsonpath-rust", "k8s-openapi", "kube-core", - "pem", + "pem 3.0.4", "rustls 0.23.13", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "secrecy", "serde", "serde_json", @@ -5575,7 +5651,7 @@ dependencies = [ "proc-macro2", "quote", "serde_json", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -5667,7 +5743,7 @@ dependencies = [ "proc-macro2", "quote", "regex", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -5691,6 +5767,15 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c2cdeb66e45e9f36bfad5bbdb4d2384e70936afbee843c6f6543f0c551ebb25" +[[package]] +name = "lexical" +version = "6.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7aefb36fd43fef7003334742cbf77b243fcd36418a1d1bdd480d613a67968f6" +dependencies = [ + "lexical-core", +] + [[package]] name = "lexical-core" version = "0.8.5" @@ -5757,9 +5842,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.158" +version = "0.2.159" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8adc4bb1803a324070e64a98ae98f38934d91957a99cfb3a43dcbc01bc56439" +checksum = "561d97a539a36e26a9a5fad1ea11a3039a67714694aaa379433e580854bc3dc5" [[package]] name = "libfuzzer-sys" @@ -5818,7 +5903,7 @@ checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ "bitflags 2.6.0", "libc", - "redox_syscall 0.5.4", + "redox_syscall 0.5.7", ] [[package]] @@ -5906,7 +5991,7 @@ dependencies = [ "rskafka", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "tokio", "uuid", @@ -5925,7 +6010,7 @@ dependencies = [ "num-traits", "quote", "regex", - "regex-syntax 0.8.4", + "regex-syntax 0.8.5", "serde", "vergen", ] @@ -5940,7 +6025,7 @@ dependencies = [ "cactus", "cfgrammar", "filetime", - "indexmap 2.5.0", + "indexmap 2.6.0", "lazy_static", "lrtable", "num-traits", @@ -5966,6 +6051,15 @@ dependencies = [ "vob", ] +[[package]] +name = "lru" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6e8aaa3f231bb4bd57b84b2d5dc3ae7f350265df8aa96492e0bc394a1571909" +dependencies = [ + "hashbrown 0.12.3", +] + [[package]] name = "lru" version = "0.12.4" @@ -5977,18 +6071,18 @@ dependencies = [ [[package]] name = "lz4" -version = "1.27.0" +version = "1.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a231296ca742e418c43660cb68e082486ff2538e8db432bc818580f3965025ed" +checksum = "4d1febb2b4a79ddd1980eede06a8f7902197960aa0383ffcfdd62fe723036725" dependencies = [ "lz4-sys", ] [[package]] name = "lz4-sys" -version = "1.11.0" +version = "1.11.1+lz4-1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb44a01837a858d47e5a630d2ccf304c8efcc4b83b8f9f75b7a9ee4fcc6e57d" +checksum = "6bd8c0d6c6ed0cd30b3652886bb8711dc4bb01d637a68105a3d5158039b418e6" dependencies = [ "cc", "libc", @@ -6200,7 +6294,7 @@ dependencies = [ "meta-srv", "rand", "serde", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", "tokio-stream", "tonic 0.11.0", @@ -6216,7 +6310,7 @@ dependencies = [ "api", "async-trait", "chrono", - "clap 4.5.18", + "clap 4.5.19", "client", "common-base", "common-catalog", @@ -6254,7 +6348,7 @@ dependencies = [ "serde_json", "servers", "session", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "table", "tokio", @@ -6311,7 +6405,7 @@ dependencies = [ "object-store", "prometheus", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "tokio", ] @@ -6449,7 +6543,7 @@ dependencies = [ "serde_with", "session", "smallvec", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "strum 0.25.0", "table", @@ -6556,6 +6650,32 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2195bf6aa996a481483b29d62a7663eed3fe39600c460e323f8ff41e90bdd89b" +[[package]] +name = "mysql" +version = "23.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f11339ca5c251941805d51362a07823605a80586ced92914ab7de84fba813f" +dependencies = [ + "bufstream", + "bytes", + "crossbeam", + "flate2", + "io-enum", + "libc", + "lru 0.8.1", + "mysql_common 0.29.2", + "named_pipe", + "native-tls", + "once_cell", + "pem 1.1.1", + "percent-encoding", + "serde", + "serde_json", + "socket2 0.4.10", + "twox-hash", + "url", +] + [[package]] name = "mysql-common-derive" version = "0.30.2" @@ -6569,7 +6689,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", "termcolor", "thiserror", ] @@ -6587,7 +6707,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", "termcolor", "thiserror", ] @@ -6606,11 +6726,11 @@ dependencies = [ "futures-util", "keyed_priority_queue", "lazy_static", - "lru", + "lru 0.12.4", "mio 0.8.11", "mysql_common 0.31.0", "once_cell", - "pem", + "pem 3.0.4", "percent-encoding", "pin-project", "rand", @@ -6629,6 +6749,43 @@ dependencies = [ "webpki-roots 0.25.4", ] +[[package]] +name = "mysql_common" +version = "0.29.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9006c95034ccf7b903d955f210469119f6c3477fc9c9e7a7845ce38a3e665c2a" +dependencies = [ + "base64 0.13.1", + "bigdecimal 0.3.1", + "bindgen 0.59.2", + "bitflags 1.3.2", + "bitvec", + "byteorder", + "bytes", + "cc", + "cmake", + "crc32fast", + "flate2", + "frunk", + "lazy_static", + "lexical", + "num-bigint", + "num-traits", + "rand", + "regex", + "rust_decimal", + "saturating", + "serde", + "serde_json", + "sha1", + "sha2", + "smallvec", + "subprocess", + "thiserror", + "time", + "uuid", +] + [[package]] name = "mysql_common" version = "0.31.0" @@ -6636,7 +6793,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "06f19e4cfa0ab5a76b627cec2d81331c49b034988eaf302c3bafeada684eadef" dependencies = [ "base64 0.21.7", - "bigdecimal", + "bigdecimal 0.4.5", "bindgen 0.70.1", "bitflags 2.6.0", "bitvec", @@ -6675,7 +6832,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "478b0ff3f7d67b79da2b96f56f334431aef65e15ba4b29dd74a4236e29582bdc" dependencies = [ "base64 0.21.7", - "bigdecimal", + "bigdecimal 0.4.5", "bindgen 0.70.1", "bitflags 2.6.0", "bitvec", @@ -6737,6 +6894,32 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "named_pipe" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad9c443cce91fc3e12f017290db75dde490d685cdaaf508d7159d7cf41f0eb2b" +dependencies = [ + "winapi", +] + +[[package]] +name = "native-tls" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8614eb2c83d59d1c8cc974dd3f920198647674a0a035e1af1fa58707e317466" +dependencies = [ + "libc", + "log", + "openssl", + "openssl-probe", + "openssl-sys", + "schannel", + "security-framework", + "security-framework-sys", + "tempfile", +] + [[package]] name = "ndk-context" version = "0.1.1" @@ -6916,7 +7099,7 @@ checksum = "ed3955f1a9c7c0c15e092f9c887db08b1fc683305fdf6eb6684f22555355e202" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -7078,9 +7261,12 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.19.0" +version = "1.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" +checksum = "82881c4be219ab5faaf2ad5e5e5ecdff8c66bd7402ca3160975c93b24961afd1" +dependencies = [ + "portable-atomic", +] [[package]] name = "oneshot" @@ -7153,12 +7339,50 @@ dependencies = [ "tokio-rustls 0.26.0", ] +[[package]] +name = "openssl" +version = "0.10.66" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9529f4786b70a3e8c61e11179af17ab6188ad8d0ded78c5529441ed39d4bd9c1" +dependencies = [ + "bitflags 2.6.0", + "cfg-if", + "foreign-types", + "libc", + "once_cell", + "openssl-macros", + "openssl-sys", +] + +[[package]] +name = "openssl-macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.79", +] + [[package]] name = "openssl-probe" version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" +[[package]] +name = "openssl-sys" +version = "0.9.103" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f9e8deee91df40a943c71b917e5874b951d32a802526c85721ce3b776c929d6" +dependencies = [ + "cc", + "libc", + "pkg-config", + "vcpkg", +] + [[package]] name = "opentelemetry" version = "0.21.0" @@ -7167,7 +7391,7 @@ checksum = "1e32339a5dc40459130b3bd269e9892439f55b33e772d2a9d402a789baaf4e8a" dependencies = [ "futures-core", "futures-sink", - "indexmap 2.5.0", + "indexmap 2.6.0", "js-sys", "once_cell", "pin-project-lite", @@ -7258,7 +7482,7 @@ dependencies = [ "glob", "once_cell", "opentelemetry 0.21.0", - "ordered-float 4.2.2", + "ordered-float 4.3.0", "percent-encoding", "rand", "thiserror", @@ -7280,7 +7504,7 @@ dependencies = [ "glob", "once_cell", "opentelemetry 0.22.0", - "ordered-float 4.2.2", + "ordered-float 4.3.0", "percent-encoding", "rand", "thiserror", @@ -7329,7 +7553,7 @@ dependencies = [ "regex", "serde_json", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "sqlparser 0.45.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09b11c0c88934690530807185d3e7)", "store-api", @@ -7392,9 +7616,9 @@ dependencies = [ [[package]] name = "ordered-float" -version = "4.2.2" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a91171844676f8c7990ce64959210cd2eaef32c2612c50f9fae9f8aaa6065a6" +checksum = "44d501f1a72f71d3c063a6bbc8f7271fa73aa09fe5d6283b6571e2ed176a2537" dependencies = [ "num-traits", ] @@ -7531,7 +7755,7 @@ dependencies = [ "cfg-if", "libc", "petgraph", - "redox_syscall 0.5.4", + "redox_syscall 0.5.7", "smallvec", "thread-id", "windows-targets 0.52.6", @@ -7597,7 +7821,7 @@ dependencies = [ "itertools 0.10.5", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "sqlparser 0.45.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09b11c0c88934690530807185d3e7)", "store-api", @@ -7669,6 +7893,21 @@ dependencies = [ "hmac", ] +[[package]] +name = "peeking_take_while" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19b17cddbe7ec3f8bc800887bab5e717348c95ea2ca0b1bf0837fb964dc67099" + +[[package]] +name = "pem" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8835c273a76a90455d7344889b0964598e3316e2a79ede8e36f16bdcf2228b8" +dependencies = [ + "base64 0.13.1", +] + [[package]] name = "pem" version = "3.0.4" @@ -7740,7 +7979,7 @@ dependencies = [ "pest_meta", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -7761,7 +8000,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset", - "indexmap 2.5.0", + "indexmap 2.6.0", ] [[package]] @@ -7852,7 +8091,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -7915,7 +8154,7 @@ dependencies = [ "serde", "serde_json", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "table", "tokio", @@ -7996,9 +8235,9 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231b230927b5e4ad203db57bbcbee2802f6bce620b1e4a9024a07d94e2907ec" +checksum = "953ec861398dccce10c670dfeaf3ec4911ca479e9c02154b3a215178c5f566f2" [[package]] name = "plotters" @@ -8037,7 +8276,7 @@ dependencies = [ "datanode", "frontend", "meta-srv", - "snafu 0.8.4", + "snafu 0.8.5", ] [[package]] @@ -8053,9 +8292,9 @@ dependencies = [ [[package]] name = "portable-atomic" -version = "1.8.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d30538d42559de6b034bc76fd6dd4c38961b1ee5c6c56e3808c50128fdbc22ce" +checksum = "cc9c68a3f6da06753e9335d63e27f6b9754dd1920d941135b7ea8224f141adb2" [[package]] name = "postgres-protocol" @@ -8195,7 +8434,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "479cf940fbbb3426c32c5d5176f62ad57549a0bb84773423ba8be9d089f5faba" dependencies = [ "proc-macro2", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -8214,7 +8453,7 @@ version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8ecf48c7ca261d60b74ab1a7b20da18bede46776b2e55535cb958eb595c5fa7b" dependencies = [ - "toml_edit 0.22.21", + "toml_edit 0.22.22", ] [[package]] @@ -8324,7 +8563,7 @@ dependencies = [ "prost 0.12.6", "query", "session", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", ] @@ -8388,7 +8627,7 @@ dependencies = [ "prost 0.12.6", "prost-types 0.12.6", "regex", - "syn 2.0.77", + "syn 2.0.79", "tempfile", ] @@ -8434,7 +8673,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -8447,7 +8686,7 @@ dependencies = [ "itertools 0.13.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -8541,7 +8780,7 @@ dependencies = [ name = "puffin" version = "0.9.3" dependencies = [ - "async-compression 0.4.12", + "async-compression 0.4.13", "async-trait", "async-walkdir", "auto_impl", @@ -8560,7 +8799,7 @@ dependencies = [ "serde", "serde_json", "sha2", - "snafu 0.8.4", + "snafu 0.8.5", "tokio", "tokio-util", "uuid", @@ -8628,7 +8867,7 @@ dependencies = [ "proc-macro2", "pyo3-macros-backend", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -8641,7 +8880,7 @@ dependencies = [ "proc-macro2", "pyo3-build-config", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -8717,7 +8956,7 @@ dependencies = [ "rand", "regex", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "sqlparser 0.45.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09b11c0c88934690530807185d3e7)", "statrs", @@ -8909,9 +9148,9 @@ dependencies = [ [[package]] name = "raw-cpuid" -version = "11.1.0" +version = "11.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb9ee317cfe3fbd54b36a511efc1edd42e216903c9cd575e686dd68a2ba90d8d" +checksum = "1ab240315c661615f2ee9f0f2cd32d5a7343a84d5ebcccb99d46e6637565e7b0" dependencies = [ "bitflags 2.6.0", ] @@ -8959,9 +9198,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.4" +version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0884ad60e090bf1345b93da0a5de8923c93884cd03f40dfcfddd3b4bee661853" +checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" dependencies = [ "bitflags 2.6.0", ] @@ -8994,19 +9233,19 @@ checksum = "bcc303e793d3734489387d205e9b186fac9c6cfacedd98cbb2e8a5943595f3e6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] name = "regex" -version = "1.10.6" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4219d74c6b67a3654a9fbebc4b419e22126d13d2f3c4a07ee0cb61ff79a79619" +checksum = "38200e5ee88914975b69f657f0801b6f6dccafd44fd9326302a4aaeecfacb1d8" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.7", - "regex-syntax 0.8.4", + "regex-automata 0.4.8", + "regex-syntax 0.8.5", ] [[package]] @@ -9020,13 +9259,13 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.7" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38caf58cc5ef2fed281f89292ef23f6365465ed9a41b7a7754eb4e26496c92df" +checksum = "368758f23274712b504848e9d5a6f010445cc8b87a7cdb4d7cbee666c1288da3" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.8.4", + "regex-syntax 0.8.5", ] [[package]] @@ -9043,9 +9282,9 @@ checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" [[package]] name = "regex-syntax" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" +checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" [[package]] name = "regress" @@ -9105,9 +9344,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.7" +version = "0.12.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8f4955649ef5c38cc7f9e8aa41761d48fb9677197daea9984dc54f56aad5e63" +checksum = "f713147fbe92361e52392c73b8c9e48c04c6625bce969ef54dc901e58e042a7b" dependencies = [ "base64 0.22.1", "bytes", @@ -9129,8 +9368,8 @@ dependencies = [ "pin-project-lite", "quinn", "rustls 0.23.13", - "rustls-native-certs 0.7.3", - "rustls-pemfile 2.1.3", + "rustls-native-certs 0.8.0", + "rustls-pemfile 2.2.0", "rustls-pki-types", "serde", "serde_json", @@ -9372,7 +9611,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.77", + "syn 2.0.79", "unicode-ident", ] @@ -9384,7 +9623,7 @@ checksum = "b3a8fb4672e840a587a66fc577a5491375df51ddb88f2a2c2a792598c326fe14" dependencies = [ "quote", "rand", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -9407,7 +9646,7 @@ dependencies = [ "proc-macro2", "quote", "rust-embed-utils", - "syn 2.0.77", + "syn 2.0.79", "walkdir", ] @@ -9617,7 +9856,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" dependencies = [ "openssl-probe", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "rustls-pki-types", "schannel", "security-framework", @@ -9630,7 +9869,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fcaf18a4f2be7326cd874a5fa579fae794320a0f388d365dca7e480e55f83f8a" dependencies = [ "openssl-probe", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "rustls-pki-types", "schannel", "security-framework", @@ -9647,19 +9886,18 @@ dependencies = [ [[package]] name = "rustls-pemfile" -version = "2.1.3" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "196fe16b00e106300d3e45ecfcb764fa292a535d7326a29a5875c579c7417425" +checksum = "dce314e5fee3f39953d46bb63bb8a46d40c2f8fb7cc5a3b6cab2bde9721d6e50" dependencies = [ - "base64 0.22.1", "rustls-pki-types", ] [[package]] name = "rustls-pki-types" -version = "1.8.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc0a2ce646f8655401bb81e7927b812614bd5d91dbc968696be50603510fcaf0" +checksum = "0e696e35370c65c9c541198af4543ccd580cf17fc25d8e05c5a242b202488c55" [[package]] name = "rustls-webpki" @@ -10134,7 +10372,7 @@ dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10192,7 +10430,7 @@ dependencies = [ "serde", "servers", "session", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "table", "tokio", @@ -10235,7 +10473,7 @@ dependencies = [ "heck 0.4.1", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10313,7 +10551,7 @@ checksum = "243902eda00fad750862fc144cea25caca5e20d615af0a81bee94ca738f1df1f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10324,7 +10562,7 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10333,7 +10571,7 @@ version = "1.0.128" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6ff5456707a1de34e7e37f2a6fd3d3f808c318259cbd01ab6377795054b483d8" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", "itoa", "memchr", "ryu", @@ -10358,14 +10596,14 @@ checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] name = "serde_spanned" -version = "0.6.7" +version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb5b1b31579f3811bf615c144393417496f152e12ac8b7663bf664f4a815306d" +checksum = "87607cb1398ed59d48732e575a4c28a7a8ebf2454b964fe3f224f2afc07909e1" dependencies = [ "serde", ] @@ -10379,7 +10617,7 @@ dependencies = [ "proc-macro2", "quote", "serde", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10396,15 +10634,15 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.9.0" +version = "3.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69cecfa94848272156ea67b2b1a53f20fc7bc638c4a46d2f8abde08f05f4b857" +checksum = "9720086b3357bcb44fce40117d769a4d068c70ecfa190850a980a71755f66fcc" dependencies = [ "base64 0.22.1", "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.5.0", + "indexmap 2.6.0", "serde", "serde_derive", "serde_json", @@ -10414,14 +10652,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.9.0" +version = "3.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8fee4991ef4f274617a51ad4af30519438dacb2f56ac773b08a1922ff743350" +checksum = "5f1abbfe725f27678f4663bcacb75a83e829fd464c25d78dd038a3a29e307cec" dependencies = [ "darling 0.20.10", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10430,7 +10668,7 @@ version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", "itoa", "ryu", "serde", @@ -10516,14 +10754,14 @@ dependencies = [ "reqwest", "rust-embed", "rustls 0.23.13", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "rustls-pki-types", "schemars", "script", "serde", "serde_json", "session", - "snafu 0.8.4", + "snafu 0.8.5", "snap", "sql", "strum 0.25.0", @@ -10560,7 +10798,7 @@ dependencies = [ "common-time", "derive_builder 0.12.0", "meter-core", - "snafu 0.8.4", + "snafu 0.8.5", "sql", ] @@ -10766,11 +11004,11 @@ dependencies = [ [[package]] name = "snafu" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b835cb902660db3415a672d862905e791e54d306c6e8189168c7f3d9ae1c79d" +checksum = "223891c85e2a29c3fe8fb900c1fae5e69c2e42415e3177752e8718475efa5019" dependencies = [ - "snafu-derive 0.8.4", + "snafu-derive 0.8.5", ] [[package]] @@ -10787,14 +11025,14 @@ dependencies = [ [[package]] name = "snafu-derive" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d1e02fca405f6280643174a50c942219f0bbf4dbf7d480f1dd864d6f211ae5" +checksum = "03c3c6b7927ffe7ecaa769ee0e3994da3b8cafc8f444578982c83ecb161af917" dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10894,7 +11132,7 @@ dependencies = [ "lazy_static", "regex", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "sqlparser 0.45.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09b11c0c88934690530807185d3e7)", "sqlparser_derive 0.1.1", "table", @@ -10933,18 +11171,21 @@ name = "sqlness-runner" version = "0.9.3" dependencies = [ "async-trait", - "clap 4.5.18", + "clap 4.5.19", "client", "common-error", "common-query", "common-recordbatch", "common-time", + "datatypes", + "mysql", "serde", "serde_json", "sqlness", "tempfile", "tinytemplate", "tokio", + "tokio-postgres", ] [[package]] @@ -10988,7 +11229,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -10998,7 +11239,7 @@ source = "git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09 dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -11167,7 +11408,7 @@ dependencies = [ "futures", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "strum 0.25.0", "tokio", ] @@ -11288,7 +11529,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -11301,7 +11542,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -11330,7 +11571,7 @@ dependencies = [ "datatypes", "promql", "prost 0.12.6", - "snafu 0.8.4", + "snafu 0.8.5", "substrait 0.37.3", "tokio", ] @@ -11354,7 +11595,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "syn 2.0.77", + "syn 2.0.79", "typify", "walkdir", ] @@ -11375,7 +11616,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "syn 2.0.77", + "syn 2.0.79", "typify", "walkdir", ] @@ -11388,9 +11629,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbolic-common" -version = "12.11.1" +version = "12.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fdf97c441f18a4f92425b896a4ec7a27e03631a0b1047ec4e34e9916a9a167e" +checksum = "366f1b4c6baf6cfefc234bbd4899535fca0b06c74443039a73f6dfb2fad88d77" dependencies = [ "debugid", "memmap2 0.9.5", @@ -11400,9 +11641,9 @@ dependencies = [ [[package]] name = "symbolic-demangle" -version = "12.11.1" +version = "12.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc8ece6b129e97e53d1fbb3f61d33a6a9e5369b11d01228c068094d6d134eaea" +checksum = "aba05ba5b9962ea5617baf556293720a8b2d0a282aa14ee4bf10e22efc7da8c8" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -11422,9 +11663,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.77" +version = "2.0.79" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f35bcdf61fd8e7be6caf75f429fdca8beb3ed76584befb503b1569faee373ed" +checksum = "89132cd0bf050864e1d38dc3bbc07a0eb8e7530af26344d3d2bbbef83499f590" dependencies = [ "proc-macro2", "quote", @@ -11459,7 +11700,7 @@ dependencies = [ "proc-macro-error", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -11545,7 +11786,7 @@ dependencies = [ "paste", "serde", "serde_json", - "snafu 0.8.4", + "snafu 0.8.5", "store-api", "tokio", "tokio-util", @@ -11579,7 +11820,7 @@ dependencies = [ "itertools 0.12.1", "levenshtein_automata", "log", - "lru", + "lru 0.12.4", "lz4_flex 0.11.3", "measure_time", "memmap2 0.9.5", @@ -11654,7 +11895,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d60769b80ad7953d8a7b2c70cdfe722bbcdcac6bccc8ac934c40c034d866fc18" dependencies = [ "byteorder", - "regex-syntax 0.8.4", + "regex-syntax 0.8.5", "utf8-ranges", ] @@ -11733,9 +11974,9 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.12.0" +version = "3.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04cbcdd0c794ebb0d4cf35e88edd2f7d2c4c3e9a5a6dab322839b321c6a87a64" +checksum = "f0f2c9fc62d0beef6951ccffd757e241266a2c833136efbe35af6cd2567dca5b" dependencies = [ "cfg-if", "fastrand", @@ -11811,7 +12052,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "sqlparser 0.45.0 (git+https://github.com/GreptimeTeam/sqlparser-rs.git?rev=54a267ac89c09b11c0c88934690530807185d3e7)", "sqlx", @@ -11834,7 +12075,7 @@ dependencies = [ "cache", "catalog", "chrono", - "clap 4.5.18", + "clap 4.5.19", "client", "cmd", "common-base", @@ -11881,7 +12122,7 @@ dependencies = [ "servers", "session", "similar-asserts", - "snafu 0.8.4", + "snafu 0.8.5", "sql", "sqlx", "store-api", @@ -11937,7 +12178,7 @@ checksum = "08904e7672f5eb876eaaf87e0ce17857500934f4981c4a0ab2b4aa98baac7fc3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -12124,7 +12365,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -12291,7 +12532,7 @@ dependencies = [ "serde", "serde_spanned", "toml_datetime", - "toml_edit 0.22.21", + "toml_edit 0.22.22", ] [[package]] @@ -12309,22 +12550,22 @@ version = "0.19.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", "toml_datetime", "winnow 0.5.40", ] [[package]] name = "toml_edit" -version = "0.22.21" +version = "0.22.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b072cee73c449a636ffd6f32bd8de3a9f7119139aff882f44943ce2986dc5cf" +checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" dependencies = [ - "indexmap 2.5.0", + "indexmap 2.6.0", "serde", "serde_spanned", "toml_datetime", - "winnow 0.6.18", + "winnow 0.6.20", ] [[package]] @@ -12375,7 +12616,7 @@ dependencies = [ "percent-encoding", "pin-project", "prost 0.12.6", - "rustls-pemfile 2.1.3", + "rustls-pemfile 2.2.0", "rustls-pki-types", "tokio", "tokio-rustls 0.25.0", @@ -12397,7 +12638,7 @@ dependencies = [ "proc-macro2", "prost-build 0.12.6", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -12440,7 +12681,7 @@ version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "61c5bb1d698276a2443e5ecfabc1008bf15a36c12e6a7176e7bf089ea9131140" dependencies = [ - "async-compression 0.4.12", + "async-compression 0.4.13", "base64 0.21.7", "bitflags 2.6.0", "bytes", @@ -12527,7 +12768,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -12784,7 +13025,7 @@ checksum = "70b20a22c42c8f1cd23ce5e34f165d4d37038f5b663ad20fb6adbdf029172483" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -12812,7 +13053,7 @@ dependencies = [ "semver", "serde", "serde_json", - "syn 2.0.77", + "syn 2.0.79", "thiserror", "unicode-ident", ] @@ -12830,7 +13071,7 @@ dependencies = [ "serde", "serde_json", "serde_tokenstream", - "syn 2.0.77", + "syn 2.0.79", "typify-impl", ] @@ -12871,9 +13112,9 @@ checksum = "fe4fa6e588762366f1eb4991ce59ad1b93651d0b769dfb4e4d1c5c4b943d1159" [[package]] name = "ucd-trie" -version = "0.1.6" +version = "0.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed646292ffc8188ef8ea4d1e0e0150fb15a5c2e12ad9b8fc191ae7a8a7f3c4b9" +checksum = "2896d95c02a80c6d6a5d6e953d479f5ddf2dfdb6a244441010e373ac0fb88971" [[package]] name = "uname" @@ -13011,9 +13252,9 @@ dependencies = [ [[package]] name = "unicode-bidi" -version = "0.3.15" +version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" +checksum = "5ab17db44d7388991a428b2ee655ce0c212e862eff1768a455c58f9aad6e7893" [[package]] name = "unicode-casing" @@ -13038,9 +13279,9 @@ dependencies = [ [[package]] name = "unicode-properties" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52ea75f83c0137a9b98608359a5f1af8144876eb67bcb1ce837368e906a9f524" +checksum = "e70f2a8b45122e719eb623c01822704c4e0907e7e426a05927e1a1cfff5b75d0" [[package]] name = "unicode-segmentation" @@ -13154,7 +13395,7 @@ checksum = "ee1cd046f83ea2c4e920d6ee9f7c3537ef928d75dce5d84a87c2c5d6b3999a3a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -13270,7 +13511,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", "wasm-bindgen-shared", ] @@ -13304,7 +13545,7 @@ checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -13317,9 +13558,9 @@ checksum = "c62a0a307cb4a311d3a07867860911ca130c3494e8c2719593806c08bc5d0484" [[package]] name = "wasm-streams" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b65dc4c90b63b118468cf747d8bf3566c1913ef60be765b5730ead9e0a3ba129" +checksum = "4e072d4e72f700fb3443d8fe94a39315df013eef1104903cdb0a2abd322bbecd" dependencies = [ "futures-util", "js-sys", @@ -13427,7 +13668,7 @@ version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "372d5b87f58ec45c384ba03563b03544dc5fadc3983e434b286913f5b4a9bb6d" dependencies = [ - "redox_syscall 0.5.4", + "redox_syscall 0.5.7", "wasite", "web-sys", ] @@ -13796,9 +14037,9 @@ dependencies = [ [[package]] name = "winnow" -version = "0.6.18" +version = "0.6.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68a9bda4691f099d435ad181000724da8e5899daa10713c2d432552b9ccd3a6f" +checksum = "36c1fec1a2bb5866f07c25f68c26e565c4c200aebb96d7e55710c19d3e8ac49b" dependencies = [ "memchr", ] @@ -13832,7 +14073,7 @@ dependencies = [ "chrono", "der 0.7.9", "hex", - "pem", + "pem 3.0.4", "ring 0.17.8", "signature", "spki 0.7.3", @@ -13888,7 +14129,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] @@ -13908,7 +14149,7 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn 2.0.79", ] [[package]] diff --git a/src/log-store/src/kafka/client_manager.rs b/src/log-store/src/kafka/client_manager.rs index 6337683c93..589d5a900e 100644 --- a/src/log-store/src/kafka/client_manager.rs +++ b/src/log-store/src/kafka/client_manager.rs @@ -132,7 +132,7 @@ impl ClientManager { } async fn try_create_client(&self, provider: &Arc) -> Result { - // Sets to Retry to retry connecting if the kafka cluter replies with an UnknownTopic error. + // Sets to Retry to retry connecting if the kafka cluster replies with an UnknownTopic error. // That's because the topic is believed to exist as the metasrv is expected to create required topics upon start. // The reconnecting won't stop until succeed or a different error returns. let client = self diff --git a/src/script/src/python/rspython/builtins/test.rs b/src/script/src/python/rspython/builtins/test.rs index 3908ab9eef..541f28c66a 100644 --- a/src/script/src/python/rspython/builtins/test.rs +++ b/src/script/src/python/rspython/builtins/test.rs @@ -145,7 +145,7 @@ enum PyValue { } impl PyValue { - /// compare if results is just as expect, not using PartialEq because it is not transtive .e.g. [1,2,3] == len(3) == [4,5,6] + /// compare if results is just as expect, not using PartialEq because it is not transitive .e.g. [1,2,3] == len(3) == [4,5,6] fn just_as_expect(&self, other: &Self) -> bool { match (self, other) { (PyValue::FloatVec(a), PyValue::FloatVec(b)) => a diff --git a/tests/cases/standalone/common/mysql.result b/tests/cases/standalone/common/mysql.result new file mode 100644 index 0000000000..980990fa7a --- /dev/null +++ b/tests/cases/standalone/common/mysql.result @@ -0,0 +1,29 @@ +-- SQLNESS PROTOCOL MYSQL +SELECt @@tx_isolation; + ++-----------------+ +| @@tx_isolation; | ++-----------------+ +| 0 | ++-----------------+ + +-- SQLNESS PROTOCOL MYSQL +SELECT @@version_comment; + ++--------------------+ +| @@version_comment; | ++--------------------+ +| 0 | ++--------------------+ + +-- SQLNESS PROTOCOL MYSQL +SHOW DATABASES; + ++--------------------+ +| Database | ++--------------------+ +| greptime_private | +| information_schema | +| public | ++--------------------+ + diff --git a/tests/cases/standalone/common/mysql.sql b/tests/cases/standalone/common/mysql.sql new file mode 100644 index 0000000000..31b2db23f9 --- /dev/null +++ b/tests/cases/standalone/common/mysql.sql @@ -0,0 +1,8 @@ +-- SQLNESS PROTOCOL MYSQL +SELECt @@tx_isolation; + +-- SQLNESS PROTOCOL MYSQL +SELECT @@version_comment; + +-- SQLNESS PROTOCOL MYSQL +SHOW DATABASES; diff --git a/tests/cases/standalone/common/system/pg_catalog.result b/tests/cases/standalone/common/system/pg_catalog.result index d30355352f..4e3e217b2e 100644 --- a/tests/cases/standalone/common/system/pg_catalog.result +++ b/tests/cases/standalone/common/system/pg_catalog.result @@ -3,11 +3,49 @@ create database pg_catalog; Error: 1004(InvalidArguments), Schema pg_catalog already exists -select * from pg_catalog.pg_type order by oid; +-- make sure all the pg_catalog tables are only visible to postgres +select * from pg_catalog.pg_class; + +Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class + +select * from pg_catalog.pg_namespace; + +Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_namespace + +select * from pg_catalog.pg_type; Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_type +-- SQLNESS PROTOCOL POSTGRES +select * from pg_catalog.pg_type order by oid; + ++-----+-----------+--------+ +| oid | typname | typlen | ++-----+-----------+--------+ +| 1 | String | -1 | +| 2 | Binary | -1 | +| 3 | Int8 | 1 | +| 4 | Int16 | 2 | +| 5 | Int32 | 4 | +| 6 | Int64 | 8 | +| 7 | UInt8 | 1 | +| 8 | UInt16 | 2 | +| 9 | UInt32 | 4 | +| 10 | UInt64 | 8 | +| 11 | Float32 | 4 | +| 12 | Float64 | 8 | +| 13 | Decimal | 16 | +| 14 | Date | 4 | +| 15 | DateTime | 8 | +| 16 | Timestamp | 8 | +| 17 | Time | 8 | +| 18 | Duration | 8 | +| 19 | Interval | 16 | +| 20 | List | -1 | ++-----+-----------+--------+ + -- \d +-- SQLNESS PROTOCOL POSTGRES SELECT n.nspname as "Schema", c.relname as "Name", CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type", @@ -21,9 +59,14 @@ WHERE c.relkind IN ('r','p','v','m','S','f','') AND pg_catalog.pg_table_is_visible(c.oid) ORDER BY 1,2; -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class ++--------+---------+-------+-------+ +| Schema | Name | Type | Owner | ++--------+---------+-------+-------+ +| public | numbers | table | | ++--------+---------+-------+-------+ -- \dt +-- SQLNESS PROTOCOL POSTGRES SELECT n.nspname as "Schema", c.relname as "Name", CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type", @@ -37,22 +80,36 @@ WHERE c.relkind IN ('r','p','') AND pg_catalog.pg_table_is_visible(c.oid) ORDER BY 1,2; -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class ++--------+---------+-------+-------+ +| Schema | Name | Type | Owner | ++--------+---------+-------+-------+ +| public | numbers | table | | ++--------+---------+-------+-------+ -- make sure oid of namespace keep stable +-- SQLNESS PROTOCOL POSTGRES SELECT * FROM pg_namespace ORDER BY oid; -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.public.pg_namespace ++------------+--------------------+ +| oid | nspname | ++------------+--------------------+ +| 667359454 | pg_catalog | +| 3174397350 | information_schema | +| 3338153620 | greptime_private | +| 3927743705 | public | ++------------+--------------------+ -create -database my_db; +-- SQLNESS PROTOCOL POSTGRES +create database my_db; Affected Rows: 1 +-- SQLNESS PROTOCOL POSTGRES use my_db; -Affected Rows: 0 + +-- SQLNESS PROTOCOL POSTGRES create table foo ( ts TIMESTAMP TIME INDEX @@ -61,6 +118,7 @@ create table foo Affected Rows: 0 -- show tables in `my_db` +-- SQLNESS PROTOCOL POSTGRES select relname from pg_catalog.pg_class where relnamespace = ( @@ -69,9 +127,14 @@ where relnamespace = ( where nspname = 'my_db' ); -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class ++---------+ +| relname | ++---------+ +| foo | ++---------+ -- \dt +-- SQLNESS PROTOCOL POSTGRES SELECT n.nspname as "Schema", c.relname as "Name", CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type", @@ -85,9 +148,15 @@ WHERE c.relkind IN ('r','p','') AND pg_catalog.pg_table_is_visible(c.oid) ORDER BY 1,2; -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class ++--------+---------+-------+-------+ +| Schema | Name | Type | Owner | ++--------+---------+-------+-------+ +| my_db | foo | table | | +| public | numbers | table | | ++--------+---------+-------+-------+ -- show tables in `my_db`, `public` +-- SQLNESS PROTOCOL POSTGRES select relname from pg_catalog.pg_class where relnamespace in ( @@ -97,8 +166,14 @@ where relnamespace in ( ) order by relname; -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class ++---------+ +| relname | ++---------+ +| foo | +| numbers | ++---------+ +-- SQLNESS PROTOCOL POSTGRES select relname from pg_catalog.pg_class where relnamespace in ( @@ -107,8 +182,13 @@ where relnamespace in ( where nspname like 'my%' ); -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class ++---------+ +| relname | ++---------+ +| foo | ++---------+ +-- SQLNESS PROTOCOL POSTGRES select relnamespace, relname, relkind from pg_catalog.pg_class where relnamespace in ( @@ -120,34 +200,58 @@ where relnamespace in ( ) order by relnamespace, relname; -Error: 4001(TableNotFound), Failed to plan SQL: Table not found: greptime.pg_catalog.pg_class ++--------------+---------+---------+ +| relnamespace | relname | relkind | ++--------------+---------+---------+ +| 434869349 | foo | r | ++--------------+---------+---------+ +-- SQLNESS PROTOCOL POSTGRES use public; -Affected Rows: 0 + +-- SQLNESS PROTOCOL POSTGRES drop schema my_db; Affected Rows: 0 +-- SQLNESS PROTOCOL POSTGRES use pg_catalog; -Affected Rows: 0 + -- pg_class +-- SQLNESS PROTOCOL POSTGRES desc table pg_class; -Error: 4001(TableNotFound), Table not found: pg_class ++--------------+--------+-----+------+---------+---------------+ +| Column | Type | Key | Null | Default | Semantic Type | ++--------------+--------+-----+------+---------+---------------+ +| oid | UInt32 | | NO | | FIELD | +| relname | String | | NO | | FIELD | +| relnamespace | UInt32 | | NO | | FIELD | +| relkind | String | | NO | | FIELD | +| relowner | UInt32 | | NO | | FIELD | ++--------------+--------+-----+------+---------+---------------+ +-- SQLNESS PROTOCOL POSTGRES desc table pg_namespace; -Error: 4001(TableNotFound), Table not found: pg_namespace ++---------+--------+-----+------+---------+---------------+ +| Column | Type | Key | Null | Default | Semantic Type | ++---------+--------+-----+------+---------+---------------+ +| oid | UInt32 | | NO | | FIELD | +| nspname | String | | NO | | FIELD | ++---------+--------+-----+------+---------+---------------+ +-- SQLNESS PROTOCOL POSTGRES drop table my_db.foo; -Error: 4001(TableNotFound), Table not found: greptime.my_db.foo +Failed to execute query, encountered: Error { kind: Db, cause: Some(DbError { severity: "ERROR", parsed_severity: None, code: SqlState(E42P01), message: "Table not found: greptime.my_db.foo", detail: None, hint: None, position: None, where_: None, schema: None, table: None, column: None, datatype: None, constraint: None, file: None, line: None, routine: None }) } +-- SQLNESS PROTOCOL POSTGRES use public; -Affected Rows: 0 + diff --git a/tests/cases/standalone/common/system/pg_catalog.sql b/tests/cases/standalone/common/system/pg_catalog.sql index 50fb7b9745..712302254d 100644 --- a/tests/cases/standalone/common/system/pg_catalog.sql +++ b/tests/cases/standalone/common/system/pg_catalog.sql @@ -1,9 +1,16 @@ -- should not able to create pg_catalog create database pg_catalog; +-- make sure all the pg_catalog tables are only visible to postgres +select * from pg_catalog.pg_class; +select * from pg_catalog.pg_namespace; +select * from pg_catalog.pg_type; + +-- SQLNESS PROTOCOL POSTGRES select * from pg_catalog.pg_type order by oid; -- \d +-- SQLNESS PROTOCOL POSTGRES SELECT n.nspname as "Schema", c.relname as "Name", CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type", @@ -18,7 +25,7 @@ WHERE c.relkind IN ('r','p','v','m','S','f','') ORDER BY 1,2; -- \dt - +-- SQLNESS PROTOCOL POSTGRES SELECT n.nspname as "Schema", c.relname as "Name", CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type", @@ -33,19 +40,23 @@ WHERE c.relkind IN ('r','p','') ORDER BY 1,2; -- make sure oid of namespace keep stable +-- SQLNESS PROTOCOL POSTGRES SELECT * FROM pg_namespace ORDER BY oid; -create -database my_db; +-- SQLNESS PROTOCOL POSTGRES +create database my_db; +-- SQLNESS PROTOCOL POSTGRES use my_db; +-- SQLNESS PROTOCOL POSTGRES create table foo ( ts TIMESTAMP TIME INDEX ); -- show tables in `my_db` +-- SQLNESS PROTOCOL POSTGRES select relname from pg_catalog.pg_class where relnamespace = ( @@ -55,6 +66,7 @@ where relnamespace = ( ); -- \dt +-- SQLNESS PROTOCOL POSTGRES SELECT n.nspname as "Schema", c.relname as "Name", CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type", @@ -69,6 +81,7 @@ WHERE c.relkind IN ('r','p','') ORDER BY 1,2; -- show tables in `my_db`, `public` +-- SQLNESS PROTOCOL POSTGRES select relname from pg_catalog.pg_class where relnamespace in ( @@ -78,6 +91,7 @@ where relnamespace in ( ) order by relname; +-- SQLNESS PROTOCOL POSTGRES select relname from pg_catalog.pg_class where relnamespace in ( @@ -86,6 +100,7 @@ where relnamespace in ( where nspname like 'my%' ); +-- SQLNESS PROTOCOL POSTGRES select relnamespace, relname, relkind from pg_catalog.pg_class where relnamespace in ( @@ -97,17 +112,24 @@ where relnamespace in ( ) order by relnamespace, relname; +-- SQLNESS PROTOCOL POSTGRES use public; +-- SQLNESS PROTOCOL POSTGRES drop schema my_db; +-- SQLNESS PROTOCOL POSTGRES use pg_catalog; -- pg_class +-- SQLNESS PROTOCOL POSTGRES desc table pg_class; +-- SQLNESS PROTOCOL POSTGRES desc table pg_namespace; +-- SQLNESS PROTOCOL POSTGRES drop table my_db.foo; +-- SQLNESS PROTOCOL POSTGRES use public; diff --git a/tests/conf/standalone-test.toml.template b/tests/conf/standalone-test.toml.template index 2e30ac35c2..f96a815762 100644 --- a/tests/conf/standalone-test.toml.template +++ b/tests/conf/standalone-test.toml.template @@ -24,6 +24,19 @@ data_home = '{data_home}' addr = '127.0.0.1:4001' runtime_size = 8 +[mysql] +enable = true +addr = "127.0.0.1:4002" +runtime_size = 2 + +[mysql.tls] +mode = "disable" + +[postgres] +enable = true +addr = "127.0.0.1:4003" +runtime_size = 2 + [procedure] max_retry_times = 3 retry_delay = "500ms" diff --git a/tests/runner/Cargo.toml b/tests/runner/Cargo.toml index 1bac0c5933..641f07c2c6 100644 --- a/tests/runner/Cargo.toml +++ b/tests/runner/Cargo.toml @@ -15,8 +15,11 @@ common-error.workspace = true common-query.workspace = true common-recordbatch.workspace = true common-time.workspace = true +datatypes = { workspace = true } +mysql = { version = "23.0.1" } serde.workspace = true serde_json.workspace = true +tokio-postgres = { workspace = true } # sqlness 0.6.0 have a bug causing `cargo sqlness` to fail(see https://github.com/CeresDB/sqlness/issues/68) which is fixed in 0.6.1 sqlness = "0.6.1" tempfile.workspace = true diff --git a/tests/runner/src/env.rs b/tests/runner/src/env.rs index 9d71489f8c..0dc0e3a9e8 100644 --- a/tests/runner/src/env.rs +++ b/tests/runner/src/env.rs @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::Cow; use std::fmt::Display; use std::fs::OpenOptions; use std::io; use std::io::Write; +use std::net::SocketAddr; use std::path::{Path, PathBuf}; use std::process::{Child, Command}; use std::sync::atomic::{AtomicU32, Ordering}; @@ -30,15 +32,25 @@ use client::{ use common_error::ext::ErrorExt; use common_query::{Output, OutputData}; use common_recordbatch::RecordBatches; +use datatypes::data_type::ConcreteDataType; +use datatypes::scalars::ScalarVectorBuilder; +use datatypes::schema::{ColumnSchema, Schema}; +use datatypes::vectors::{StringVectorBuilder, VectorRef}; +use mysql::prelude::Queryable; +use mysql::{Conn as MySqlClient, Row as MySqlRow}; use serde::Serialize; use sqlness::{Database, EnvController, QueryContext}; use tinytemplate::TinyTemplate; use tokio::sync::Mutex as TokioMutex; +use tokio_postgres::{Client as PgClient, SimpleQueryMessage as PgRow}; -use crate::util; +use crate::protocol_interceptor::{MYSQL, PROTOCOL_KEY}; +use crate::{util, ServerAddr}; const METASRV_ADDR: &str = "127.0.0.1:3002"; -const SERVER_ADDR: &str = "127.0.0.1:4001"; +const GRPC_SERVER_ADDR: &str = "127.0.0.1:4001"; +const MYSQL_SERVER_ADDR: &str = "127.0.0.1:4002"; +const POSTGRES_SERVER_ADDR: &str = "127.0.0.1:4003"; const DEFAULT_LOG_LEVEL: &str = "--log-level=debug,hyper=warn,tower=warn,datafusion=warn,reqwest=warn,sqlparser=warn,h2=info,opendal=info"; #[derive(Clone)] @@ -55,7 +67,7 @@ pub enum WalConfig { #[derive(Clone)] pub struct Env { sqlness_home: PathBuf, - server_addr: Option, + server_addrs: ServerAddr, wal: WalConfig, /// The path to the directory that contains the pre-built GreptimeDB binary. @@ -86,21 +98,21 @@ impl EnvController for Env { impl Env { pub fn new( data_home: PathBuf, - server_addr: Option, + server_addrs: ServerAddr, wal: WalConfig, bins_dir: Option, ) -> Self { Self { sqlness_home: data_home, - server_addr, + server_addrs, wal, bins_dir: Arc::new(Mutex::new(bins_dir)), } } async fn start_standalone(&self) -> GreptimeDB { - if let Some(server_addr) = self.server_addr.clone() { - self.connect_db(&server_addr) + if self.server_addrs.server_addr.is_some() { + self.connect_db(&self.server_addrs).await } else { self.build_db(); self.setup_wal(); @@ -109,25 +121,18 @@ impl Env { let server_process = self.start_server("standalone", &db_ctx, true).await; - let client = Client::with_urls(vec![SERVER_ADDR]); - let db = DB::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, client); + let mut greptimedb = self.connect_db(&Default::default()).await; + greptimedb.server_processes = Some(Arc::new(Mutex::new(vec![server_process]))); + greptimedb.is_standalone = true; + greptimedb.ctx = db_ctx; - GreptimeDB { - server_processes: Some(Arc::new(Mutex::new(vec![server_process]))), - metasrv_process: None, - frontend_process: None, - flownode_process: None, - client: TokioMutex::new(db), - ctx: db_ctx, - is_standalone: true, - env: self.clone(), - } + greptimedb } } async fn start_distributed(&self) -> GreptimeDB { - if let Some(server_addr) = self.server_addr.clone() { - self.connect_db(&server_addr) + if self.server_addrs.server_addr.is_some() { + self.connect_db(&self.server_addrs).await } else { self.build_db(); self.setup_wal(); @@ -145,29 +150,92 @@ impl Env { let flownode = self.start_server("flownode", &db_ctx, true).await; - let client = Client::with_urls(vec![SERVER_ADDR]); - let db = DB::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, client); + let mut greptimedb = self.connect_db(&Default::default()).await; - GreptimeDB { - server_processes: Some(Arc::new(Mutex::new(vec![ - datanode_1, datanode_2, datanode_3, - ]))), - metasrv_process: Some(meta_server), - frontend_process: Some(frontend), - flownode_process: Some(flownode), - client: TokioMutex::new(db), - ctx: db_ctx, - is_standalone: false, - env: self.clone(), - } + greptimedb.metasrv_process = Some(meta_server); + greptimedb.server_processes = Some(Arc::new(Mutex::new(vec![ + datanode_1, datanode_2, datanode_3, + ]))); + greptimedb.frontend_process = Some(frontend); + greptimedb.flownode_process = Some(flownode); + greptimedb.is_standalone = false; + greptimedb.ctx = db_ctx; + + greptimedb } } - fn connect_db(&self, server_addr: &str) -> GreptimeDB { - let client = Client::with_urls(vec![server_addr.to_owned()]); - let db = DB::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, client); + async fn create_pg_client(&self, pg_server_addr: &str) -> PgClient { + let sockaddr: SocketAddr = pg_server_addr.parse().expect( + "Failed to parse the Postgres server address. Please check if the address is in the format of `ip:port`.", + ); + let mut config = tokio_postgres::config::Config::new(); + config.host(sockaddr.ip().to_string()); + config.port(sockaddr.port()); + config.dbname(DEFAULT_SCHEMA_NAME); + + // retry to connect to Postgres server until success + const MAX_RETRY: usize = 3; + let mut backoff = Duration::from_millis(500); + for _ in 0..MAX_RETRY { + if let Ok((pg_client, conn)) = config.connect(tokio_postgres::NoTls).await { + tokio::spawn(conn); + return pg_client; + } + tokio::time::sleep(backoff).await; + backoff *= 2; + } + panic!("Failed to connect to Postgres server. Please check if the server is running."); + } + + async fn create_mysql_client(&self, mysql_server_addr: &str) -> MySqlClient { + let sockaddr: SocketAddr = mysql_server_addr.parse().expect( + "Failed to parse the MySQL server address. Please check if the address is in the format of `ip:port`.", + ); + let ops = mysql::OptsBuilder::new() + .ip_or_hostname(Some(sockaddr.ip().to_string())) + .tcp_port(sockaddr.port()) + .db_name(Some(DEFAULT_SCHEMA_NAME)); + // retry to connect to MySQL server until success + const MAX_RETRY: usize = 3; + let mut backoff = Duration::from_millis(500); + + for _ in 0..MAX_RETRY { + // exponential backoff + if let Ok(client) = mysql::Conn::new(ops.clone()) { + return client; + } + tokio::time::sleep(backoff).await; + backoff *= 2; + } + + panic!("Failed to connect to MySQL server. Please check if the server is running.") + } + + async fn connect_db(&self, server_addr: &ServerAddr) -> GreptimeDB { + let grpc_server_addr = server_addr + .server_addr + .clone() + .unwrap_or(GRPC_SERVER_ADDR.to_owned()); + let pg_server_addr = server_addr + .pg_server_addr + .clone() + .unwrap_or(POSTGRES_SERVER_ADDR.to_owned()); + let mysql_server_addr = server_addr + .mysql_server_addr + .clone() + .unwrap_or(MYSQL_SERVER_ADDR.to_owned()); + + let grpc_client = Client::with_urls(vec![grpc_server_addr.clone()]); + let db = DB::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, grpc_client); + let pg_client = self.create_pg_client(&pg_server_addr).await; + + let mysql_client = self.create_mysql_client(&mysql_server_addr).await; + GreptimeDB { - client: TokioMutex::new(db), + grpc_client: TokioMutex::new(db), + pg_client: TokioMutex::new(pg_client), + mysql_client: TokioMutex::new(mysql_client), server_processes: None, metasrv_process: None, frontend_process: None, @@ -217,9 +285,15 @@ impl Env { .open(stdout_file_name) .unwrap(); - let (args, check_ip_addr) = match subcommand { - "datanode" => self.datanode_start_args(db_ctx), - "flownode" => self.flownode_start_args(db_ctx, &self.sqlness_home), + let (args, check_ip_addrs) = match subcommand { + "datanode" => { + let (args, addr) = self.datanode_start_args(db_ctx); + (args, vec![addr]) + } + "flownode" => { + let (args, addr) = self.flownode_start_args(db_ctx, &self.sqlness_home); + (args, vec![addr]) + } "standalone" => { let args = vec![ DEFAULT_LOG_LEVEL.to_string(), @@ -233,7 +307,7 @@ impl Env { self.generate_config_file(subcommand, db_ctx), "--http-addr=127.0.0.1:5002".to_string(), ]; - (args, SERVER_ADDR.to_string()) + (args, vec![GRPC_SERVER_ADDR.to_string()]) } "frontend" => { let args = vec![ @@ -247,7 +321,7 @@ impl Env { self.sqlness_home.display() ), ]; - (args, SERVER_ADDR.to_string()) + (args, vec![GRPC_SERVER_ADDR.to_string()]) } "metasrv" => { let args = vec![ @@ -266,16 +340,18 @@ impl Env { "-c".to_string(), self.generate_config_file(subcommand, db_ctx), ]; - (args, METASRV_ADDR.to_string()) + (args, vec![METASRV_ADDR.to_string()]) } _ => panic!("Unexpected subcommand: {subcommand}"), }; - if util::check_port(check_ip_addr.parse().unwrap(), Duration::from_secs(1)).await { - panic!( - "Port {check_ip_addr} is already in use, please check and retry.", - check_ip_addr = check_ip_addr - ); + for check_ip_addr in &check_ip_addrs { + if util::check_port(check_ip_addr.parse().unwrap(), Duration::from_secs(1)).await { + panic!( + "Port {check_ip_addr} is already in use, please check and retry.", + check_ip_addr = check_ip_addr + ); + } } #[cfg(not(windows))] @@ -297,9 +373,11 @@ impl Env { panic!("Failed to start the DB with subcommand {subcommand},Error: {error}") }); - if !util::check_port(check_ip_addr.parse().unwrap(), Duration::from_secs(10)).await { - Env::stop_server(&mut process); - panic!("{subcommand} doesn't up in 10 seconds, quit.") + for check_ip_addr in &check_ip_addrs { + if !util::check_port(check_ip_addr.parse().unwrap(), Duration::from_secs(10)).await { + Env::stop_server(&mut process); + panic!("{subcommand} doesn't up in 10 seconds, quit.") + } } process @@ -379,6 +457,9 @@ impl Env { }; if let Some(server_process) = db.server_processes.clone() { + *db.pg_client.lock().await = self.create_pg_client(&self.pg_server_addr()).await; + *db.mysql_client.lock().await = + self.create_mysql_client(&self.mysql_server_addr()).await; let mut server_processes = server_process.lock().unwrap(); *server_processes = new_server_processes; } @@ -433,6 +514,20 @@ impl Env { conf_file } + fn pg_server_addr(&self) -> String { + self.server_addrs + .pg_server_addr + .clone() + .unwrap_or(POSTGRES_SERVER_ADDR.to_owned()) + } + + fn mysql_server_addr(&self) -> String { + self.server_addrs + .mysql_server_addr + .clone() + .unwrap_or(MYSQL_SERVER_ADDR.to_owned()) + } + /// Build the DB with `cargo build --bin greptime` fn build_db(&self) { if self.bins_dir.lock().unwrap().is_some() { @@ -467,20 +562,51 @@ pub struct GreptimeDB { metasrv_process: Option, frontend_process: Option, flownode_process: Option, - client: TokioMutex, + grpc_client: TokioMutex, + pg_client: TokioMutex, + mysql_client: TokioMutex, ctx: GreptimeDBContext, is_standalone: bool, env: Env, } -#[async_trait] -impl Database for GreptimeDB { - async fn query(&self, ctx: QueryContext, query: String) -> Box { - if ctx.context.contains_key("restart") && self.env.server_addr.is_none() { - self.env.restart_server(self).await; +impl GreptimeDB { + async fn postgres_query(&self, _ctx: QueryContext, query: String) -> Box { + let client = self.pg_client.lock().await; + match client.simple_query(&query).await { + Ok(rows) => Box::new(PostgresqlFormatter { rows }), + Err(e) => Box::new(format!("Failed to execute query, encountered: {:?}", e)), } + } - let mut client = self.client.lock().await; + async fn mysql_query(&self, _ctx: QueryContext, query: String) -> Box { + let mut conn = self.mysql_client.lock().await; + let result = conn.query_iter(query); + Box::new(match result { + Ok(result) => { + let mut rows = vec![]; + let affected_rows = result.affected_rows(); + for row in result { + match row { + Ok(r) => rows.push(r), + Err(e) => { + return Box::new(format!("Failed to parse query result, err: {:?}", e)) + } + } + } + + if rows.is_empty() { + format!("affected_rows: {}", affected_rows) + } else { + format!("{}", MysqlFormatter { rows }) + } + } + Err(e) => format!("Failed to execute query, err: {:?}", e), + }) + } + + async fn grpc_query(&self, _ctx: QueryContext, query: String) -> Box { + let mut client = self.grpc_client.lock().await; let query_str = query.trim().to_lowercase(); @@ -542,6 +668,25 @@ impl Database for GreptimeDB { } } +#[async_trait] +impl Database for GreptimeDB { + async fn query(&self, ctx: QueryContext, query: String) -> Box { + if ctx.context.contains_key("restart") && self.env.server_addrs.server_addr.is_none() { + self.env.restart_server(self).await; + } + if let Some(protocol) = ctx.context.get(PROTOCOL_KEY) { + // protocol is bound to be either "mysql" or "postgres" + if protocol == MYSQL { + self.mysql_query(ctx, query).await + } else { + self.postgres_query(ctx, query).await + } + } else { + self.grpc_query(ctx, query).await + } + } +} + impl GreptimeDB { fn stop(&mut self) { if let Some(server_processes) = self.server_processes.clone() { @@ -575,7 +720,7 @@ impl GreptimeDB { impl Drop for GreptimeDB { fn drop(&mut self) { - if self.env.server_addr.is_none() { + if self.env.server_addrs.server_addr.is_none() { self.stop(); } } @@ -657,3 +802,117 @@ impl Display for ResultDisplayer { } } } + +struct PostgresqlFormatter { + pub rows: Vec, +} + +impl Display for PostgresqlFormatter { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + if self.rows.is_empty() { + return f.write_fmt(format_args!("(Empty response)")); + } + + // create schema + let schema = match &self.rows[0] { + PgRow::CommandComplete(affected_rows) => { + write!( + f, + "{}", + ResultDisplayer { + result: Ok(Output::new_with_affected_rows(*affected_rows as usize)), + } + )?; + return Ok(()); + } + PgRow::RowDescription(desc) => Arc::new(Schema::new( + desc.iter() + .map(|column| { + ColumnSchema::new(column.name(), ConcreteDataType::string_datatype(), false) + }) + .collect(), + )), + _ => unreachable!(), + }; + if schema.num_columns() == 0 { + return Ok(()); + } + + // convert to string vectors + let mut columns: Vec = (0..schema.num_columns()) + .map(|_| StringVectorBuilder::with_capacity(schema.num_columns())) + .collect(); + for row in self.rows.iter().skip(1) { + if let PgRow::Row(row) = row { + for (i, column) in columns.iter_mut().enumerate().take(schema.num_columns()) { + column.push(row.get(i)); + } + } + } + let columns: Vec = columns + .into_iter() + .map(|mut col| Arc::new(col.finish()) as VectorRef) + .collect(); + + // construct recordbatch + let recordbatches = RecordBatches::try_from_columns(schema, columns) + .expect("Failed to construct recordbatches from columns. Please check the schema."); + let result_displayer = ResultDisplayer { + result: Ok(Output::new_with_record_batches(recordbatches)), + }; + write!(f, "{}", result_displayer)?; + + Ok(()) + } +} + +struct MysqlFormatter { + pub rows: Vec, +} + +impl Display for MysqlFormatter { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + if self.rows.is_empty() { + return f.write_fmt(format_args!("(Empty response)")); + } + // create schema + let head_column = &self.rows[0]; + let head_binding = head_column.columns(); + let names = head_binding + .iter() + .map(|column| column.name_str()) + .collect::>>(); + let schema = Arc::new(Schema::new( + names + .iter() + .map(|name| { + ColumnSchema::new(name.to_string(), ConcreteDataType::string_datatype(), false) + }) + .collect(), + )); + + // convert to string vectors + let mut columns: Vec = (0..schema.num_columns()) + .map(|_| StringVectorBuilder::with_capacity(schema.num_columns())) + .collect(); + for row in self.rows.iter() { + for (i, name) in names.iter().enumerate() { + columns[i].push(row.get::(name).as_deref()); + } + } + let columns: Vec = columns + .into_iter() + .map(|mut col| Arc::new(col.finish()) as VectorRef) + .collect(); + + // construct recordbatch + let recordbatches = RecordBatches::try_from_columns(schema, columns) + .expect("Failed to construct recordbatches from columns. Please check the schema."); + let result_displayer = ResultDisplayer { + result: Ok(Output::new_with_record_batches(recordbatches)), + }; + write!(f, "{}", result_displayer)?; + + Ok(()) + } +} diff --git a/tests/runner/src/main.rs b/tests/runner/src/main.rs index 4474676802..eca72f280e 100644 --- a/tests/runner/src/main.rs +++ b/tests/runner/src/main.rs @@ -15,12 +15,15 @@ #![allow(clippy::print_stdout)] use std::path::PathBuf; +use std::sync::Arc; use clap::{Parser, ValueEnum}; use env::{Env, WalConfig}; +use sqlness::interceptor::Registry; use sqlness::{ConfigBuilder, Runner}; mod env; +mod protocol_interceptor; mod util; #[derive(ValueEnum, Debug, Clone)] @@ -30,6 +33,23 @@ enum Wal { Kafka, } +// add a group to ensure that all server addresses are set together +#[derive(clap::Args, Debug, Clone, Default)] +#[group(multiple = true, requires_all=["server_addr", "pg_server_addr", "mysql_server_addr"])] +struct ServerAddr { + /// Address of the grpc server. + #[clap(short, long)] + server_addr: Option, + + /// Address of the postgres server. Must be set if server_addr is set. + #[clap(short, long, requires = "server_addr")] + pg_server_addr: Option, + + /// Address of the mysql server. Must be set if server_addr is set. + #[clap(short, long, requires = "server_addr")] + mysql_server_addr: Option, +} + #[derive(Parser, Debug)] #[clap(author, version, about, long_about = None)] /// SQL Harness for GrepTimeDB @@ -50,9 +70,9 @@ struct Args { #[clap(short, long, default_value = ".*")] test_filter: String, - /// Address of the server. - #[clap(short, long)] - server_addr: Option, + /// Addresses of the server. + #[command(flatten)] + server_addr: ServerAddr, /// The type of Wal. #[clap(short, long, default_value = "raft_engine")] @@ -84,12 +104,19 @@ async fn main() { .unwrap(); let sqlness_home = temp_dir.into_path(); + let mut interceptor_registry: Registry = Default::default(); + interceptor_registry.register( + protocol_interceptor::PREFIX, + Arc::new(protocol_interceptor::ProtocolInterceptorFactory), + ); + let config = ConfigBuilder::default() .case_dir(util::get_case_dir(args.case_dir)) .fail_fast(args.fail_fast) .test_filter(args.test_filter) .follow_links(true) .env_config_file(args.env_config_file) + .interceptor_registry(interceptor_registry) .build() .unwrap(); @@ -107,7 +134,12 @@ async fn main() { let runner = Runner::new( config, - Env::new(sqlness_home.clone(), args.server_addr, wal, args.bins_dir), + Env::new( + sqlness_home.clone(), + args.server_addr.clone(), + wal, + args.bins_dir, + ), ); runner.run().await.unwrap(); diff --git a/tests/runner/src/protocol_interceptor.rs b/tests/runner/src/protocol_interceptor.rs new file mode 100644 index 0000000000..0f8a01e235 --- /dev/null +++ b/tests/runner/src/protocol_interceptor.rs @@ -0,0 +1,48 @@ +// 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 sqlness::interceptor::{Interceptor, InterceptorFactory, InterceptorRef}; +use sqlness::SqlnessError; + +pub const PROTOCOL_KEY: &str = "protocol"; +pub const POSTGRES: &str = "postgres"; +pub const MYSQL: &str = "mysql"; +pub const PREFIX: &str = "PROTOCOL"; + +pub struct ProtocolInterceptor { + protocol: String, +} + +impl Interceptor for ProtocolInterceptor { + fn before_execute(&self, _: &mut Vec, context: &mut sqlness::QueryContext) { + context + .context + .insert(PROTOCOL_KEY.to_string(), self.protocol.clone()); + } +} + +pub struct ProtocolInterceptorFactory; + +impl InterceptorFactory for ProtocolInterceptorFactory { + fn try_new(&self, ctx: &str) -> Result { + let protocol = ctx.to_lowercase(); + match protocol.as_str() { + POSTGRES | MYSQL => Ok(Box::new(ProtocolInterceptor { protocol })), + _ => Err(SqlnessError::InvalidContext { + prefix: PREFIX.to_string(), + msg: format!("Unsupported protocol: {}", ctx), + }), + } + } +} From a283e13da744b6bbe3b08e7b0940ac2d3e03e4c6 Mon Sep 17 00:00:00 2001 From: Kaifeng Zheng <100595273+Kev1n8@users.noreply.github.com> Date: Sat, 5 Oct 2024 02:05:40 +0800 Subject: [PATCH 054/128] feat: set max log files to 720 by default, info log only (#4787) * feat: set max log files to 720 by default, info log only * expose max_log_files in tomls * include dir info when panicing, limit max_log_files of err_log to 30, and that of slow_queries to opt.max_log_files * fix clippy * update config.md * update expected config str * limit err_log max files size to `max_log_files` too, include err info when panicing, put `max_l_f` in right position * fix typos * chore: config Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com> --------- Co-authored-by: dennis zhuang Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com> --- config/config.md | 5 +++ config/datanode.example.toml | 3 ++ config/flownode.example.toml | 3 ++ config/frontend.example.toml | 3 ++ config/metasrv.example.toml | 3 ++ config/standalone.example.toml | 3 ++ src/common/telemetry/src/logging.rs | 44 +++++++++++++++++++---- src/log-store/src/kafka/client_manager.rs | 2 +- tests-integration/tests/http.rs | 1 + 9 files changed, 60 insertions(+), 7 deletions(-) diff --git a/config/config.md b/config/config.md index d413936ff3..6a48ed717e 100644 --- a/config/config.md +++ b/config/config.md @@ -161,6 +161,7 @@ | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | +| `logging.max_log_files` | Integer | `720` | The maximum amount of log files. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | | `logging.slow_query` | -- | -- | The slow query log options. | @@ -251,6 +252,7 @@ | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | +| `logging.max_log_files` | Integer | `720` | The maximum amount of log files. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | | `logging.slow_query` | -- | -- | The slow query log options. | @@ -320,6 +322,7 @@ | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | +| `logging.max_log_files` | Integer | `720` | The maximum amount of log files. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | | `logging.slow_query` | -- | -- | The slow query log options. | @@ -476,6 +479,7 @@ | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | +| `logging.max_log_files` | Integer | `720` | The maximum amount of log files. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | | `logging.slow_query` | -- | -- | The slow query log options. | @@ -526,6 +530,7 @@ | `logging.otlp_endpoint` | String | `http://localhost:4317` | The OTLP tracing endpoint. | | `logging.append_stdout` | Bool | `true` | Whether to append logs to stdout. | | `logging.log_format` | String | `text` | The log format. Can be `text`/`json`. | +| `logging.max_log_files` | Integer | `720` | The maximum amount of log files. | | `logging.tracing_sample_ratio` | -- | -- | The percentage of tracing will be sampled and exported.
Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1.
ratio > 1 are treated as 1. Fractions < 0 are treated as 0 | | `logging.tracing_sample_ratio.default_ratio` | Float | `1.0` | -- | | `logging.slow_query` | -- | -- | The slow query log options. | diff --git a/config/datanode.example.toml b/config/datanode.example.toml index 06a59ebd6f..22670f857c 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -580,6 +580,9 @@ append_stdout = true ## The log format. Can be `text`/`json`. log_format = "text" +## The maximum amount of log files. +max_log_files = 720 + ## The percentage of tracing will be sampled and exported. ## Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1. ## ratio > 1 are treated as 1. Fractions < 0 are treated as 0 diff --git a/config/flownode.example.toml b/config/flownode.example.toml index 9d6d2fe5aa..34825542fa 100644 --- a/config/flownode.example.toml +++ b/config/flownode.example.toml @@ -78,6 +78,9 @@ append_stdout = true ## The log format. Can be `text`/`json`. log_format = "text" +## The maximum amount of log files. +max_log_files = 720 + ## The percentage of tracing will be sampled and exported. ## Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1. ## ratio > 1 are treated as 1. Fractions < 0 are treated as 0 diff --git a/config/frontend.example.toml b/config/frontend.example.toml index cc9698f61e..83e7808d46 100644 --- a/config/frontend.example.toml +++ b/config/frontend.example.toml @@ -185,6 +185,9 @@ append_stdout = true ## The log format. Can be `text`/`json`. log_format = "text" +## The maximum amount of log files. +max_log_files = 720 + ## The percentage of tracing will be sampled and exported. ## Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1. ## ratio > 1 are treated as 1. Fractions < 0 are treated as 0 diff --git a/config/metasrv.example.toml b/config/metasrv.example.toml index 8431940b45..416f5ee6ef 100644 --- a/config/metasrv.example.toml +++ b/config/metasrv.example.toml @@ -172,6 +172,9 @@ append_stdout = true ## The log format. Can be `text`/`json`. log_format = "text" +## The maximum amount of log files. +max_log_files = 720 + ## The percentage of tracing will be sampled and exported. ## Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1. ## ratio > 1 are treated as 1. Fractions < 0 are treated as 0 diff --git a/config/standalone.example.toml b/config/standalone.example.toml index 2d14b6550f..4c40d2dde6 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -624,6 +624,9 @@ append_stdout = true ## The log format. Can be `text`/`json`. log_format = "text" +## The maximum amount of log files. +max_log_files = 720 + ## The percentage of tracing will be sampled and exported. ## Valid range `[0, 1]`, 1 means all traces are sampled, 0 means all traces are not sampled, the default value is 1. ## ratio > 1 are treated as 1. Fractions < 0 are treated as 0 diff --git a/src/common/telemetry/src/logging.rs b/src/common/telemetry/src/logging.rs index c21766f04d..4c5a9ee18f 100644 --- a/src/common/telemetry/src/logging.rs +++ b/src/common/telemetry/src/logging.rs @@ -54,6 +54,9 @@ pub struct LoggingOptions { /// The log format that can be one of "json" or "text". Default is "text". pub log_format: LogFormat, + /// The maximum number of log files set by default. + pub max_log_files: usize, + /// Whether to append logs to stdout. Default is true. pub append_stdout: bool, @@ -116,6 +119,8 @@ impl Default for LoggingOptions { tracing_sample_ratio: None, append_stdout: true, slow_query: SlowQueryOptions::default(), + // Rotation hourly, 24 files per day, keeps info log files of 30 days + max_log_files: 720, } } } @@ -206,8 +211,17 @@ pub fn init_global_logging( // Configure the file logging layer with rolling policy. let file_logging_layer = if !opts.dir.is_empty() { - let rolling_appender = - RollingFileAppender::new(Rotation::HOURLY, &opts.dir, "greptimedb"); + let rolling_appender = RollingFileAppender::builder() + .rotation(Rotation::HOURLY) + .filename_prefix("greptimedb") + .max_log_files(opts.max_log_files) + .build(&opts.dir) + .unwrap_or_else(|e| { + panic!( + "initializing rolling file appender at {} failed: {}", + &opts.dir, e + ) + }); let (writer, guard) = tracing_appender::non_blocking(rolling_appender); guards.push(guard); @@ -228,8 +242,17 @@ pub fn init_global_logging( // Configure the error file logging layer with rolling policy. let err_file_logging_layer = if !opts.dir.is_empty() { - let rolling_appender = - RollingFileAppender::new(Rotation::HOURLY, &opts.dir, "greptimedb-err"); + let rolling_appender = RollingFileAppender::builder() + .rotation(Rotation::HOURLY) + .filename_prefix("greptimedb-err") + .max_log_files(opts.max_log_files) + .build(&opts.dir) + .unwrap_or_else(|e| { + panic!( + "initializing rolling file appender at {} failed: {}", + &opts.dir, e + ) + }); let (writer, guard) = tracing_appender::non_blocking(rolling_appender); guards.push(guard); @@ -256,8 +279,17 @@ pub fn init_global_logging( }; let slow_query_logging_layer = if !opts.dir.is_empty() && opts.slow_query.enable { - let rolling_appender = - RollingFileAppender::new(Rotation::HOURLY, &opts.dir, "greptimedb-slow-queries"); + let rolling_appender = RollingFileAppender::builder() + .rotation(Rotation::HOURLY) + .filename_prefix("greptimedb-slow-queries") + .max_log_files(opts.max_log_files) + .build(&opts.dir) + .unwrap_or_else(|e| { + panic!( + "initializing rolling file appender at {} failed: {}", + &opts.dir, e + ) + }); let (writer, guard) = tracing_appender::non_blocking(rolling_appender); guards.push(guard); diff --git a/src/log-store/src/kafka/client_manager.rs b/src/log-store/src/kafka/client_manager.rs index 589d5a900e..6dfb97b027 100644 --- a/src/log-store/src/kafka/client_manager.rs +++ b/src/log-store/src/kafka/client_manager.rs @@ -182,7 +182,7 @@ mod tests { use super::*; - /// Creates `num_topiocs` number of topics each will be decorated by the given decorator. + /// Creates `num_topics` number of topics each will be decorated by the given decorator. pub async fn create_topics( num_topics: usize, decorator: F, diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 5f3eb854bb..20a0f4edd2 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -833,6 +833,7 @@ max_retry_times = 3 retry_delay = "500ms" [logging] +max_log_files = 720 append_stdout = true enable_otlp_tracing = false From 4dea4cac47ebdd83cae2277392c61ac6fb9653b2 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Tue, 8 Oct 2024 00:33:24 -0700 Subject: [PATCH 055/128] refactor: change sqlness ports to avoid conflict with local instance (#4794) --- .../information_schema/cluster_info.result | 20 +++---- .../information_schema/cluster_info.sql | 10 ++-- tests/conf/datanode-test.toml.template | 4 +- tests/conf/standalone-test.toml.template | 8 +-- tests/runner/src/env.rs | 55 +++++++++++++------ 5 files changed, 59 insertions(+), 38 deletions(-) diff --git a/tests/cases/distributed/information_schema/cluster_info.result b/tests/cases/distributed/information_schema/cluster_info.result index a59d59ee60..9f19362591 100644 --- a/tests/cases/distributed/information_schema/cluster_info.result +++ b/tests/cases/distributed/information_schema/cluster_info.result @@ -20,52 +20,52 @@ DESC TABLE CLUSTER_INFO; -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO ORDER BY peer_type; -+++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|DATANODE|127.0.0.1:4101|Version|Hash|Start_time|Duration|Duration||2|DATANODE|127.0.0.1:4102|Version|Hash|Start_time|Duration|Duration||3|DATANODE|127.0.0.1:4103|Version|Hash|Start_time|Duration|Duration||0|FLOWNODE|127.0.0.1:6800|Version|Hash|Start_time|Duration|Duration||1|FRONTEND|127.0.0.1:4001|Version|Hash|Start_time|Duration|Duration||1|METASRV|127.0.0.1:3002|Version|Hash|Start_time|Duration||+++++++++ ++++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|DATANODE|127.0.0.1:29411|Version|Hash|Start_time|Duration|Duration||2|DATANODE|127.0.0.1:29412|Version|Hash|Start_time|Duration|Duration||3|DATANODE|127.0.0.1:29413|Version|Hash|Start_time|Duration|Duration||0|FLOWNODE|127.0.0.1:29680|Version|Hash|Start_time|Duration|Duration||1|FRONTEND|127.0.0.1:29401|Version|Hash|Start_time|Duration|Duration||1|METASRV|127.0.0.1:29302|Version|Hash|Start_time|Duration||+++++++++ -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE = 'METASRV' ORDER BY peer_type; -+++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|METASRV|127.0.0.1:3002|Version|Hash|Start_time|Duration||+++++++++ ++++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|METASRV|127.0.0.1:29302|Version|Hash|Start_time|Duration||+++++++++ -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE = 'FRONTEND' ORDER BY peer_type; -+++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|FRONTEND|127.0.0.1:4001|Version|Hash|Start_time|Duration|Duration|+++++++++ ++++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|FRONTEND|127.0.0.1:29401|Version|Hash|Start_time|Duration|Duration|+++++++++ -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE != 'FRONTEND' ORDER BY peer_type; -+++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|DATANODE|127.0.0.1:4101|Version|Hash|Start_time|Duration|Duration||2|DATANODE|127.0.0.1:4102|Version|Hash|Start_time|Duration|Duration||3|DATANODE|127.0.0.1:4103|Version|Hash|Start_time|Duration|Duration||0|FLOWNODE|127.0.0.1:6800|Version|Hash|Start_time|Duration|Duration||1|METASRV|127.0.0.1:3002|Version|Hash|Start_time|Duration||+++++++++ ++++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|1|DATANODE|127.0.0.1:29411|Version|Hash|Start_time|Duration|Duration||2|DATANODE|127.0.0.1:29412|Version|Hash|Start_time|Duration|Duration||3|DATANODE|127.0.0.1:29413|Version|Hash|Start_time|Duration|Duration||0|FLOWNODE|127.0.0.1:29680|Version|Hash|Start_time|Duration|Duration||1|METASRV|127.0.0.1:29302|Version|Hash|Start_time|Duration||+++++++++ -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_ID > 1 ORDER BY peer_type; -+++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|2|DATANODE|127.0.0.1:4102|Version|Hash|Start_time|Duration|Duration||3|DATANODE|127.0.0.1:4103|Version|Hash|Start_time|Duration|Duration|+++++++++ ++++++++++|peer_id|peer_type|peer_addr|node_version|git_commit|start_time|uptime|active_time|+++++++++|2|DATANODE|127.0.0.1:29412|Version|Hash|Start_time|Duration|Duration||3|DATANODE|127.0.0.1:29413|Version|Hash|Start_time|Duration|Duration|+++++++++ USE PUBLIC; diff --git a/tests/cases/distributed/information_schema/cluster_info.sql b/tests/cases/distributed/information_schema/cluster_info.sql index f017e2f7fd..ec0f3bb273 100644 --- a/tests/cases/distributed/information_schema/cluster_info.sql +++ b/tests/cases/distributed/information_schema/cluster_info.sql @@ -5,7 +5,7 @@ DESC TABLE CLUSTER_INFO; -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO ORDER BY peer_type; @@ -13,7 +13,7 @@ SELECT * FROM CLUSTER_INFO ORDER BY peer_type; -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE = 'METASRV' ORDER BY peer_type; @@ -21,7 +21,7 @@ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE = 'METASRV' ORDER BY peer_type; -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE = 'FRONTEND' ORDER BY peer_type; @@ -29,7 +29,7 @@ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE = 'FRONTEND' ORDER BY peer_type; -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE != 'FRONTEND' ORDER BY peer_type; @@ -37,7 +37,7 @@ SELECT * FROM CLUSTER_INFO WHERE PEER_TYPE != 'FRONTEND' ORDER BY peer_type; -- SQLNESS REPLACE version node_version -- SQLNESS REPLACE (\s\d\.\d\.\d\s) Version -- SQLNESS REPLACE (\s[a-z0-9]{7,8}\s) Hash --- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) Start_time +-- SQLNESS REPLACE (\s[\-0-9T:\.]{19,}) Start_time -- SQLNESS REPLACE ((\d+(s|ms|m)\s)+) Duration -- SQLNESS REPLACE [\s\-]+ SELECT * FROM CLUSTER_INFO WHERE PEER_ID > 1 ORDER BY peer_type; diff --git a/tests/conf/datanode-test.toml.template b/tests/conf/datanode-test.toml.template index 04968bc956..5ed5352124 100644 --- a/tests/conf/datanode-test.toml.template +++ b/tests/conf/datanode-test.toml.template @@ -1,7 +1,7 @@ node_id = 1 mode = 'distributed' require_lease_before_startup = true -rpc_addr = '127.0.0.1:4100' +rpc_addr = '127.0.0.1:29410' rpc_hostname = '127.0.0.1' rpc_runtime_size = 8 @@ -24,7 +24,7 @@ type = 'File' data_home = '{data_home}' [meta_client_options] -metasrv_addrs = ['127.0.0.1:3002'] +metasrv_addrs = ['127.0.0.1:29302'] timeout_millis = 3000 connect_timeout_millis = 5000 tcp_nodelay = false diff --git a/tests/conf/standalone-test.toml.template b/tests/conf/standalone-test.toml.template index f96a815762..da51f3b3f4 100644 --- a/tests/conf/standalone-test.toml.template +++ b/tests/conf/standalone-test.toml.template @@ -20,13 +20,13 @@ linger = "5ms" type = 'File' data_home = '{data_home}' -[grpc_options] -addr = '127.0.0.1:4001' +[grpc] +addr = '127.0.0.1:29401' runtime_size = 8 [mysql] enable = true -addr = "127.0.0.1:4002" +addr = "127.0.0.1:29402" runtime_size = 2 [mysql.tls] @@ -34,7 +34,7 @@ mode = "disable" [postgres] enable = true -addr = "127.0.0.1:4003" +addr = "127.0.0.1:29403" runtime_size = 2 [procedure] diff --git a/tests/runner/src/env.rs b/tests/runner/src/env.rs index 0dc0e3a9e8..bb5d74a267 100644 --- a/tests/runner/src/env.rs +++ b/tests/runner/src/env.rs @@ -47,10 +47,10 @@ use tokio_postgres::{Client as PgClient, SimpleQueryMessage as PgRow}; use crate::protocol_interceptor::{MYSQL, PROTOCOL_KEY}; use crate::{util, ServerAddr}; -const METASRV_ADDR: &str = "127.0.0.1:3002"; -const GRPC_SERVER_ADDR: &str = "127.0.0.1:4001"; -const MYSQL_SERVER_ADDR: &str = "127.0.0.1:4002"; -const POSTGRES_SERVER_ADDR: &str = "127.0.0.1:4003"; +const METASRV_ADDR: &str = "127.0.0.1:29302"; +const GRPC_SERVER_ADDR: &str = "127.0.0.1:29401"; +const MYSQL_SERVER_ADDR: &str = "127.0.0.1:29402"; +const POSTGRES_SERVER_ADDR: &str = "127.0.0.1:29403"; const DEFAULT_LOG_LEVEL: &str = "--log-level=debug,hyper=warn,tower=warn,datafusion=warn,reqwest=warn,sqlparser=warn,h2=info,opendal=info"; #[derive(Clone)] @@ -305,34 +305,55 @@ impl Env { ), "-c".to_string(), self.generate_config_file(subcommand, db_ctx), - "--http-addr=127.0.0.1:5002".to_string(), + "--http-addr=127.0.0.1:29502".to_string(), ]; - (args, vec![GRPC_SERVER_ADDR.to_string()]) + ( + args, + vec![ + GRPC_SERVER_ADDR.to_string(), + MYSQL_SERVER_ADDR.to_string(), + POSTGRES_SERVER_ADDR.to_string(), + ], + ) } "frontend" => { let args = vec![ DEFAULT_LOG_LEVEL.to_string(), subcommand.to_string(), "start".to_string(), - "--metasrv-addrs=127.0.0.1:3002".to_string(), - "--http-addr=127.0.0.1:5003".to_string(), + "--metasrv-addrs=127.0.0.1:29302".to_string(), + "--http-addr=127.0.0.1:29503".to_string(), + format!("--rpc-addr={}", GRPC_SERVER_ADDR), + format!("--mysql-addr={}", MYSQL_SERVER_ADDR), + format!("--postgres-addr={}", POSTGRES_SERVER_ADDR), format!( "--log-dir={}/greptimedb-frontend/logs", self.sqlness_home.display() ), ]; - (args, vec![GRPC_SERVER_ADDR.to_string()]) + ( + args, + vec![ + GRPC_SERVER_ADDR.to_string(), + MYSQL_SERVER_ADDR.to_string(), + POSTGRES_SERVER_ADDR.to_string(), + ], + ) } "metasrv" => { let args = vec![ DEFAULT_LOG_LEVEL.to_string(), subcommand.to_string(), "start".to_string(), + "--bind-addr".to_string(), + "127.0.0.1:29302".to_string(), + "--server-addr".to_string(), + "127.0.0.1:29302".to_string(), "--backend".to_string(), "memory-store".to_string(), "--enable-region-failover".to_string(), "false".to_string(), - "--http-addr=127.0.0.1:5002".to_string(), + "--http-addr=127.0.0.1:29502".to_string(), format!( "--log-dir={}/greptimedb-metasrv/logs", self.sqlness_home.display() @@ -396,15 +417,15 @@ impl Env { subcommand.to_string(), "start".to_string(), ]; - args.push(format!("--rpc-addr=127.0.0.1:410{id}")); - args.push(format!("--http-addr=127.0.0.1:430{id}")); + args.push(format!("--rpc-addr=127.0.0.1:2941{id}")); + args.push(format!("--http-addr=127.0.0.1:2943{id}")); args.push(format!("--data-home={}", data_home.display())); args.push(format!("--log-dir={}/logs", data_home.display())); args.push(format!("--node-id={id}")); args.push("-c".to_string()); args.push(self.generate_config_file(subcommand, db_ctx)); - args.push("--metasrv-addrs=127.0.0.1:3002".to_string()); - (args, format!("127.0.0.1:410{id}")) + args.push("--metasrv-addrs=127.0.0.1:29302".to_string()); + (args, format!("127.0.0.1:2941{id}")) } fn flownode_start_args( @@ -420,14 +441,14 @@ impl Env { subcommand.to_string(), "start".to_string(), ]; - args.push(format!("--rpc-addr=127.0.0.1:680{id}")); + args.push(format!("--rpc-addr=127.0.0.1:2968{id}")); args.push(format!("--node-id={id}")); args.push(format!( "--log-dir={}/greptimedb-flownode/logs", sqlness_home.display() )); - args.push("--metasrv-addrs=127.0.0.1:3002".to_string()); - (args, format!("127.0.0.1:680{id}")) + args.push("--metasrv-addrs=127.0.0.1:29302".to_string()); + (args, format!("127.0.0.1:2968{id}")) } /// stop and restart the server process From 2cdd103874a649282612bd05f94023fc0556f6b2 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Tue, 8 Oct 2024 17:02:06 +0800 Subject: [PATCH 056/128] feat: introduce `HeartbeatHandlerGroupBuilderCustomizer` (#4803) * feat: introduce `HeartbeatHandlerGroupBuilderFinalizer` * chore: rename to `HeartbeatHandlerGroupBuilderCustomizer` --- src/meta-srv/src/handler.rs | 42 +++++++++++++++++++++-------- src/meta-srv/src/metasrv/builder.rs | 2 +- 2 files changed, 32 insertions(+), 12 deletions(-) diff --git a/src/meta-srv/src/handler.rs b/src/meta-srv/src/handler.rs index 9d82920c08..b45a1ac7e3 100644 --- a/src/meta-srv/src/handler.rs +++ b/src/meta-srv/src/handler.rs @@ -527,11 +527,22 @@ impl HeartbeatHandlerGroupBuilder { } /// Builds the group of heartbeat handlers. - pub fn build(self) -> HeartbeatHandlerGroup { - HeartbeatHandlerGroup { + /// + /// Applies the customizer if it exists. + pub fn build(mut self) -> Result { + if let Some(customizer) = self + .plugins + .as_ref() + .and_then(|plugins| plugins.get::()) + { + debug!("Customizing the heartbeat handler group builder"); + customizer.customize(&mut self)?; + } + + Ok(HeartbeatHandlerGroup { handlers: self.handlers.into_iter().collect(), pushers: self.pushers, - } + }) } /// Adds the handler after the specified handler. @@ -582,6 +593,14 @@ impl HeartbeatHandlerGroupBuilder { } } +pub type HeartbeatHandlerGroupBuilderCustomizerRef = + Arc; + +/// The customizer of the [`HeartbeatHandlerGroupBuilder`]. +pub trait HeartbeatHandlerGroupBuilderCustomizer: Send + Sync { + fn customize(&self, builder: &mut HeartbeatHandlerGroupBuilder) -> Result<()>; +} + #[cfg(test)] mod tests { @@ -670,7 +689,8 @@ mod tests { fn test_handler_group_builder() { let group = HeartbeatHandlerGroupBuilder::new(Pushers::default()) .add_default_handlers() - .build(); + .build() + .unwrap(); let handlers = group.handlers; assert_eq!(12, handlers.len()); @@ -706,7 +726,7 @@ mod tests { ) .unwrap(); - let group = builder.build(); + let group = builder.build().unwrap(); let handlers = group.handlers; assert_eq!(13, handlers.len()); @@ -739,7 +759,7 @@ mod tests { .add_handler_before("ResponseHeaderHandler", CollectStatsHandler::default()) .unwrap(); - let group = builder.build(); + let group = builder.build().unwrap(); let handlers = group.handlers; assert_eq!(13, handlers.len()); @@ -772,7 +792,7 @@ mod tests { .add_handler_after("MailboxHandler", CollectStatsHandler::default()) .unwrap(); - let group = builder.build(); + let group = builder.build().unwrap(); let handlers = group.handlers; assert_eq!(13, handlers.len()); @@ -805,7 +825,7 @@ mod tests { .add_handler_after("CollectStatsHandler", ResponseHeaderHandler) .unwrap(); - let group = builder.build(); + let group = builder.build().unwrap(); let handlers = group.handlers; assert_eq!(13, handlers.len()); @@ -838,7 +858,7 @@ mod tests { .replace_handler("MailboxHandler", CollectStatsHandler::default()) .unwrap(); - let group = builder.build(); + let group = builder.build().unwrap(); let handlers = group.handlers; assert_eq!(12, handlers.len()); @@ -870,7 +890,7 @@ mod tests { .replace_handler("CollectStatsHandler", ResponseHeaderHandler) .unwrap(); - let group = builder.build(); + let group = builder.build().unwrap(); let handlers = group.handlers; assert_eq!(12, handlers.len()); @@ -902,7 +922,7 @@ mod tests { .replace_handler("ResponseHeaderHandler", CollectStatsHandler::default()) .unwrap(); - let group = builder.build(); + let group = builder.build().unwrap(); let handlers = group.handlers; assert_eq!(12, handlers.len()); diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 88d7e5bd6d..050f825174 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -363,7 +363,7 @@ impl MetasrvBuilder { .with_region_failure_handler(region_failover_handler) .with_region_lease_handler(Some(region_lease_handler)) .add_default_handlers() - .build() + .build()? } }; From 71a66d15f7e8353000f966dd1dc05bc58733cf15 Mon Sep 17 00:00:00 2001 From: localhost Date: Tue, 8 Oct 2024 20:11:09 +0800 Subject: [PATCH 057/128] chore: add json write (#4744) * chore: add json write * chore: add test for write json log api * chore: enhancement of Error Handling * chore: fix by pr comment * chore: fix by pr comment * chore: enhancement of error content and add some doc --- Cargo.lock | 1 + src/pipeline/Cargo.toml | 1 + src/pipeline/src/etl/error.rs | 118 ++++--- src/pipeline/src/etl/transform/transformer.rs | 1 + .../src/etl/transform/transformer/greptime.rs | 313 +++++++++++++++++- src/pipeline/src/lib.rs | 1 + src/servers/src/http/event.rs | 90 ++--- tests-integration/tests/http.rs | 71 ++++ 8 files changed, 505 insertions(+), 91 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 418e68a88a..c2356a4c2e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8141,6 +8141,7 @@ dependencies = [ "futures", "greptime-proto", "itertools 0.10.5", + "jsonb", "lazy_static", "moka", "once_cell", diff --git a/src/pipeline/Cargo.toml b/src/pipeline/Cargo.toml index 5b85b999fe..2402605f7f 100644 --- a/src/pipeline/Cargo.toml +++ b/src/pipeline/Cargo.toml @@ -40,6 +40,7 @@ enum_dispatch = "0.3" futures.workspace = true greptime-proto.workspace = true itertools.workspace = true +jsonb.workspace = true lazy_static.workspace = true moka = { workspace = true, features = ["sync"] } once_cell.workspace = true diff --git a/src/pipeline/src/etl/error.rs b/src/pipeline/src/etl/error.rs index 3680053ba0..0a28cb7423 100644 --- a/src/pipeline/src/etl/error.rs +++ b/src/pipeline/src/etl/error.rs @@ -40,7 +40,7 @@ pub enum Error { location: Location, }, - #[snafu(display("{processor} processor: missing field: {field}"))] + #[snafu(display("Processor {processor}: missing field: {field}"))] ProcessorMissingField { processor: String, field: String, @@ -48,7 +48,7 @@ pub enum Error { location: Location, }, - #[snafu(display("{processor} processor: expect string value, but got {v:?}"))] + #[snafu(display("Processor {processor}: expect string value, but got {v:?}"))] ProcessorExpectString { processor: String, v: crate::etl::Value, @@ -56,7 +56,7 @@ pub enum Error { location: Location, }, - #[snafu(display("{processor} processor: unsupported value {val}"))] + #[snafu(display("Processor {processor}: unsupported value {val}"))] ProcessorUnsupportedValue { processor: &'static str, val: String, @@ -64,13 +64,13 @@ pub enum Error { location: Location, }, - #[snafu(display("processor key must be a string"))] + #[snafu(display("Processor key must be a string"))] ProcessorKeyMustBeString { #[snafu(implicit)] location: Location, }, - #[snafu(display("{kind} processor: failed to parse {value}"))] + #[snafu(display("Processor {kind}: failed to parse {value}"))] ProcessorFailedToParseString { kind: String, value: String, @@ -78,13 +78,13 @@ pub enum Error { location: Location, }, - #[snafu(display("processor must have a string key"))] + #[snafu(display("Processor must have a string key"))] ProcessorMustHaveStringKey { #[snafu(implicit)] location: Location, }, - #[snafu(display("unsupported {processor} processor"))] + #[snafu(display("Unsupported {processor} processor"))] UnsupportedProcessor { processor: String, #[snafu(implicit)] @@ -108,7 +108,7 @@ pub enum Error { location: Location, }, - #[snafu(display("failed to parse {key} as int: {value}"))] + #[snafu(display("Failed to parse {key} as int: {value}"))] FailedToParseIntKey { key: String, value: String, @@ -126,7 +126,7 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("failed to parse {key} as float: {value}"))] + #[snafu(display("Failed to parse {key} as float: {value}"))] FailedToParseFloatKey { key: String, value: String, @@ -136,7 +136,7 @@ pub enum Error { location: Location, }, - #[snafu(display("{kind} processor.{key} not found in intermediate keys"))] + #[snafu(display("Processor {kind}: {key} not found in intermediate keys"))] IntermediateKeyIndex { kind: String, key: String, @@ -144,41 +144,41 @@ pub enum Error { location: Location, }, - #[snafu(display("{k} missing value in {s}"))] + #[snafu(display("Cmcd {k} missing value in {s}"))] CmcdMissingValue { k: String, s: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("{part} missing key in {s}"))] + #[snafu(display("Part: {part} missing key in {s}"))] CmcdMissingKey { part: String, s: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("key must be a string, but got {k:?}"))] + #[snafu(display("Key must be a string, but got {k:?}"))] KeyMustBeString { k: yaml_rust::Yaml, #[snafu(implicit)] location: Location, }, - #[snafu(display("csv read error"))] + #[snafu(display("Csv read error"))] CsvRead { #[snafu(implicit)] location: Location, #[snafu(source)] error: csv::Error, }, - #[snafu(display("expected at least one record from csv format, but got none"))] + #[snafu(display("Expected at least one record from csv format, but got none"))] CsvNoRecord { #[snafu(implicit)] location: Location, }, - #[snafu(display("'{separator}' must be a single character, but got '{value}'"))] + #[snafu(display("Separator '{separator}' must be a single character, but got '{value}'"))] CsvSeparatorName { separator: &'static str, value: String, @@ -186,7 +186,7 @@ pub enum Error { location: Location, }, - #[snafu(display("'{quote}' must be a single character, but got '{value}'"))] + #[snafu(display("Quote '{quote}' must be a single character, but got '{value}'"))] CsvQuoteName { quote: &'static str, value: String, @@ -212,19 +212,19 @@ pub enum Error { location: Location, }, - #[snafu(display("failed to get local timezone"))] + #[snafu(display("Failed to get local timezone"))] DateFailedToGetLocalTimezone { #[snafu(implicit)] location: Location, }, - #[snafu(display("failed to get timestamp"))] + #[snafu(display("Failed to get timestamp"))] DateFailedToGetTimestamp { #[snafu(implicit)] location: Location, }, - #[snafu(display("{processor} processor: invalid format {s}"))] + #[snafu(display("Processor {processor}: invalid format {s}"))] DateInvalidFormat { s: String, processor: String, @@ -245,20 +245,20 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("'{split}' exceeds the input"))] + #[snafu(display("Split: '{split}' exceeds the input"))] DissectSplitExceedsInput { split: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("'{split}' does not match the input '{input}'"))] + #[snafu(display("Split: '{split}' does not match the input '{input}'"))] DissectSplitNotMatchInput { split: String, input: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("consecutive names are not allowed: '{name1}' '{name2}'"))] + #[snafu(display("Consecutive names are not allowed: '{name1}' '{name2}'"))] DissectConsecutiveNames { name1: String, name2: String, @@ -270,7 +270,7 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("'{m}' modifier already set, but found {modifier}"))] + #[snafu(display("Modifier '{m}' already set, but found {modifier}"))] DissectModifierAlreadySet { m: String, modifier: String, @@ -304,23 +304,23 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("invalid resolution: {resolution}"))] + #[snafu(display("Invalid resolution: {resolution}"))] EpochInvalidResolution { resolution: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("pattern is required"))] + #[snafu(display("Pattern is required"))] GsubPatternRequired { #[snafu(implicit)] location: Location, }, - #[snafu(display("replacement is required"))] + #[snafu(display("Replacement is required"))] GsubReplacementRequired { #[snafu(implicit)] location: Location, }, - #[snafu(display("invalid regex pattern: {pattern}"))] + #[snafu(display("Invalid regex pattern: {pattern}"))] Regex { #[snafu(source)] error: regex::Error, @@ -328,72 +328,72 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("separator is required"))] + #[snafu(display("Separator is required"))] JoinSeparatorRequired { #[snafu(implicit)] location: Location, }, - #[snafu(display("invalid method: {method}"))] + #[snafu(display("Invalid method: {method}"))] LetterInvalidMethod { method: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("no named group found in regex {origin}"))] + #[snafu(display("No named group found in regex {origin}"))] RegexNamedGroupNotFound { origin: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("no valid field found in {processor} processor"))] + #[snafu(display("No valid field found in {processor} processor"))] RegexNoValidField { processor: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("no valid pattern found in {processor} processor"))] + #[snafu(display("No valid pattern found in {processor} processor"))] RegexNoValidPattern { processor: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("invalid method: {s}"))] + #[snafu(display("Invalid method: {s}"))] UrlEncodingInvalidMethod { s: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("url decoding error"))] + #[snafu(display("Url decoding error"))] UrlEncodingDecode { #[snafu(source)] error: std::string::FromUtf8Error, #[snafu(implicit)] location: Location, }, - #[snafu(display("invalid transform on_failure value: {value}"))] + #[snafu(display("Invalid transform on_failure value: {value}"))] TransformOnFailureInvalidValue { value: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("transform element must be a map"))] + #[snafu(display("Transform element must be a map"))] TransformElementMustBeMap { #[snafu(implicit)] location: Location, }, - #[snafu(display("transform {fields:?} type MUST BE set before default {default}"))] + #[snafu(display("Transform {fields:?} type MUST BE set before default {default}"))] TransformTypeMustBeSet { fields: String, default: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("transform cannot be empty"))] + #[snafu(display("Transform cannot be empty"))] TransformEmpty { #[snafu(implicit)] location: Location, }, - #[snafu(display("column name must be unique, but got duplicated: {duplicates}"))] + #[snafu(display("Column name must be unique, but got duplicated: {duplicates}"))] TransformColumnNameMustBeUnique { duplicates: String, #[snafu(implicit)] @@ -407,7 +407,7 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("transform must have exactly one field specified as timestamp Index, but got {count}: {columns}"))] + #[snafu(display("Transform must have exactly one field specified as timestamp Index, but got {count}: {columns}"))] TransformTimestampIndexCount { count: usize, columns: String, @@ -425,13 +425,13 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("{ty} value not supported for Epoch"))] + #[snafu(display("Type: {ty} value not supported for Epoch"))] CoerceUnsupportedEpochType { ty: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("failed to coerce string value '{s}' to type '{ty}'"))] + #[snafu(display("Failed to coerce string value '{s}' to type '{ty}'"))] CoerceStringToType { s: String, ty: String, @@ -440,7 +440,7 @@ pub enum Error { }, #[snafu(display( - "invalid resolution: '{resolution}'. Available resolutions: {valid_resolution}" + "Invalid resolution: '{resolution}'. Available resolutions: {valid_resolution}" ))] ValueInvalidResolution { resolution: String, @@ -449,14 +449,14 @@ pub enum Error { location: Location, }, - #[snafu(display("failed to parse type: '{t}'"))] + #[snafu(display("Failed to parse type: '{t}'"))] ValueParseType { t: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("failed to parse {ty}: {v}"))] + #[snafu(display("Failed to parse {ty}: {v}"))] ValueParseInt { ty: String, v: String, @@ -466,7 +466,7 @@ pub enum Error { location: Location, }, - #[snafu(display("failed to parse {ty}: {v}"))] + #[snafu(display("Failed to parse {ty}: {v}"))] ValueParseFloat { ty: String, v: String, @@ -476,7 +476,7 @@ pub enum Error { location: Location, }, - #[snafu(display("failed to parse {ty}: {v}"))] + #[snafu(display("Failed to parse {ty}: {v}"))] ValueParseBoolean { ty: String, v: String, @@ -485,19 +485,19 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("default value not unsupported for type {value}"))] + #[snafu(display("Default value not unsupported for type {value}"))] ValueDefaultValueUnsupported { value: String, #[snafu(implicit)] location: Location, }, - #[snafu(display("unsupported number type: {value}"))] + #[snafu(display("Unsupported number type: {value}"))] ValueUnsupportedNumberType { value: serde_json::Number, #[snafu(implicit)] location: Location, }, - #[snafu(display("unsupported yaml type: {value:?}"))] + #[snafu(display("Unsupported yaml type: {value:?}"))] ValueUnsupportedYamlType { value: yaml_rust::Yaml, #[snafu(implicit)] @@ -531,12 +531,26 @@ pub enum Error { #[snafu(implicit)] location: Location, }, - #[snafu(display("unsupported index type: {value}"))] + #[snafu(display("Unsupported index type: {value}"))] UnsupportedIndexType { value: String, #[snafu(implicit)] location: Location, }, + #[snafu(display("Unsupported number type: {value:?}"))] + UnsupportedNumberType { + value: serde_json::Number, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Column datatype mismatch. For column: {column}, expected datatype: {expected}, actual datatype: {actual}"))] + IdentifyPipelineColumnTypeMismatch { + column: String, + expected: String, + actual: String, + #[snafu(implicit)] + location: Location, + }, } pub type Result = std::result::Result; diff --git a/src/pipeline/src/etl/transform/transformer.rs b/src/pipeline/src/etl/transform/transformer.rs index 28aacdcbfb..7a18524662 100644 --- a/src/pipeline/src/etl/transform/transformer.rs +++ b/src/pipeline/src/etl/transform/transformer.rs @@ -13,3 +13,4 @@ // limitations under the License. pub mod greptime; +pub use greptime::identity_pipeline; diff --git a/src/pipeline/src/etl/transform/transformer/greptime.rs b/src/pipeline/src/etl/transform/transformer/greptime.rs index 000a2ddc26..11d63c4503 100644 --- a/src/pipeline/src/etl/transform/transformer/greptime.rs +++ b/src/pipeline/src/etl/transform/transformer/greptime.rs @@ -16,13 +16,20 @@ pub mod coerce; use std::collections::HashSet; +use ahash::HashMap; +use api::helper::proto_value_type; +use api::v1::column_data_type_extension::TypeExt; +use api::v1::value::ValueData; +use api::v1::{ColumnDataType, ColumnDataTypeExtension, JsonTypeExtension, SemanticType}; use coerce::{coerce_columns, coerce_value}; use greptime_proto::v1::{ColumnSchema, Row, Rows, Value as GreptimeValue}; use itertools::Itertools; +use serde_json::{Map, Number}; use crate::etl::error::{ - Result, TransformColumnNameMustBeUniqueSnafu, TransformEmptySnafu, - TransformMultipleTimestampIndexSnafu, TransformTimestampIndexCountSnafu, + IdentifyPipelineColumnTypeMismatchSnafu, Result, TransformColumnNameMustBeUniqueSnafu, + TransformEmptySnafu, TransformMultipleTimestampIndexSnafu, TransformTimestampIndexCountSnafu, + UnsupportedNumberTypeSnafu, }; use crate::etl::field::{InputFieldInfo, OneInputOneOutputField}; use crate::etl::transform::index::Index; @@ -120,6 +127,7 @@ impl Transformer for GreptimeTransformer { if let Some(idx) = transform.index { if idx == Index::Time { match transform.real_fields.len() { + //Safety unwrap is fine here because we have checked the length of real_fields 1 => timestamp_columns .push(transform.real_fields.first().unwrap().input_name()), _ => { @@ -194,3 +202,304 @@ impl Transformer for GreptimeTransformer { &mut self.transforms } } + +/// This is used to record the current state schema information and a sequential cache of field names. +/// As you traverse the user input JSON, this will change. +/// It will record a superset of all user input schemas. +#[derive(Debug, Default)] +struct SchemaInfo { + /// schema info + schema: Vec, + /// index of the column name + index: HashMap, +} + +fn resolve_schema( + index: Option, + value_data: ValueData, + column_schema: ColumnSchema, + row: &mut Vec, + schema_info: &mut SchemaInfo, +) -> Result<()> { + if let Some(index) = index { + let api_value = GreptimeValue { + value_data: Some(value_data), + }; + // Safety unwrap is fine here because api_value is always valid + let value_column_data_type = proto_value_type(&api_value).unwrap(); + // Safety unwrap is fine here because index is always valid + let schema_column_data_type = schema_info.schema.get(index).unwrap().datatype(); + if value_column_data_type != schema_column_data_type { + IdentifyPipelineColumnTypeMismatchSnafu { + column: column_schema.column_name, + expected: schema_column_data_type.as_str_name(), + actual: value_column_data_type.as_str_name(), + } + .fail() + } else { + row[index] = api_value; + Ok(()) + } + } else { + let key = column_schema.column_name.clone(); + schema_info.schema.push(column_schema); + schema_info.index.insert(key, schema_info.schema.len() - 1); + let api_value = GreptimeValue { + value_data: Some(value_data), + }; + row.push(api_value); + Ok(()) + } +} + +fn resolve_number_schema( + n: Number, + column_name: String, + index: Option, + row: &mut Vec, + schema_info: &mut SchemaInfo, +) -> Result<()> { + let (value, datatype, semantic_type) = if n.is_i64() { + ( + ValueData::I64Value(n.as_i64().unwrap()), + ColumnDataType::Int64 as i32, + SemanticType::Field as i32, + ) + } else if n.is_u64() { + ( + ValueData::U64Value(n.as_u64().unwrap()), + ColumnDataType::Uint64 as i32, + SemanticType::Field as i32, + ) + } else if n.is_f64() { + ( + ValueData::F64Value(n.as_f64().unwrap()), + ColumnDataType::Float64 as i32, + SemanticType::Field as i32, + ) + } else { + return UnsupportedNumberTypeSnafu { value: n }.fail(); + }; + resolve_schema( + index, + value, + ColumnSchema { + column_name, + datatype, + semantic_type, + datatype_extension: None, + options: None, + }, + row, + schema_info, + ) +} + +fn json_value_to_row( + schema_info: &mut SchemaInfo, + map: Map, +) -> Result { + let mut row: Vec = Vec::with_capacity(schema_info.schema.len()); + for _ in 0..schema_info.schema.len() { + row.push(GreptimeValue { value_data: None }); + } + for (column_name, value) in map { + if column_name == DEFAULT_GREPTIME_TIMESTAMP_COLUMN { + continue; + } + let index = schema_info.index.get(&column_name).copied(); + match value { + serde_json::Value::Null => { + // do nothing + } + serde_json::Value::String(s) => { + resolve_schema( + index, + ValueData::StringValue(s), + ColumnSchema { + column_name, + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Field as i32, + datatype_extension: None, + options: None, + }, + &mut row, + schema_info, + )?; + } + serde_json::Value::Bool(b) => { + resolve_schema( + index, + ValueData::BoolValue(b), + ColumnSchema { + column_name, + datatype: ColumnDataType::Boolean as i32, + semantic_type: SemanticType::Field as i32, + datatype_extension: None, + options: None, + }, + &mut row, + schema_info, + )?; + } + serde_json::Value::Number(n) => { + resolve_number_schema(n, column_name, index, &mut row, schema_info)?; + } + serde_json::Value::Array(_) | serde_json::Value::Object(_) => { + resolve_schema( + index, + ValueData::BinaryValue(jsonb::Value::from(value).to_vec()), + ColumnSchema { + column_name, + datatype: ColumnDataType::Binary as i32, + semantic_type: SemanticType::Field as i32, + datatype_extension: Some(ColumnDataTypeExtension { + type_ext: Some(TypeExt::JsonType(JsonTypeExtension::JsonBinary.into())), + }), + options: None, + }, + &mut row, + schema_info, + )?; + } + } + } + Ok(Row { values: row }) +} + +/// Identity pipeline for Greptime +/// This pipeline will convert the input JSON array to Greptime Rows +/// 1. The pipeline will add a default timestamp column to the schema +/// 2. The pipeline not resolve NULL value +/// 3. The pipeline assumes that the json format is fixed +/// 4. The pipeline will return an error if the same column datatype is mismatched +/// 5. The pipeline will analyze the schema of each json record and merge them to get the final schema. +pub fn identity_pipeline(array: Vec) -> Result { + let mut rows = Vec::with_capacity(array.len()); + + let mut schema = SchemaInfo::default(); + for value in array { + if let serde_json::Value::Object(map) = value { + let row = json_value_to_row(&mut schema, map)?; + rows.push(row); + } + } + let greptime_timestamp_schema = ColumnSchema { + column_name: DEFAULT_GREPTIME_TIMESTAMP_COLUMN.to_string(), + datatype: ColumnDataType::TimestampNanosecond as i32, + semantic_type: SemanticType::Timestamp as i32, + datatype_extension: None, + options: None, + }; + let ns = chrono::Utc::now().timestamp_nanos_opt().unwrap_or(0); + let ts = GreptimeValue { + value_data: Some(ValueData::TimestampNanosecondValue(ns)), + }; + let column_count = schema.schema.len(); + for row in rows.iter_mut() { + let diff = column_count - row.values.len(); + for _ in 0..diff { + row.values.push(GreptimeValue { value_data: None }); + } + row.values.push(ts.clone()); + } + schema.schema.push(greptime_timestamp_schema); + Ok(Rows { + schema: schema.schema, + rows, + }) +} + +#[cfg(test)] +mod tests { + use crate::identity_pipeline; + + #[test] + fn test_identify_pipeline() { + { + let array = vec![ + serde_json::json!({ + "woshinull": null, + "name": "Alice", + "age": 20, + "is_student": true, + "score": 99.5, + "hobbies": "reading", + "address": "Beijing", + }), + serde_json::json!({ + "name": "Bob", + "age": 21, + "is_student": false, + "score": "88.5", + "hobbies": "swimming", + "address": "Shanghai", + "gaga": "gaga" + }), + ]; + let rows = identity_pipeline(array); + assert!(rows.is_err()); + assert_eq!( + rows.err().unwrap().to_string(), + "Column datatype mismatch. For column: score, expected datatype: FLOAT64, actual datatype: STRING".to_string(), + ); + } + { + let array = vec![ + serde_json::json!({ + "woshinull": null, + "name": "Alice", + "age": 20, + "is_student": true, + "score": 99.5, + "hobbies": "reading", + "address": "Beijing", + }), + serde_json::json!({ + "name": "Bob", + "age": 21, + "is_student": false, + "score": 88, + "hobbies": "swimming", + "address": "Shanghai", + "gaga": "gaga" + }), + ]; + let rows = identity_pipeline(array); + assert!(rows.is_err()); + assert_eq!( + rows.err().unwrap().to_string(), + "Column datatype mismatch. For column: score, expected datatype: FLOAT64, actual datatype: INT64".to_string(), + ); + } + { + let array = vec![ + serde_json::json!({ + "woshinull": null, + "name": "Alice", + "age": 20, + "is_student": true, + "score": 99.5, + "hobbies": "reading", + "address": "Beijing", + }), + serde_json::json!({ + "name": "Bob", + "age": 21, + "is_student": false, + "score": 88.5, + "hobbies": "swimming", + "address": "Shanghai", + "gaga": "gaga" + }), + ]; + let rows = identity_pipeline(array); + assert!(rows.is_ok()); + let rows = rows.unwrap(); + assert_eq!(rows.schema.len(), 8); + assert_eq!(rows.rows.len(), 2); + assert_eq!(8, rows.rows[0].values.len()); + assert_eq!(8, rows.rows[1].values.len()); + } + } +} diff --git a/src/pipeline/src/lib.rs b/src/pipeline/src/lib.rs index 8fc72c5844..36ef3023f6 100644 --- a/src/pipeline/src/lib.rs +++ b/src/pipeline/src/lib.rs @@ -18,6 +18,7 @@ mod metrics; pub use etl::error::Result; pub use etl::processor::Processor; +pub use etl::transform::transformer::identity_pipeline; pub use etl::transform::{GreptimeTransformer, Transformer}; pub use etl::value::{Array, Map, Value}; pub use etl::{parse, Content, Pipeline}; diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 7877d2b84a..80f15ff211 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -50,6 +50,9 @@ use crate::metrics::{ }; use crate::query_handler::LogHandlerRef; +const GREPTIME_INTERNAL_PIPELINE_NAME_PREFIX: &str = "greptime_"; +const GREPTIME_INTERNAL_IDENTITY_PIPELINE_NAME: &str = "greptime_identity"; + #[derive(Debug, Default, Serialize, Deserialize, JsonSchema)] pub struct LogIngesterQueryParams { pub table: Option, @@ -121,6 +124,12 @@ pub async fn add_pipeline( reason: "pipeline_name is required in path", } ); + ensure!( + !pipeline_name.starts_with(GREPTIME_INTERNAL_PIPELINE_NAME_PREFIX), + InvalidParameterSnafu { + reason: "pipeline_name cannot start with greptime_", + } + ); ensure!( !payload.is_empty(), InvalidParameterSnafu { @@ -425,47 +434,54 @@ async fn ingest_logs_inner( let db = query_ctx.get_db_string(); let exec_timer = std::time::Instant::now(); - let pipeline = state - .get_pipeline(&pipeline_name, version, query_ctx.clone()) - .await?; - - let transform_timer = std::time::Instant::now(); - let mut intermediate_state = pipeline.init_intermediate_state(); - let mut results = Vec::with_capacity(pipeline_data.len()); + let transformed_data: Rows; + if pipeline_name == GREPTIME_INTERNAL_IDENTITY_PIPELINE_NAME { + let rows = pipeline::identity_pipeline(pipeline_data) + .context(PipelineTransformSnafu) + .context(PipelineSnafu)?; + transformed_data = rows; + } else { + let pipeline = state + .get_pipeline(&pipeline_name, version, query_ctx.clone()) + .await?; - for v in pipeline_data { - pipeline - .prepare(v, &mut intermediate_state) - .inspect_err(|_| { - METRIC_HTTP_LOGS_TRANSFORM_ELAPSED - .with_label_values(&[db.as_str(), METRIC_FAILURE_VALUE]) - .observe(transform_timer.elapsed().as_secs_f64()); - }) - .context(PipelineTransformSnafu) - .context(PipelineSnafu)?; - let r = pipeline - .exec_mut(&mut intermediate_state) - .inspect_err(|_| { - METRIC_HTTP_LOGS_TRANSFORM_ELAPSED - .with_label_values(&[db.as_str(), METRIC_FAILURE_VALUE]) - .observe(transform_timer.elapsed().as_secs_f64()); - }) - .context(PipelineTransformSnafu) - .context(PipelineSnafu)?; - results.push(r); - pipeline.reset_intermediate_state(&mut intermediate_state); + let transform_timer = std::time::Instant::now(); + let mut intermediate_state = pipeline.init_intermediate_state(); + + for v in pipeline_data { + pipeline + .prepare(v, &mut intermediate_state) + .inspect_err(|_| { + METRIC_HTTP_LOGS_TRANSFORM_ELAPSED + .with_label_values(&[db.as_str(), METRIC_FAILURE_VALUE]) + .observe(transform_timer.elapsed().as_secs_f64()); + }) + .context(PipelineTransformSnafu) + .context(PipelineSnafu)?; + let r = pipeline + .exec_mut(&mut intermediate_state) + .inspect_err(|_| { + METRIC_HTTP_LOGS_TRANSFORM_ELAPSED + .with_label_values(&[db.as_str(), METRIC_FAILURE_VALUE]) + .observe(transform_timer.elapsed().as_secs_f64()); + }) + .context(PipelineTransformSnafu) + .context(PipelineSnafu)?; + results.push(r); + pipeline.reset_intermediate_state(&mut intermediate_state); + } + + METRIC_HTTP_LOGS_TRANSFORM_ELAPSED + .with_label_values(&[db.as_str(), METRIC_SUCCESS_VALUE]) + .observe(transform_timer.elapsed().as_secs_f64()); + + transformed_data = Rows { + rows: results, + schema: pipeline.schemas().clone(), + }; } - METRIC_HTTP_LOGS_TRANSFORM_ELAPSED - .with_label_values(&[db.as_str(), METRIC_SUCCESS_VALUE]) - .observe(transform_timer.elapsed().as_secs_f64()); - - let transformed_data: Rows = Rows { - rows: results, - schema: pipeline.schemas().clone(), - }; - let insert_request = RowInsertRequest { rows: Some(transformed_data), table_name: table_name.clone(), diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 20a0f4edd2..f70df8176a 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -86,6 +86,7 @@ macro_rules! http_tests { test_pipeline_api, test_test_pipeline_api, test_plain_text_ingestion, + test_identify_pipeline, test_otlp_metrics, test_otlp_traces, @@ -1076,6 +1077,21 @@ transform: "#; // 1. create pipeline + let res = client + .post("/v1/events/pipelines/greptime_guagua") + .header("Content-Type", "application/x-yaml") + .body(body) + .send() + .await; + + assert_eq!(res.status(), StatusCode::BAD_REQUEST); + assert_eq!( + res.json::().await["error"] + .as_str() + .unwrap(), + "Invalid request parameter: pipeline_name cannot start with greptime_" + ); + let res = client .post("/v1/events/pipelines/test") .header("Content-Type", "application/x-yaml") @@ -1161,6 +1177,61 @@ transform: guard.remove_all().await; } +pub async fn test_identify_pipeline(store_type: StorageType) { + common_telemetry::init_default_ut_logging(); + let (app, mut guard) = setup_test_http_app_with_frontend(store_type, "test_pipeline_api").await; + + // handshake + let client = TestClient::new(app); + let body = r#"{"__time__":1453809242,"__topic__":"","__source__":"10.170.***.***","ip":"10.200.**.***","time":"26/Jan/2016:19:54:02 +0800","url":"POST/PutData?Category=YunOsAccountOpLog&AccessKeyId=&Date=Fri%2C%2028%20Jun%202013%2006%3A53%3A30%20GMT&Topic=raw&Signature=HTTP/1.1","status":"200","user-agent":"aliyun-sdk-java"} +{"__time__":1453809242,"__topic__":"","__source__":"10.170.***.***","ip":"10.200.**.***","time":"26/Jan/2016:19:54:02 +0800","url":"POST/PutData?Category=YunOsAccountOpLog&AccessKeyId=&Date=Fri%2C%2028%20Jun%202013%2006%3A53%3A30%20GMT&Topic=raw&Signature=HTTP/1.1","status":"200","user-agent":"aliyun-sdk-java","hasagei":"hasagei","dongdongdong":"guaguagua"}"#; + let res = client + .post("/v1/events/logs?db=public&table=logs&pipeline_name=greptime_identity") + .header("Content-Type", "application/json") + .body(body) + .send() + .await; + + assert_eq!(res.status(), StatusCode::OK); + + let body: serde_json::Value = res.json().await; + + assert!(body.get("execution_time_ms").unwrap().is_number()); + assert_eq!(body["output"][0]["affectedrows"], 2); + + let res = client.get("/v1/sql?sql=select * from logs").send().await; + + assert_eq!(res.status(), StatusCode::OK); + + let line1_expected = r#"["10.170.***.***",1453809242,"","10.200.**.***","200","26/Jan/2016:19:54:02 +0800","POST/PutData?Category=YunOsAccountOpLog&AccessKeyId=&Date=Fri%2C%2028%20Jun%202013%2006%3A53%3A30%20GMT&Topic=raw&Signature=HTTP/1.1","aliyun-sdk-java","guaguagua","hasagei",null]"#; + let line2_expected = r#"["10.170.***.***",1453809242,"","10.200.**.***","200","26/Jan/2016:19:54:02 +0800","POST/PutData?Category=YunOsAccountOpLog&AccessKeyId=&Date=Fri%2C%2028%20Jun%202013%2006%3A53%3A30%20GMT&Topic=raw&Signature=HTTP/1.1","aliyun-sdk-java",null,null,null]"#; + let res = client.get("/v1/sql?sql=select * from logs").send().await; + assert_eq!(res.status(), StatusCode::OK); + let resp: serde_json::Value = res.json().await; + let result = resp["output"][0]["records"]["rows"].as_array().unwrap(); + assert_eq!(result.len(), 2); + let mut line1 = result[0].as_array().unwrap().clone(); + let mut line2 = result[1].as_array().unwrap().clone(); + assert!(line1.last().unwrap().is_i64()); + assert!(line2.last().unwrap().is_i64()); + *line1.last_mut().unwrap() = serde_json::Value::Null; + *line2.last_mut().unwrap() = serde_json::Value::Null; + + assert_eq!( + line1, + serde_json::from_str::>(line1_expected).unwrap() + ); + assert_eq!( + line2, + serde_json::from_str::>(line2_expected).unwrap() + ); + + let expected = r#"[["__source__","String","","YES","","FIELD"],["__time__","Int64","","YES","","FIELD"],["__topic__","String","","YES","","FIELD"],["ip","String","","YES","","FIELD"],["status","String","","YES","","FIELD"],["time","String","","YES","","FIELD"],["url","String","","YES","","FIELD"],["user-agent","String","","YES","","FIELD"],["dongdongdong","String","","YES","","FIELD"],["hasagei","String","","YES","","FIELD"],["greptime_timestamp","TimestampNanosecond","PRI","NO","","TIMESTAMP"]]"#; + validate_data(&client, "desc logs", expected).await; + + guard.remove_all().await; +} + pub async fn test_test_pipeline_api(store_type: StorageType) { common_telemetry::init_default_ut_logging(); let (app, mut guard) = setup_test_http_app_with_frontend(store_type, "test_pipeline_api").await; From 5f0a83b2b1876e1e4b0ce93ecd4c8c5a65f91c03 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Wed, 9 Oct 2024 01:54:44 +0800 Subject: [PATCH 058/128] fix: ts conversion during transform phase (#4790) * fix: allow ts conversion during transform phase * chore: replace `unimplemented` with snafu --- src/pipeline/src/etl/error.rs | 11 ++++ .../transform/transformer/greptime/coerce.rs | 57 ++++++++++--------- src/pipeline/tests/epoch.rs | 24 ++++---- src/pipeline/tests/timestamp.rs | 23 ++++---- 4 files changed, 66 insertions(+), 49 deletions(-) diff --git a/src/pipeline/src/etl/error.rs b/src/pipeline/src/etl/error.rs index 0a28cb7423..08654c2c82 100644 --- a/src/pipeline/src/etl/error.rs +++ b/src/pipeline/src/etl/error.rs @@ -438,6 +438,17 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + #[snafu(display("failed to coerce complex value, not supported"))] + CoerceComplexType { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("failed to coerce value: {msg}"))] + CoerceIncompatibleTypes { + msg: String, + #[snafu(implicit)] + location: Location, + }, #[snafu(display( "Invalid resolution: '{resolution}'. Available resolutions: {valid_resolution}" diff --git a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs index 827613b02b..8d07b34d46 100644 --- a/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs +++ b/src/pipeline/src/etl/transform/transformer/greptime/coerce.rs @@ -20,7 +20,8 @@ use greptime_proto::v1::{ColumnDataType, ColumnSchema, SemanticType}; use snafu::ResultExt; use crate::etl::error::{ - CoerceStringToTypeSnafu, CoerceUnsupportedEpochTypeSnafu, CoerceUnsupportedNullTypeSnafu, + CoerceComplexTypeSnafu, CoerceIncompatibleTypesSnafu, CoerceStringToTypeSnafu, + CoerceUnsupportedEpochTypeSnafu, CoerceUnsupportedNullTypeSnafu, CoerceUnsupportedNullTypeToSnafu, ColumnOptionsSnafu, Error, Result, }; use crate::etl::transform::index::Index; @@ -61,8 +62,7 @@ impl TryFrom for ValueData { } Value::Timestamp(Timestamp::Second(s)) => Ok(ValueData::TimestampSecondValue(s)), - Value::Array(_) => unimplemented!("Array type not supported"), - Value::Map(_) => unimplemented!("Object type not supported"), + Value::Array(_) | Value::Map(_) => CoerceComplexTypeSnafu.fail(), } } } @@ -134,8 +134,7 @@ fn coerce_type(transform: &Transform) -> Result { Value::Timestamp(Timestamp::Millisecond(_)) => Ok(ColumnDataType::TimestampMillisecond), Value::Timestamp(Timestamp::Second(_)) => Ok(ColumnDataType::TimestampSecond), - Value::Array(_) => unimplemented!("Array"), - Value::Map(_) => unimplemented!("Object"), + Value::Array(_) | Value::Map(_) => CoerceComplexTypeSnafu.fail(), Value::Null => CoerceUnsupportedNullTypeToSnafu { ty: transform.type_.to_str_type(), @@ -176,19 +175,28 @@ pub(crate) fn coerce_value(val: &Value, transform: &Transform) -> Result coerce_bool_value(*b, transform), Value::String(s) => coerce_string_value(s, transform), - Value::Timestamp(Timestamp::Nanosecond(ns)) => { - Ok(Some(ValueData::TimestampNanosecondValue(*ns))) - } - Value::Timestamp(Timestamp::Microsecond(us)) => { - Ok(Some(ValueData::TimestampMicrosecondValue(*us))) - } - Value::Timestamp(Timestamp::Millisecond(ms)) => { - Ok(Some(ValueData::TimestampMillisecondValue(*ms))) - } - Value::Timestamp(Timestamp::Second(s)) => Ok(Some(ValueData::TimestampSecondValue(*s))), + Value::Timestamp(input_timestamp) => match &transform.type_ { + Value::Timestamp(target_timestamp) => match target_timestamp { + Timestamp::Nanosecond(_) => Ok(Some(ValueData::TimestampNanosecondValue( + input_timestamp.timestamp_nanos(), + ))), + Timestamp::Microsecond(_) => Ok(Some(ValueData::TimestampMicrosecondValue( + input_timestamp.timestamp_micros(), + ))), + Timestamp::Millisecond(_) => Ok(Some(ValueData::TimestampMillisecondValue( + input_timestamp.timestamp_millis(), + ))), + Timestamp::Second(_) => Ok(Some(ValueData::TimestampSecondValue( + input_timestamp.timestamp(), + ))), + }, + _ => CoerceIncompatibleTypesSnafu { + msg: "Timestamp can only be coerced to another timestamp", + } + .fail(), + }, - Value::Array(_) => unimplemented!("Array type not supported"), - Value::Map(_) => unimplemented!("Object type not supported"), + Value::Array(_) | Value::Map(_) => CoerceComplexTypeSnafu.fail(), } } @@ -220,8 +228,7 @@ fn coerce_bool_value(b: bool, transform: &Transform) -> Result } }, - Value::Array(_) => unimplemented!("Array type not supported"), - Value::Map(_) => unimplemented!("Object type not supported"), + Value::Array(_) | Value::Map(_) => return CoerceComplexTypeSnafu.fail(), Value::Null => return Ok(None), }; @@ -257,8 +264,7 @@ fn coerce_i64_value(n: i64, transform: &Transform) -> Result> } }, - Value::Array(_) => unimplemented!("Array type not supported"), - Value::Map(_) => unimplemented!("Object type not supported"), + Value::Array(_) | Value::Map(_) => return CoerceComplexTypeSnafu.fail(), Value::Null => return Ok(None), }; @@ -294,8 +300,7 @@ fn coerce_u64_value(n: u64, transform: &Transform) -> Result> } }, - Value::Array(_) => unimplemented!("Array type not supported"), - Value::Map(_) => unimplemented!("Object type not supported"), + Value::Array(_) | Value::Map(_) => return CoerceComplexTypeSnafu.fail(), Value::Null => return Ok(None), }; @@ -331,8 +336,7 @@ fn coerce_f64_value(n: f64, transform: &Transform) -> Result> } }, - Value::Array(_) => unimplemented!("Array type not supported"), - Value::Map(_) => unimplemented!("Object type not supported"), + Value::Array(_) | Value::Map(_) => return CoerceComplexTypeSnafu.fail(), Value::Null => return Ok(None), }; @@ -407,8 +411,7 @@ fn coerce_string_value(s: &String, transform: &Transform) -> Result CoerceUnsupportedEpochTypeSnafu { ty: "String" }.fail(), }, - Value::Array(_) => unimplemented!("Array type not supported"), - Value::Map(_) => unimplemented!("Object type not supported"), + Value::Array(_) | Value::Map(_) => CoerceComplexTypeSnafu.fail(), Value::Null => Ok(None), } diff --git a/src/pipeline/tests/epoch.rs b/src/pipeline/tests/epoch.rs index 35a2ab635c..84662793b9 100644 --- a/src/pipeline/tests/epoch.rs +++ b/src/pipeline/tests/epoch.rs @@ -200,6 +200,8 @@ transform: #[test] fn test_default_wrong_resolution() { + // given a number, we have no ways to guess its resolution + // but we can convert resolution during transform phase let test_input = r#" { "input_s": "1722580862", @@ -209,28 +211,30 @@ fn test_default_wrong_resolution() { let pipeline_yaml = r#" processors: - epoch: - fields: - - input_s - - input_nano + field: input_s + resolution: s + - epoch: + field: input_nano + resolution: ns transform: - fields: - input_s - type: epoch, s + type: epoch, ms - fields: - input_nano - type: epoch, nano + type: epoch, ms "#; let expected_schema = vec![ common::make_column_schema( "input_s".to_string(), - ColumnDataType::TimestampSecond, + ColumnDataType::TimestampMillisecond, SemanticType::Field, ), common::make_column_schema( "input_nano".to_string(), - ColumnDataType::TimestampNanosecond, + ColumnDataType::TimestampMillisecond, SemanticType::Field, ), common::make_column_schema( @@ -242,14 +246,12 @@ transform: let output = common::parse_and_exec(test_input, pipeline_yaml); assert_eq!(output.schema, expected_schema); - // this is actually wrong - // TODO(shuiyisong): add check for type when converting epoch assert_eq!( output.rows[0].values[0].value_data, - Some(ValueData::TimestampMillisecondValue(1722580862)) + Some(ValueData::TimestampMillisecondValue(1722580862000)) ); assert_eq!( output.rows[0].values[1].value_data, - Some(ValueData::TimestampMillisecondValue(1722583122284583936)) + Some(ValueData::TimestampMillisecondValue(1722583122284)) ); } diff --git a/src/pipeline/tests/timestamp.rs b/src/pipeline/tests/timestamp.rs index cba68d6cb1..85cf0643fc 100644 --- a/src/pipeline/tests/timestamp.rs +++ b/src/pipeline/tests/timestamp.rs @@ -318,6 +318,7 @@ transform: #[test] fn test_timestamp_default_wrong_resolution() { + // same as test_default_wrong_resolution from epoch tests let test_input = r#" { "input_s": "1722580862", @@ -327,28 +328,30 @@ fn test_timestamp_default_wrong_resolution() { let pipeline_yaml = r#" processors: - timestamp: - fields: - - input_s - - input_nano + field: input_s + resolution: s + - timestamp: + field: input_nano + resolution: ns transform: - fields: - input_s - type: timestamp, s + type: timestamp, ms - fields: - input_nano - type: timestamp, nano + type: timestamp, ms "#; let expected_schema = vec![ common::make_column_schema( "input_s".to_string(), - ColumnDataType::TimestampSecond, + ColumnDataType::TimestampMillisecond, SemanticType::Field, ), common::make_column_schema( "input_nano".to_string(), - ColumnDataType::TimestampNanosecond, + ColumnDataType::TimestampMillisecond, SemanticType::Field, ), common::make_column_schema( @@ -360,14 +363,12 @@ transform: let output = common::parse_and_exec(test_input, pipeline_yaml); assert_eq!(output.schema, expected_schema); - // this is actually wrong - // TODO(shuiyisong): add check for type when converting epoch assert_eq!( output.rows[0].values[0].value_data, - Some(ValueData::TimestampMillisecondValue(1722580862)) + Some(ValueData::TimestampMillisecondValue(1722580862000)) ); assert_eq!( output.rows[0].values[1].value_data, - Some(ValueData::TimestampMillisecondValue(1722583122284583936)) + Some(ValueData::TimestampMillisecondValue(1722583122284)) ); } From 19918928c59e43bd9d8b5f0bc58ffbf6702ed232 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Tue, 8 Oct 2024 19:38:44 -0700 Subject: [PATCH 059/128] feat: add function to aggregate path into a geojson path (#4798) * feat: add geojson function to aggregate paths * test: add sqlness results * test: add sqlness * refactor: corrected to aggregation function * chore: update comments * fix: make linter happy again * refactor: rename to remove `geo` from `geojson` function name The return type is not geojson at all. It's just compatible with geojson's coordinates part and superset's deckgl path plugin. --- src/common/function/src/scalars/aggregate.rs | 3 + .../function/src/scalars/aggregate/argmax.rs | 5 +- .../function/src/scalars/aggregate/argmin.rs | 5 +- .../function/src/scalars/aggregate/diff.rs | 4 +- .../function/src/scalars/aggregate/mean.rs | 4 +- .../function/src/scalars/aggregate/polyval.rs | 3 +- .../scalars/aggregate/scipy_stats_norm_cdf.rs | 4 +- .../scalars/aggregate/scipy_stats_norm_pdf.rs | 4 +- src/common/function/src/scalars/geo.rs | 2 + .../function/src/scalars/geo/encoding.rs | 223 ++++++++++++++++++ .../function/src/scalars/geo/helpers.rs | 61 +++++ src/common/macro/src/aggr_func.rs | 18 +- src/common/macro/tests/test_derive.rs | 2 +- src/datatypes/src/data_type.rs | 9 + src/query/src/tests/my_sum_udaf_example.rs | 1 + .../standalone/common/function/geo.result | 25 ++ .../cases/standalone/common/function/geo.sql | 13 + 17 files changed, 367 insertions(+), 19 deletions(-) create mode 100644 src/common/function/src/scalars/geo/encoding.rs create mode 100644 src/common/function/src/scalars/geo/helpers.rs diff --git a/src/common/function/src/scalars/aggregate.rs b/src/common/function/src/scalars/aggregate.rs index c6875a680c..f7ce316b6c 100644 --- a/src/common/function/src/scalars/aggregate.rs +++ b/src/common/function/src/scalars/aggregate.rs @@ -31,6 +31,7 @@ pub use polyval::PolyvalAccumulatorCreator; pub use scipy_stats_norm_cdf::ScipyStatsNormCdfAccumulatorCreator; pub use scipy_stats_norm_pdf::ScipyStatsNormPdfAccumulatorCreator; +use super::geo::encoding::JsonPathEncodeFunctionCreator; use crate::function_registry::FunctionRegistry; /// A function creates `AggregateFunctionCreator`. @@ -91,5 +92,7 @@ impl AggregateFunctions { register_aggr_func!("argmin", 1, ArgminAccumulatorCreator); register_aggr_func!("scipystatsnormcdf", 2, ScipyStatsNormCdfAccumulatorCreator); register_aggr_func!("scipystatsnormpdf", 2, ScipyStatsNormPdfAccumulatorCreator); + + register_aggr_func!("json_encode_path", 3, JsonPathEncodeFunctionCreator); } } diff --git a/src/common/function/src/scalars/aggregate/argmax.rs b/src/common/function/src/scalars/aggregate/argmax.rs index c5c5264f19..4749ff9a3a 100644 --- a/src/common/function/src/scalars/aggregate/argmax.rs +++ b/src/common/function/src/scalars/aggregate/argmax.rs @@ -16,7 +16,10 @@ use std::cmp::Ordering; use std::sync::Arc; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; -use common_query::error::{BadAccumulatorImplSnafu, CreateAccumulatorSnafu, Result}; +use common_query::error::{ + BadAccumulatorImplSnafu, CreateAccumulatorSnafu, InvalidInputStateSnafu, Result, +}; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use datatypes::prelude::*; diff --git a/src/common/function/src/scalars/aggregate/argmin.rs b/src/common/function/src/scalars/aggregate/argmin.rs index 7233f43b77..fe89184460 100644 --- a/src/common/function/src/scalars/aggregate/argmin.rs +++ b/src/common/function/src/scalars/aggregate/argmin.rs @@ -16,7 +16,10 @@ use std::cmp::Ordering; use std::sync::Arc; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; -use common_query::error::{BadAccumulatorImplSnafu, CreateAccumulatorSnafu, Result}; +use common_query::error::{ + BadAccumulatorImplSnafu, CreateAccumulatorSnafu, InvalidInputStateSnafu, Result, +}; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use datatypes::prelude::*; diff --git a/src/common/function/src/scalars/aggregate/diff.rs b/src/common/function/src/scalars/aggregate/diff.rs index b83ed6d004..25d1614e4b 100644 --- a/src/common/function/src/scalars/aggregate/diff.rs +++ b/src/common/function/src/scalars/aggregate/diff.rs @@ -17,8 +17,10 @@ use std::sync::Arc; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; use common_query::error::{ - CreateAccumulatorSnafu, DowncastVectorSnafu, FromScalarValueSnafu, Result, + CreateAccumulatorSnafu, DowncastVectorSnafu, FromScalarValueSnafu, InvalidInputStateSnafu, + Result, }; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use datatypes::prelude::*; diff --git a/src/common/function/src/scalars/aggregate/mean.rs b/src/common/function/src/scalars/aggregate/mean.rs index 3dc3e18535..ed66c90bdb 100644 --- a/src/common/function/src/scalars/aggregate/mean.rs +++ b/src/common/function/src/scalars/aggregate/mean.rs @@ -17,8 +17,10 @@ use std::sync::Arc; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; use common_query::error::{ - BadAccumulatorImplSnafu, CreateAccumulatorSnafu, DowncastVectorSnafu, Result, + BadAccumulatorImplSnafu, CreateAccumulatorSnafu, DowncastVectorSnafu, InvalidInputStateSnafu, + Result, }; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use datatypes::prelude::*; diff --git a/src/common/function/src/scalars/aggregate/polyval.rs b/src/common/function/src/scalars/aggregate/polyval.rs index ae6ca101c4..bc3986fd0e 100644 --- a/src/common/function/src/scalars/aggregate/polyval.rs +++ b/src/common/function/src/scalars/aggregate/polyval.rs @@ -18,8 +18,9 @@ use std::sync::Arc; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; use common_query::error::{ self, BadAccumulatorImplSnafu, CreateAccumulatorSnafu, DowncastVectorSnafu, - FromScalarValueSnafu, InvalidInputColSnafu, Result, + FromScalarValueSnafu, InvalidInputColSnafu, InvalidInputStateSnafu, Result, }; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use datatypes::prelude::*; diff --git a/src/common/function/src/scalars/aggregate/scipy_stats_norm_cdf.rs b/src/common/function/src/scalars/aggregate/scipy_stats_norm_cdf.rs index e6c92225a6..09a9c820d8 100644 --- a/src/common/function/src/scalars/aggregate/scipy_stats_norm_cdf.rs +++ b/src/common/function/src/scalars/aggregate/scipy_stats_norm_cdf.rs @@ -17,8 +17,10 @@ use std::sync::Arc; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; use common_query::error::{ self, BadAccumulatorImplSnafu, CreateAccumulatorSnafu, DowncastVectorSnafu, - FromScalarValueSnafu, GenerateFunctionSnafu, InvalidInputColSnafu, Result, + FromScalarValueSnafu, GenerateFunctionSnafu, InvalidInputColSnafu, InvalidInputStateSnafu, + Result, }; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use datatypes::prelude::*; diff --git a/src/common/function/src/scalars/aggregate/scipy_stats_norm_pdf.rs b/src/common/function/src/scalars/aggregate/scipy_stats_norm_pdf.rs index 3045ae8665..2d5025ea3a 100644 --- a/src/common/function/src/scalars/aggregate/scipy_stats_norm_pdf.rs +++ b/src/common/function/src/scalars/aggregate/scipy_stats_norm_pdf.rs @@ -17,8 +17,10 @@ use std::sync::Arc; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; use common_query::error::{ self, BadAccumulatorImplSnafu, CreateAccumulatorSnafu, DowncastVectorSnafu, - FromScalarValueSnafu, GenerateFunctionSnafu, InvalidInputColSnafu, Result, + FromScalarValueSnafu, GenerateFunctionSnafu, InvalidInputColSnafu, InvalidInputStateSnafu, + Result, }; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use datatypes::prelude::*; diff --git a/src/common/function/src/scalars/geo.rs b/src/common/function/src/scalars/geo.rs index 8ad6a7aef2..37b6c0704b 100644 --- a/src/common/function/src/scalars/geo.rs +++ b/src/common/function/src/scalars/geo.rs @@ -13,8 +13,10 @@ // limitations under the License. use std::sync::Arc; +pub(crate) mod encoding; mod geohash; mod h3; +mod helpers; use geohash::{GeohashFunction, GeohashNeighboursFunction}; diff --git a/src/common/function/src/scalars/geo/encoding.rs b/src/common/function/src/scalars/geo/encoding.rs new file mode 100644 index 0000000000..0600120598 --- /dev/null +++ b/src/common/function/src/scalars/geo/encoding.rs @@ -0,0 +1,223 @@ +// 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::sync::Arc; + +use common_error::ext::{BoxedError, PlainError}; +use common_error::status_code::StatusCode; +use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; +use common_query::error::{self, InvalidFuncArgsSnafu, InvalidInputStateSnafu, Result}; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; +use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; +use common_query::prelude::AccumulatorCreatorFunction; +use common_time::Timestamp; +use datatypes::prelude::ConcreteDataType; +use datatypes::value::{ListValue, Value}; +use datatypes::vectors::VectorRef; +use snafu::{ensure, ResultExt}; + +use super::helpers::{ensure_columns_len, ensure_columns_n}; + +/// Accumulator of lat, lng, timestamp tuples +#[derive(Debug)] +pub struct JsonPathAccumulator { + timestamp_type: ConcreteDataType, + lat: Vec>, + lng: Vec>, + timestamp: Vec>, +} + +impl JsonPathAccumulator { + fn new(timestamp_type: ConcreteDataType) -> Self { + Self { + lat: Vec::default(), + lng: Vec::default(), + timestamp: Vec::default(), + timestamp_type, + } + } +} + +impl Accumulator for JsonPathAccumulator { + fn state(&self) -> Result> { + Ok(vec![ + Value::List(ListValue::new( + self.lat.iter().map(|i| Value::from(*i)).collect(), + ConcreteDataType::float64_datatype(), + )), + Value::List(ListValue::new( + self.lng.iter().map(|i| Value::from(*i)).collect(), + ConcreteDataType::float64_datatype(), + )), + Value::List(ListValue::new( + self.timestamp.iter().map(|i| Value::from(*i)).collect(), + self.timestamp_type.clone(), + )), + ]) + } + + fn update_batch(&mut self, columns: &[VectorRef]) -> Result<()> { + // update batch as in datafusion just provides the accumulator original + // input. + // + // columns is vec of [`lat`, `lng`, `timestamp`] + // where + // - `lat` is a vector of `Value::Float64` or similar type. Each item in + // the vector is a row in given dataset. + // - so on so forth for `lng` and `timestamp` + ensure_columns_n!(columns, 3); + + let lat = &columns[0]; + let lng = &columns[1]; + let ts = &columns[2]; + + let size = lat.len(); + + for idx in 0..size { + self.lat.push(lat.get(idx).as_f64_lossy()); + self.lng.push(lng.get(idx).as_f64_lossy()); + self.timestamp.push(ts.get(idx).as_timestamp()); + } + + Ok(()) + } + + fn merge_batch(&mut self, states: &[VectorRef]) -> Result<()> { + // merge batch as in datafusion gives state accumulated from the data + // returned from child accumulators' state() call + // In our particular implementation, the data structure is like + // + // states is vec of [`lat`, `lng`, `timestamp`] + // where + // - `lat` is a vector of `Value::List`. Each item in the list is all + // coordinates from a child accumulator. + // - so on so forth for `lng` and `timestamp` + + ensure_columns_n!(states, 3); + + let lat_lists = &states[0]; + let lng_lists = &states[1]; + let ts_lists = &states[2]; + + let len = lat_lists.len(); + + for idx in 0..len { + if let Some(lat_list) = lat_lists + .get(idx) + .as_list() + .map_err(BoxedError::new) + .context(error::ExecuteSnafu)? + { + for v in lat_list.items() { + self.lat.push(v.as_f64_lossy()); + } + } + + if let Some(lng_list) = lng_lists + .get(idx) + .as_list() + .map_err(BoxedError::new) + .context(error::ExecuteSnafu)? + { + for v in lng_list.items() { + self.lng.push(v.as_f64_lossy()); + } + } + + if let Some(ts_list) = ts_lists + .get(idx) + .as_list() + .map_err(BoxedError::new) + .context(error::ExecuteSnafu)? + { + for v in ts_list.items() { + self.timestamp.push(v.as_timestamp()); + } + } + } + + Ok(()) + } + + fn evaluate(&self) -> Result { + let mut work_vec: Vec<(&Option, &Option, &Option)> = self + .lat + .iter() + .zip(self.lng.iter()) + .zip(self.timestamp.iter()) + .map(|((a, b), c)| (a, b, c)) + .collect(); + + // sort by timestamp, we treat null timestamp as 0 + work_vec.sort_unstable_by_key(|tuple| tuple.2.unwrap_or_else(|| Timestamp::new_second(0))); + + let result = serde_json::to_string( + &work_vec + .into_iter() + // note that we transform to lng,lat for geojson compatibility + .map(|(lat, lng, _)| vec![lng, lat]) + .collect::>>>(), + ) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("Serialization failure: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?; + + Ok(Value::String(result.into())) + } +} + +/// This function accept rows of lat, lng and timestamp, sort with timestamp and +/// encoding them into a geojson-like path. +/// +/// Example: +/// +/// ```sql +/// SELECT json_encode_path(lat, lon, timestamp) FROM table [group by ...]; +/// ``` +/// +#[as_aggr_func_creator] +#[derive(Debug, Default, AggrFuncTypeStore)] +pub struct JsonPathEncodeFunctionCreator {} + +impl AggregateFunctionCreator for JsonPathEncodeFunctionCreator { + fn creator(&self) -> AccumulatorCreatorFunction { + let creator: AccumulatorCreatorFunction = Arc::new(move |types: &[ConcreteDataType]| { + let ts_type = types[2].clone(); + Ok(Box::new(JsonPathAccumulator::new(ts_type))) + }); + + creator + } + + fn output_type(&self) -> Result { + Ok(ConcreteDataType::string_datatype()) + } + + fn state_types(&self) -> Result> { + let input_types = self.input_types()?; + ensure!(input_types.len() == 3, InvalidInputStateSnafu); + + let timestamp_type = input_types[2].clone(); + + Ok(vec![ + ConcreteDataType::list_datatype(ConcreteDataType::float64_datatype()), + ConcreteDataType::list_datatype(ConcreteDataType::float64_datatype()), + ConcreteDataType::list_datatype(timestamp_type), + ]) + } +} diff --git a/src/common/function/src/scalars/geo/helpers.rs b/src/common/function/src/scalars/geo/helpers.rs new file mode 100644 index 0000000000..f07eaefb15 --- /dev/null +++ b/src/common/function/src/scalars/geo/helpers.rs @@ -0,0 +1,61 @@ +// 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. + +macro_rules! ensure_columns_len { + ($columns:ident) => { + ensure!( + $columns.windows(2).all(|c| c[0].len() == c[1].len()), + InvalidFuncArgsSnafu { + err_msg: "The length of input columns are in different size" + } + ) + }; + ($column_a:ident, $column_b:ident, $($column_n:ident),*) => { + ensure!( + { + let mut result = $column_a.len() == $column_b.len(); + $( + result = result && ($column_a.len() == $column_n.len()); + )* + result + } + InvalidFuncArgsSnafu { + err_msg: "The length of input columns are in different size" + } + ) + }; +} + +pub(super) use ensure_columns_len; + +macro_rules! ensure_columns_n { + ($columns:ident, $n:literal) => { + ensure!( + $columns.len() == $n, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of arguments is not correct, expect {}, provided : {}", + stringify!($n), + $columns.len() + ), + } + ); + + if $n > 1 { + ensure_columns_len!($columns); + } + }; +} + +pub(super) use ensure_columns_n; diff --git a/src/common/macro/src/aggr_func.rs b/src/common/macro/src/aggr_func.rs index 4c3ccccdee..2e17e70b5a 100644 --- a/src/common/macro/src/aggr_func.rs +++ b/src/common/macro/src/aggr_func.rs @@ -21,23 +21,19 @@ use syn::{parse_macro_input, DeriveInput, ItemStruct}; pub(crate) fn impl_aggr_func_type_store(ast: &DeriveInput) -> TokenStream { let name = &ast.ident; let gen = quote! { - use common_query::logical_plan::accumulator::AggrFuncTypeStore; - use common_query::error::{InvalidInputStateSnafu, Error as QueryError}; - use datatypes::prelude::ConcreteDataType; - - impl AggrFuncTypeStore for #name { - fn input_types(&self) -> std::result::Result, QueryError> { + impl common_query::logical_plan::accumulator::AggrFuncTypeStore for #name { + fn input_types(&self) -> std::result::Result, common_query::error::Error> { let input_types = self.input_types.load(); - snafu::ensure!(input_types.is_some(), InvalidInputStateSnafu); + snafu::ensure!(input_types.is_some(), common_query::error::InvalidInputStateSnafu); Ok(input_types.as_ref().unwrap().as_ref().clone()) } - fn set_input_types(&self, input_types: Vec) -> std::result::Result<(), QueryError> { + fn set_input_types(&self, input_types: Vec) -> std::result::Result<(), common_query::error::Error> { let old = self.input_types.swap(Some(std::sync::Arc::new(input_types.clone()))); if let Some(old) = old { - snafu::ensure!(old.len() == input_types.len(), InvalidInputStateSnafu); + snafu::ensure!(old.len() == input_types.len(), common_query::error::InvalidInputStateSnafu); for (x, y) in old.iter().zip(input_types.iter()) { - snafu::ensure!(x == y, InvalidInputStateSnafu); + snafu::ensure!(x == y, common_query::error::InvalidInputStateSnafu); } } Ok(()) @@ -51,7 +47,7 @@ pub(crate) fn impl_as_aggr_func_creator(_args: TokenStream, input: TokenStream) let mut item_struct = parse_macro_input!(input as ItemStruct); if let syn::Fields::Named(ref mut fields) = item_struct.fields { let result = syn::Field::parse_named.parse2(quote! { - input_types: arc_swap::ArcSwapOption> + input_types: arc_swap::ArcSwapOption> }); match result { Ok(field) => fields.named.push(field), diff --git a/src/common/macro/tests/test_derive.rs b/src/common/macro/tests/test_derive.rs index 9c648c788d..c614800572 100644 --- a/src/common/macro/tests/test_derive.rs +++ b/src/common/macro/tests/test_derive.rs @@ -24,5 +24,5 @@ struct Foo {} fn test_derive() { let _ = Foo::default(); assert_fields!(Foo: input_types); - assert_impl_all!(Foo: std::fmt::Debug, Default, AggrFuncTypeStore); + assert_impl_all!(Foo: std::fmt::Debug, Default, common_query::logical_plan::accumulator::AggrFuncTypeStore); } diff --git a/src/datatypes/src/data_type.rs b/src/datatypes/src/data_type.rs index 495c47dc5f..12f5c8d1f2 100644 --- a/src/datatypes/src/data_type.rs +++ b/src/datatypes/src/data_type.rs @@ -249,6 +249,15 @@ impl ConcreteDataType { ] } + pub fn timestamps() -> Vec { + vec![ + ConcreteDataType::timestamp_second_datatype(), + ConcreteDataType::timestamp_millisecond_datatype(), + ConcreteDataType::timestamp_microsecond_datatype(), + ConcreteDataType::timestamp_nanosecond_datatype(), + ] + } + /// Convert arrow data type to [ConcreteDataType]. /// /// # Panics diff --git a/src/query/src/tests/my_sum_udaf_example.rs b/src/query/src/tests/my_sum_udaf_example.rs index 8220dcf72d..63653ffbce 100644 --- a/src/query/src/tests/my_sum_udaf_example.rs +++ b/src/query/src/tests/my_sum_udaf_example.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use common_function::scalars::aggregate::AggregateFunctionMeta; use common_macro::{as_aggr_func_creator, AggrFuncTypeStore}; use common_query::error::{CreateAccumulatorSnafu, Result as QueryResult}; +use common_query::logical_plan::accumulator::AggrFuncTypeStore; use common_query::logical_plan::{Accumulator, AggregateFunctionCreator}; use common_query::prelude::*; use common_recordbatch::{RecordBatch, RecordBatches}; diff --git a/tests/cases/standalone/common/function/geo.result b/tests/cases/standalone/common/function/geo.result index 8954447650..4f9d168ac0 100644 --- a/tests/cases/standalone/common/function/geo.result +++ b/tests/cases/standalone/common/function/geo.result @@ -236,3 +236,28 @@ SELECT geohash_neighbours(37.76938, -122.3889, 11); | [9q8yygxnefv, 9q8yygxnefu, 9q8yygxnefs, 9q8yygxnefk, 9q8yygxnefm, 9q8yygxnefq, 9q8yygxnefw, 9q8yygxnefy] | +----------------------------------------------------------------------------------------------------------+ +SELECT json_encode_path(37.76938, -122.3889, 1728083375::TimestampSecond); + ++----------------------------------------------------------------------------------------------------------------------+ +| json_encode_path(Float64(37.76938),Float64(-122.3889),arrow_cast(Int64(1728083375),Utf8("Timestamp(Second, None)"))) | ++----------------------------------------------------------------------------------------------------------------------+ +| [[-122.3889,37.76938]] | ++----------------------------------------------------------------------------------------------------------------------+ + +SELECT json_encode_path(lat, lon, ts) +FROM( + SELECT 37.76938 AS lat, -122.3889 AS lon, 1728083375::TimestampSecond AS ts + UNION ALL + SELECT 37.76928 AS lat, -122.3839 AS lon, 1728083373::TimestampSecond AS ts + UNION ALL + SELECT 37.76930 AS lat, -122.3820 AS lon, 1728083379::TimestampSecond AS ts + UNION ALL + SELECT 37.77001 AS lat, -122.3888 AS lon, 1728083372::TimestampSecond AS ts +); + ++-------------------------------------------------------------------------------------+ +| json_encode_path(lat,lon,ts) | ++-------------------------------------------------------------------------------------+ +| [[-122.3888,37.77001],[-122.3839,37.76928],[-122.3889,37.76938],[-122.382,37.7693]] | ++-------------------------------------------------------------------------------------+ + diff --git a/tests/cases/standalone/common/function/geo.sql b/tests/cases/standalone/common/function/geo.sql index be2b3947bb..cd9d403e6e 100644 --- a/tests/cases/standalone/common/function/geo.sql +++ b/tests/cases/standalone/common/function/geo.sql @@ -66,3 +66,16 @@ SELECT geohash(37.76938, -122.3889, 11::UInt32); SELECT geohash(37.76938, -122.3889, 11::UInt64); SELECT geohash_neighbours(37.76938, -122.3889, 11); + +SELECT json_encode_path(37.76938, -122.3889, 1728083375::TimestampSecond); + +SELECT json_encode_path(lat, lon, ts) +FROM( + SELECT 37.76938 AS lat, -122.3889 AS lon, 1728083375::TimestampSecond AS ts + UNION ALL + SELECT 37.76928 AS lat, -122.3839 AS lon, 1728083373::TimestampSecond AS ts + UNION ALL + SELECT 37.76930 AS lat, -122.3820 AS lon, 1728083379::TimestampSecond AS ts + UNION ALL + SELECT 37.77001 AS lat, -122.3888 AS lon, 1728083372::TimestampSecond AS ts +); From 0dd11f53f55ff1b342fda40d20a5125f568b244a Mon Sep 17 00:00:00 2001 From: Kaifeng Zheng <100595273+Kev1n8@users.noreply.github.com> Date: Wed, 9 Oct 2024 15:11:57 +0800 Subject: [PATCH 060/128] feat: add json format output for http interface (#4797) * feat: json output format for http * feat: add json result test case * fix: typo and refactor a piece of code * fix: cargo check * move affected_rows to top level --- src/servers/src/http.rs | 32 +++++++ src/servers/src/http/handler.rs | 2 + src/servers/src/http/json_result.rs | 137 ++++++++++++++++++++++++++++ tests-integration/tests/http.rs | 18 +++- 4 files changed, 188 insertions(+), 1 deletion(-) create mode 100644 src/servers/src/http/json_result.rs diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index a2b72b548b..953ff9e73a 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -59,6 +59,7 @@ use crate::http::error_result::ErrorResponse; use crate::http::greptime_result_v1::GreptimedbV1Response; use crate::http::influxdb::{influxdb_health, influxdb_ping, influxdb_write_v1, influxdb_write_v2}; use crate::http::influxdb_result_v1::InfluxdbV1Response; +use crate::http::json_result::JsonResponse; use crate::http::prometheus::{ build_info_query, format_query, instant_query, label_values_query, labels_query, range_query, series_query, @@ -97,6 +98,7 @@ pub mod error_result; pub mod greptime_manage_resp; pub mod greptime_result_v1; pub mod influxdb_result_v1; +pub mod json_result; pub mod table_result; #[cfg(any(test, feature = "testing"))] @@ -279,6 +281,7 @@ pub enum ResponseFormat { #[default] GreptimedbV1, InfluxdbV1, + Json, } impl ResponseFormat { @@ -289,6 +292,7 @@ impl ResponseFormat { "table" => Some(ResponseFormat::Table), "greptimedb_v1" => Some(ResponseFormat::GreptimedbV1), "influxdb_v1" => Some(ResponseFormat::InfluxdbV1), + "json" => Some(ResponseFormat::Json), _ => None, } } @@ -300,6 +304,7 @@ impl ResponseFormat { ResponseFormat::Table => "table", ResponseFormat::GreptimedbV1 => "greptimedb_v1", ResponseFormat::InfluxdbV1 => "influxdb_v1", + ResponseFormat::Json => "json", } } } @@ -356,6 +361,7 @@ pub enum HttpResponse { Error(ErrorResponse), GreptimedbV1(GreptimedbV1Response), InfluxdbV1(InfluxdbV1Response), + Json(JsonResponse), } impl HttpResponse { @@ -366,6 +372,7 @@ impl HttpResponse { HttpResponse::Table(resp) => resp.with_execution_time(execution_time).into(), HttpResponse::GreptimedbV1(resp) => resp.with_execution_time(execution_time).into(), HttpResponse::InfluxdbV1(resp) => resp.with_execution_time(execution_time).into(), + HttpResponse::Json(resp) => resp.with_execution_time(execution_time).into(), HttpResponse::Error(resp) => resp.with_execution_time(execution_time).into(), } } @@ -375,6 +382,7 @@ impl HttpResponse { HttpResponse::Csv(resp) => resp.with_limit(limit).into(), HttpResponse::Table(resp) => resp.with_limit(limit).into(), HttpResponse::GreptimedbV1(resp) => resp.with_limit(limit).into(), + HttpResponse::Json(resp) => resp.with_limit(limit).into(), _ => self, } } @@ -407,6 +415,7 @@ impl IntoResponse for HttpResponse { HttpResponse::Table(resp) => resp.into_response(), HttpResponse::GreptimedbV1(resp) => resp.into_response(), HttpResponse::InfluxdbV1(resp) => resp.into_response(), + HttpResponse::Json(resp) => resp.into_response(), HttpResponse::Error(resp) => resp.into_response(), } } @@ -452,6 +461,12 @@ impl From for HttpResponse { } } +impl From for HttpResponse { + fn from(value: JsonResponse) -> Self { + HttpResponse::Json(value) + } +} + async fn serve_api(Extension(api): Extension) -> impl IntoApiResponse { Json(api) } @@ -1131,6 +1146,7 @@ mod test { ResponseFormat::Csv, ResponseFormat::Table, ResponseFormat::Arrow, + ResponseFormat::Json, ] { let recordbatches = RecordBatches::try_new(schema.clone(), vec![recordbatch.clone()]).unwrap(); @@ -1141,6 +1157,7 @@ mod test { ResponseFormat::Table => TableResponse::from_output(outputs).await, ResponseFormat::GreptimedbV1 => GreptimedbV1Response::from_output(outputs).await, ResponseFormat::InfluxdbV1 => InfluxdbV1Response::from_output(outputs, None).await, + ResponseFormat::Json => JsonResponse::from_output(outputs).await, }; match json_resp { @@ -1210,6 +1227,21 @@ mod test { assert_eq!(rb.num_columns(), 2); assert_eq!(rb.num_rows(), 4); } + + HttpResponse::Json(resp) => { + let output = &resp.output()[0]; + if let GreptimeQueryOutput::Records(r) = output { + assert_eq!(r.num_rows(), 4); + assert_eq!(r.num_cols(), 2); + assert_eq!(r.schema.column_schemas[0].name, "numbers"); + assert_eq!(r.schema.column_schemas[0].data_type, "UInt32"); + assert_eq!(r.rows[0][0], serde_json::Value::from(1)); + assert_eq!(r.rows[0][1], serde_json::Value::Null); + } else { + panic!("invalid output type"); + } + } + HttpResponse::Error(err) => unreachable!("{err:?}"), } } diff --git a/src/servers/src/http/handler.rs b/src/servers/src/http/handler.rs index 1befc22240..4925c79639 100644 --- a/src/servers/src/http/handler.rs +++ b/src/servers/src/http/handler.rs @@ -39,6 +39,7 @@ use crate::http::csv_result::CsvResponse; use crate::http::error_result::ErrorResponse; use crate::http::greptime_result_v1::GreptimedbV1Response; use crate::http::influxdb_result_v1::InfluxdbV1Response; +use crate::http::json_result::JsonResponse; use crate::http::table_result::TableResponse; use crate::http::{ ApiState, Epoch, GreptimeOptionsConfigState, GreptimeQueryOutput, HttpRecordsOutput, @@ -138,6 +139,7 @@ pub async fn sql( ResponseFormat::Table => TableResponse::from_output(outputs).await, ResponseFormat::GreptimedbV1 => GreptimedbV1Response::from_output(outputs).await, ResponseFormat::InfluxdbV1 => InfluxdbV1Response::from_output(outputs, epoch).await, + ResponseFormat::Json => JsonResponse::from_output(outputs).await, }; if let Some(limit) = query_params.limit { diff --git a/src/servers/src/http/json_result.rs b/src/servers/src/http/json_result.rs new file mode 100644 index 0000000000..bf4e4d7770 --- /dev/null +++ b/src/servers/src/http/json_result.rs @@ -0,0 +1,137 @@ +// 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 axum::http::{header, HeaderValue}; +use axum::response::{IntoResponse, Response}; +use common_error::status_code::StatusCode; +use common_query::Output; +use mime_guess::mime; +use schemars::JsonSchema; +use serde::{Deserialize, Serialize}; +use serde_json::{json, Map, Value}; + +use super::process_with_limit; +use crate::http::error_result::ErrorResponse; +use crate::http::header::{GREPTIME_DB_HEADER_EXECUTION_TIME, GREPTIME_DB_HEADER_FORMAT}; +use crate::http::{handler, GreptimeQueryOutput, HttpResponse, ResponseFormat}; + +/// The json format here is different from the default json output of `GreptimedbV1` result. +/// `JsonResponse` is intended to make it easier for user to consume data. +#[derive(Serialize, Deserialize, Debug, JsonSchema)] +pub struct JsonResponse { + output: Vec, + execution_time_ms: u64, +} + +impl JsonResponse { + pub async fn from_output(outputs: Vec>) -> HttpResponse { + match handler::from_output(outputs).await { + Err(err) => HttpResponse::Error(err), + Ok((output, _)) => { + if output.len() > 1 { + HttpResponse::Error(ErrorResponse::from_error_message( + StatusCode::InvalidArguments, + "cannot output multi-statements result in json format".to_string(), + )) + } else { + HttpResponse::Json(JsonResponse { + output, + execution_time_ms: 0, + }) + } + } + } + } + + pub fn output(&self) -> &[GreptimeQueryOutput] { + &self.output + } + + pub fn with_execution_time(mut self, execution_time: u64) -> Self { + self.execution_time_ms = execution_time; + self + } + + pub fn execution_time_ms(&self) -> u64 { + self.execution_time_ms + } + + pub fn with_limit(mut self, limit: usize) -> Self { + self.output = process_with_limit(self.output, limit); + self + } +} + +impl IntoResponse for JsonResponse { + fn into_response(mut self) -> Response { + debug_assert!( + self.output.len() <= 1, + "self.output has extra elements: {}", + self.output.len() + ); + + let execution_time = self.execution_time_ms; + let payload = match self.output.pop() { + None => String::default(), + Some(GreptimeQueryOutput::AffectedRows(n)) => json!({ + "data": [], + "affected_rows": n, + "execution_time_ms": execution_time, + }) + .to_string(), + + Some(GreptimeQueryOutput::Records(records)) => { + let schema = records.schema(); + + let data: Vec> = records + .rows + .iter() + .map(|row| { + schema + .column_schemas + .iter() + .enumerate() + .map(|(i, col)| (col.name.clone(), row[i].clone())) + .collect::>() + }) + .collect(); + + json!({ + "data": data, + "execution_time_ms": execution_time, + }) + .to_string() + } + }; + + ( + [ + ( + header::CONTENT_TYPE, + HeaderValue::from_static(mime::APPLICATION_JSON.as_ref()), + ), + ( + GREPTIME_DB_HEADER_FORMAT.clone(), + HeaderValue::from_static(ResponseFormat::Json.as_str()), + ), + ( + GREPTIME_DB_HEADER_EXECUTION_TIME.clone(), + HeaderValue::from(execution_time), + ), + ], + payload, + ) + .into_response() + } +} diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index f70df8176a..7a030aad56 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -182,6 +182,22 @@ pub async fn test_sql_api(store_type: StorageType) { })).unwrap() ); + // test json result format + let res = client + .get("/v1/sql?format=json&sql=select * from numbers limit 10") + .send() + .await; + assert_eq!(res.status(), StatusCode::OK); + + let body = res.json::().await; + let data = body.get("data").expect("Missing 'data' field in response"); + + let expected = json!([ + {"number": 0}, {"number": 1}, {"number": 2}, {"number": 3}, {"number": 4}, + {"number": 5}, {"number": 6}, {"number": 7}, {"number": 8}, {"number": 9} + ]); + assert_eq!(data, &expected); + // test insert and select let res = client .get("/v1/sql?sql=insert into demo values('host', 66.6, 1024, 0)") @@ -1307,7 +1323,7 @@ transform: .send() .await; assert_eq!(res.status(), StatusCode::OK); - let body: serde_json::Value = res.json().await; + let body: Value = res.json().await; let schema = &body["schema"]; let rows = &body["rows"]; assert_eq!( From a8ed3db0aa272396c5818d42303e263a8faf4be4 Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Wed, 9 Oct 2024 17:37:27 +0800 Subject: [PATCH 061/128] feat: Merge sort Logical plan (#4768) * feat(WIP): MergeSort * wip * feat: MergeSort LogicalPlan * update sqlness result * Apply suggestions from code review Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com> * refactor: per review advice * refactor: more per review * chore: per review --------- Co-authored-by: Ruihang Xia Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com> --- src/query/src/dist_plan.rs | 3 +- src/query/src/dist_plan/analyzer.rs | 4 +- src/query/src/dist_plan/commutativity.rs | 9 +- src/query/src/dist_plan/merge_scan.rs | 8 ++ src/query/src/dist_plan/merge_sort.rs | 124 ++++++++++++++++++ src/query/src/dist_plan/planner.rs | 62 ++++++++- src/query/src/query_engine/state.rs | 3 +- .../explain/multi_partitions.result | 2 +- .../standalone/common/order/order_by.result | 2 +- 9 files changed, 207 insertions(+), 10 deletions(-) create mode 100644 src/query/src/dist_plan/merge_sort.rs diff --git a/src/query/src/dist_plan.rs b/src/query/src/dist_plan.rs index 6ab93d4e1d..b2bad029c4 100644 --- a/src/query/src/dist_plan.rs +++ b/src/query/src/dist_plan.rs @@ -15,8 +15,9 @@ mod analyzer; mod commutativity; mod merge_scan; +mod merge_sort; mod planner; pub use analyzer::DistPlannerAnalyzer; pub use merge_scan::{MergeScanExec, MergeScanLogicalPlan}; -pub use planner::DistExtensionPlanner; +pub use planner::{DistExtensionPlanner, MergeSortExtensionPlanner}; diff --git a/src/query/src/dist_plan/analyzer.rs b/src/query/src/dist_plan/analyzer.rs index c182eee1cd..335c23c58a 100644 --- a/src/query/src/dist_plan/analyzer.rs +++ b/src/query/src/dist_plan/analyzer.rs @@ -160,7 +160,6 @@ impl PlanRewriter { { return true; } - match Categorizer::check_plan(plan, self.partition_cols.clone()) { Commutativity::Commutative => {} Commutativity::PartialCommutative => { @@ -265,9 +264,10 @@ impl PlanRewriter { // add merge scan as the new root let mut node = MergeScanLogicalPlan::new(on_node, false).into_logical_plan(); + // expand stages for new_stage in self.stage.drain(..) { - node = new_stage.with_new_exprs(new_stage.expressions(), vec![node.clone()])? + node = new_stage.with_new_exprs(new_stage.expressions(), vec![node.clone()])?; } self.set_expanded(); diff --git a/src/query/src/dist_plan/commutativity.rs b/src/query/src/dist_plan/commutativity.rs index c6b3ca2c62..8166400b8f 100644 --- a/src/query/src/dist_plan/commutativity.rs +++ b/src/query/src/dist_plan/commutativity.rs @@ -21,6 +21,7 @@ use promql::extension_plan::{ EmptyMetric, InstantManipulate, RangeManipulate, SeriesDivide, SeriesNormalize, }; +use crate::dist_plan::merge_sort::{merge_sort_transformer, MergeSortLogicalPlan}; use crate::dist_plan::MergeScanLogicalPlan; #[allow(dead_code)] @@ -68,8 +69,9 @@ impl Categorizer { } // sort plan needs to consider column priority - // We can implement a merge-sort on partial ordered data - Commutativity::PartialCommutative + // Change Sort to MergeSort which assumes the input streams are already sorted hence can be more efficient + // We should ensure the number of partition is not smaller than the number of region at present. Otherwise this would result in incorrect output. + Commutativity::ConditionalCommutative(Some(Arc::new(merge_sort_transformer))) } LogicalPlan::Join(_) => Commutativity::NonCommutative, LogicalPlan::CrossJoin(_) => Commutativity::NonCommutative, @@ -118,7 +120,8 @@ impl Categorizer { || name == SeriesNormalize::name() || name == RangeManipulate::name() || name == SeriesDivide::name() - || name == MergeScanLogicalPlan::name() => + || name == MergeScanLogicalPlan::name() + || name == MergeSortLogicalPlan::name() => { Commutativity::Unimplemented } diff --git a/src/query/src/dist_plan/merge_scan.rs b/src/query/src/dist_plan/merge_scan.rs index a3fb8004cf..8ed77eabd4 100644 --- a/src/query/src/dist_plan/merge_scan.rs +++ b/src/query/src/dist_plan/merge_scan.rs @@ -298,6 +298,14 @@ impl MergeScanExec { pub fn sub_stage_metrics(&self) -> Vec { self.sub_stage_metrics.lock().unwrap().clone() } + + pub fn partition_count(&self) -> usize { + self.target_partition + } + + pub fn region_count(&self) -> usize { + self.regions.len() + } } impl ExecutionPlan for MergeScanExec { diff --git a/src/query/src/dist_plan/merge_sort.rs b/src/query/src/dist_plan/merge_sort.rs new file mode 100644 index 0000000000..df0a9fda10 --- /dev/null +++ b/src/query/src/dist_plan/merge_sort.rs @@ -0,0 +1,124 @@ +// 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. + +//! Merge sort logical plan for distributed query execution, roughly corresponding to the +//! `SortPreservingMergeExec` operator in datafusion +//! + +use std::fmt; +use std::sync::Arc; + +use datafusion_common::{DataFusionError, Result}; +use datafusion_expr::{Expr, Extension, LogicalPlan, UserDefinedLogicalNodeCore}; + +/// MergeSort Logical Plan, have same field as `Sort`, but indicate it is a merge sort, +/// which assume each input partition is a sorted stream, and will use `SortPreserveingMergeExec` +/// to merge them into a single sorted stream. +#[derive(Hash, PartialEq, Eq, Clone)] +pub struct MergeSortLogicalPlan { + pub expr: Vec, + pub input: Arc, + pub fetch: Option, +} + +impl fmt::Debug for MergeSortLogicalPlan { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + UserDefinedLogicalNodeCore::fmt_for_explain(self, f) + } +} + +impl MergeSortLogicalPlan { + pub fn new(input: Arc, expr: Vec, fetch: Option) -> Self { + Self { input, expr, fetch } + } + + pub fn name() -> &'static str { + "MergeSort" + } + + /// Create a [`LogicalPlan::Extension`] node from this merge sort plan + pub fn into_logical_plan(self) -> LogicalPlan { + LogicalPlan::Extension(Extension { + node: Arc::new(self), + }) + } + + /// Convert self to a [`Sort`] logical plan with same input and expressions + pub fn into_sort(self) -> LogicalPlan { + LogicalPlan::Sort(datafusion::logical_expr::Sort { + input: self.input.clone(), + expr: self.expr, + fetch: self.fetch, + }) + } +} + +impl UserDefinedLogicalNodeCore for MergeSortLogicalPlan { + fn name(&self) -> &str { + Self::name() + } + + // Allow optimization here + fn inputs(&self) -> Vec<&LogicalPlan> { + vec![self.input.as_ref()] + } + + fn schema(&self) -> &datafusion_common::DFSchemaRef { + self.input.schema() + } + + // Allow further optimization + fn expressions(&self) -> Vec { + self.expr.clone() + } + + fn fmt_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "MergeSort: ")?; + for (i, expr_item) in self.expr.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{expr_item}")?; + } + if let Some(a) = self.fetch { + write!(f, ", fetch={a}")?; + } + Ok(()) + } + + fn with_exprs_and_inputs( + &self, + exprs: Vec, + mut inputs: Vec, + ) -> Result { + let mut zelf = self.clone(); + zelf.expr = exprs; + zelf.input = Arc::new(inputs.pop().ok_or_else(|| { + DataFusionError::Internal("Expected exactly one input with MergeSort".to_string()) + })?); + Ok(zelf) + } +} + +/// Turn `Sort` into `MergeSort` if possible +pub fn merge_sort_transformer(plan: &LogicalPlan) -> Option { + if let LogicalPlan::Sort(sort) = plan { + Some( + MergeSortLogicalPlan::new(sort.input.clone(), sort.expr.clone(), sort.fetch) + .into_logical_plan(), + ) + } else { + None + } +} diff --git a/src/query/src/dist_plan/planner.rs b/src/query/src/dist_plan/planner.rs index a94a798461..833ff1c9bc 100644 --- a/src/query/src/dist_plan/planner.rs +++ b/src/query/src/dist_plan/planner.rs @@ -25,7 +25,7 @@ use datafusion::execution::context::SessionState; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_planner::{ExtensionPlanner, PhysicalPlanner}; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; -use datafusion_common::TableReference; +use datafusion_common::{DataFusionError, TableReference}; use datafusion_expr::{LogicalPlan, UserDefinedLogicalNode}; use session::context::QueryContext; use snafu::{OptionExt, ResultExt}; @@ -35,9 +35,69 @@ use table::table::adapter::DfTableProviderAdapter; use table::table_name::TableName; use crate::dist_plan::merge_scan::{MergeScanExec, MergeScanLogicalPlan}; +use crate::dist_plan::merge_sort::MergeSortLogicalPlan; use crate::error::{CatalogSnafu, TableNotFoundSnafu}; use crate::region_query::RegionQueryHandlerRef; +/// Planner for convert merge sort logical plan to physical plan +/// it is currently a fallback to sort, and doesn't change the execution plan: +/// `MergeSort(MergeScan) -> Sort(MergeScan) - to physical plan -> ...` +/// It should be applied after `DistExtensionPlanner` +/// +/// (Later when actually impl this merge sort) +/// +/// We should ensure the number of partition is not smaller than the number of region at present. Otherwise this would result in incorrect output. +pub struct MergeSortExtensionPlanner {} + +#[async_trait] +impl ExtensionPlanner for MergeSortExtensionPlanner { + async fn plan_extension( + &self, + planner: &dyn PhysicalPlanner, + node: &dyn UserDefinedLogicalNode, + _logical_inputs: &[&LogicalPlan], + physical_inputs: &[Arc], + session_state: &SessionState, + ) -> Result>> { + if let Some(merge_sort) = node.as_any().downcast_ref::() { + if let LogicalPlan::Extension(ext) = &merge_sort.input.as_ref() + && ext + .node + .as_any() + .downcast_ref::() + .is_some() + { + let merge_scan_exec = physical_inputs + .first() + .and_then(|p| p.as_any().downcast_ref::()) + .ok_or(DataFusionError::Internal(format!( + "Expect MergeSort's input is a MergeScanExec, found {:?}", + physical_inputs + )))?; + + let partition_cnt = merge_scan_exec.partition_count(); + let region_cnt = merge_scan_exec.region_count(); + // if partition >= region, we know that every partition stream of merge scan is ordered + // and we only need to do a merge sort, otherwise fallback to quick sort + let can_merge_sort = partition_cnt >= region_cnt; + if can_merge_sort { + // TODO(discord9): use `SortPreversingMergeExec here` + } + // for now merge sort only exist in logical plan, and have the same effect as `Sort` + // doesn't change the execution plan, this will change in the future + let ret = planner + .create_physical_plan(&merge_sort.clone().into_sort(), session_state) + .await?; + Ok(Some(ret)) + } else { + Ok(None) + } + } else { + Ok(None) + } + } +} + pub struct DistExtensionPlanner { catalog_manager: CatalogManagerRef, region_query_handler: RegionQueryHandlerRef, diff --git a/src/query/src/query_engine/state.rs b/src/query/src/query_engine/state.rs index f0e2ef53a5..97386e9b2d 100644 --- a/src/query/src/query_engine/state.rs +++ b/src/query/src/query_engine/state.rs @@ -42,7 +42,7 @@ use promql::extension_plan::PromExtensionPlanner; use table::table::adapter::DfTableProviderAdapter; use table::TableRef; -use crate::dist_plan::{DistExtensionPlanner, DistPlannerAnalyzer}; +use crate::dist_plan::{DistExtensionPlanner, DistPlannerAnalyzer, MergeSortExtensionPlanner}; use crate::optimizer::count_wildcard::CountWildcardToTimeIndexRule; use crate::optimizer::parallelize_scan::ParallelizeScan; use crate::optimizer::remove_duplicate::RemoveDuplicate; @@ -295,6 +295,7 @@ impl DfQueryPlanner { catalog_manager, region_query_handler, ))); + planners.push(Arc::new(MergeSortExtensionPlanner {})); } Self { physical_planner: DefaultPhysicalPlanner::with_extension_planners(planners), diff --git a/tests/cases/distributed/explain/multi_partitions.result b/tests/cases/distributed/explain/multi_partitions.result index 6bbb6abb8e..c06869d459 100644 --- a/tests/cases/distributed/explain/multi_partitions.result +++ b/tests/cases/distributed/explain/multi_partitions.result @@ -25,7 +25,7 @@ explain SELECT * FROM demo WHERE ts > cast(1000000000 as timestamp) ORDER BY hos +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Sort: demo.host ASC NULLS LAST_| +| logical_plan_| MergeSort: demo.host ASC NULLS LAST_| |_|_MergeScan [is_placeholder=false]_| | physical_plan | SortPreservingMergeExec: [host@0 ASC NULLS LAST]_| |_|_SortExec: expr=[host@0 ASC NULLS LAST], preserve_partitioning=[true]_| diff --git a/tests/cases/standalone/common/order/order_by.result b/tests/cases/standalone/common/order/order_by.result index f6385537c3..8cf92d9fbf 100644 --- a/tests/cases/standalone/common/order/order_by.result +++ b/tests/cases/standalone/common/order/order_by.result @@ -294,7 +294,7 @@ explain analyze select tag from t where num > 6 order by ts desc limit 2; +-+-+-+ | 0_| 0_|_GlobalLimitExec: skip=0, fetch=2 REDACTED |_|_|_SortPreservingMergeExec: [ts@1 DESC] REDACTED -|_|_|_SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[true] REDACTED +|_|_|_SortExec: expr=[ts@1 DESC], preserve_partitioning=[true] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| | 1_| 0_|_GlobalLimitExec: skip=0, fetch=2 REDACTED From 288fdc31453f7db461a6218328850bfc45081d39 Mon Sep 17 00:00:00 2001 From: Yohan Wal <59358312+CookiePieWw@users.noreply.github.com> Date: Thu, 10 Oct 2024 22:15:34 +0800 Subject: [PATCH 062/128] feat: json_path_exists udf (#4807) * feat: json_path_exists udf * chore: fix comments * fix: caution when copy&paste QAQ --- src/common/function/src/scalars/json.rs | 3 + .../src/scalars/json/json_path_exists.rs | 172 ++++++++++++++++++ .../common/function/json/json.result | 33 ++++ .../standalone/common/function/json/json.sql | 8 + 4 files changed, 216 insertions(+) create mode 100644 src/common/function/src/scalars/json/json_path_exists.rs create mode 100644 tests/cases/standalone/common/function/json/json.result create mode 100644 tests/cases/standalone/common/function/json/json.sql diff --git a/src/common/function/src/scalars/json.rs b/src/common/function/src/scalars/json.rs index 279577b495..133eb145a7 100644 --- a/src/common/function/src/scalars/json.rs +++ b/src/common/function/src/scalars/json.rs @@ -15,6 +15,7 @@ use std::sync::Arc; mod json_get; mod json_is; +mod json_path_exists; mod json_to_string; mod parse_json; @@ -46,5 +47,7 @@ impl JsonFunction { registry.register(Arc::new(JsonIsBool)); registry.register(Arc::new(JsonIsArray)); registry.register(Arc::new(JsonIsObject)); + + registry.register(Arc::new(json_path_exists::JsonPathExistsFunction)); } } diff --git a/src/common/function/src/scalars/json/json_path_exists.rs b/src/common/function/src/scalars/json/json_path_exists.rs new file mode 100644 index 0000000000..9254b5f396 --- /dev/null +++ b/src/common/function/src/scalars/json/json_path_exists.rs @@ -0,0 +1,172 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::{self, Display}; + +use common_query::error::{InvalidFuncArgsSnafu, Result, UnsupportedInputDataTypeSnafu}; +use common_query::prelude::Signature; +use datafusion::logical_expr::Volatility; +use datatypes::data_type::ConcreteDataType; +use datatypes::prelude::VectorRef; +use datatypes::scalars::ScalarVectorBuilder; +use datatypes::vectors::{BooleanVectorBuilder, MutableVector}; +use snafu::ensure; + +use crate::function::{Function, FunctionContext}; + +/// Check if the given JSON data contains the given JSON path. +#[derive(Clone, Debug, Default)] +pub struct JsonPathExistsFunction; + +const NAME: &str = "json_path_exists"; + +impl Function for JsonPathExistsFunction { + fn name(&self) -> &str { + NAME + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::boolean_datatype()) + } + + fn signature(&self) -> Signature { + Signature::exact( + vec![ + ConcreteDataType::json_datatype(), + ConcreteDataType::string_datatype(), + ], + Volatility::Immutable, + ) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure!( + columns.len() == 2, + InvalidFuncArgsSnafu { + err_msg: format!( + "The length of the args is not correct, expect exactly two, have: {}", + columns.len() + ), + } + ); + let jsons = &columns[0]; + let paths = &columns[1]; + + let size = jsons.len(); + let datatype = jsons.data_type(); + let mut results = BooleanVectorBuilder::with_capacity(size); + + match datatype { + // JSON data type uses binary vector + ConcreteDataType::Binary(_) => { + for i in 0..size { + let json = jsons.get_ref(i); + let path = paths.get_ref(i); + + let json = json.as_binary(); + let path = path.as_string(); + let result = match (json, path) { + (Ok(Some(json)), Ok(Some(path))) => { + let json_path = jsonb::jsonpath::parse_json_path(path.as_bytes()); + match json_path { + Ok(json_path) => jsonb::path_exists(json, json_path).ok(), + Err(_) => None, + } + } + _ => None, + }; + + results.push(result); + } + } + _ => { + return UnsupportedInputDataTypeSnafu { + function: NAME, + datatypes: columns.iter().map(|c| c.data_type()).collect::>(), + } + .fail(); + } + } + + Ok(results.to_vector()) + } +} + +impl Display for JsonPathExistsFunction { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "JSON_PATH_EXISTS") + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use common_query::prelude::TypeSignature; + use datatypes::scalars::ScalarVector; + use datatypes::vectors::{BinaryVector, StringVector}; + + use super::*; + + #[test] + fn test_json_path_exists_function() { + let json_path_exists = JsonPathExistsFunction; + + assert_eq!("json_path_exists", json_path_exists.name()); + assert_eq!( + ConcreteDataType::boolean_datatype(), + json_path_exists + .return_type(&[ConcreteDataType::json_datatype()]) + .unwrap() + ); + + assert!(matches!(json_path_exists.signature(), + Signature { + type_signature: TypeSignature::Exact(valid_types), + volatility: Volatility::Immutable + } if valid_types == vec![ConcreteDataType::json_datatype(), ConcreteDataType::string_datatype()] + )); + + let json_strings = [ + r#"{"a": {"b": 2}, "b": 2, "c": 3}"#, + r#"{"a": 4, "b": {"c": 6}, "c": 6}"#, + r#"{"a": 7, "b": 8, "c": {"a": 7}}"#, + r#"{"a": 7, "b": 8, "c": {"a": 7}}"#, + ]; + let paths = vec!["$.a.b.c", "$.b", "$.c.a", ".d"]; + let results = [false, true, true, false]; + + let jsonbs = json_strings + .iter() + .map(|s| { + let value = jsonb::parse_value(s.as_bytes()).unwrap(); + value.to_vec() + }) + .collect::>(); + + let json_vector = BinaryVector::from_vec(jsonbs); + let path_vector = StringVector::from_vec(paths); + let args: Vec = vec![Arc::new(json_vector), Arc::new(path_vector)]; + let vector = json_path_exists + .eval(FunctionContext::default(), &args) + .unwrap(); + + assert_eq!(4, vector.len()); + for (i, gt) in results.iter().enumerate() { + let result = vector.get_ref(i); + let result = result.as_boolean().unwrap().unwrap(); + assert_eq!(*gt, result); + } + } +} diff --git a/tests/cases/standalone/common/function/json/json.result b/tests/cases/standalone/common/function/json/json.result new file mode 100644 index 0000000000..bae3313548 --- /dev/null +++ b/tests/cases/standalone/common/function/json/json.result @@ -0,0 +1,33 @@ +--- json_path_exists --- +SELECT json_path_exists(parse_json('{"a": 1, "b": 2}'), '$.a'); + ++--------------------------------------------------------------------+ +| json_path_exists(parse_json(Utf8("{"a": 1, "b": 2}")),Utf8("$.a")) | ++--------------------------------------------------------------------+ +| true | ++--------------------------------------------------------------------+ + +SELECT json_path_exists(parse_json('{"a": 1, "b": 2}'), '$.c'); + ++--------------------------------------------------------------------+ +| json_path_exists(parse_json(Utf8("{"a": 1, "b": 2}")),Utf8("$.c")) | ++--------------------------------------------------------------------+ +| false | ++--------------------------------------------------------------------+ + +SELECT json_path_exists(parse_json('[1, 2]'), '[0]'); + ++----------------------------------------------------------+ +| json_path_exists(parse_json(Utf8("[1, 2]")),Utf8("[0]")) | ++----------------------------------------------------------+ +| true | ++----------------------------------------------------------+ + +SELECT json_path_exists(parse_json('[1, 2]'), '[2]'); + ++----------------------------------------------------------+ +| json_path_exists(parse_json(Utf8("[1, 2]")),Utf8("[2]")) | ++----------------------------------------------------------+ +| false | ++----------------------------------------------------------+ + diff --git a/tests/cases/standalone/common/function/json/json.sql b/tests/cases/standalone/common/function/json/json.sql new file mode 100644 index 0000000000..76914e2575 --- /dev/null +++ b/tests/cases/standalone/common/function/json/json.sql @@ -0,0 +1,8 @@ +--- json_path_exists --- +SELECT json_path_exists(parse_json('{"a": 1, "b": 2}'), '$.a'); + +SELECT json_path_exists(parse_json('{"a": 1, "b": 2}'), '$.c'); + +SELECT json_path_exists(parse_json('[1, 2]'), '[0]'); + +SELECT json_path_exists(parse_json('[1, 2]'), '[2]'); From 695ff1e03713c554b50eb9f6245dbebc8decb91c Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 11 Oct 2024 10:40:51 +0800 Subject: [PATCH 063/128] feat: expose `RegionMigrationManagerRef` (#4812) * chore: expose `RegionMigrationProcedureTask` * fix: fix typos * chore: expose `tracker` --- src/meta-srv/src/procedure/region_migration.rs | 6 ++++-- src/meta-srv/src/procedure/region_migration/manager.rs | 4 ++-- src/meta-srv/src/region/supervisor.rs | 8 ++++---- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/meta-srv/src/procedure/region_migration.rs b/src/meta-srv/src/procedure/region_migration.rs index 9b5bdfc006..72b886418c 100644 --- a/src/meta-srv/src/procedure/region_migration.rs +++ b/src/meta-srv/src/procedure/region_migration.rs @@ -43,8 +43,10 @@ use common_procedure::error::{ Error as ProcedureError, FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu, }; use common_procedure::{Context as ProcedureContext, LockKey, Procedure, Status, StringKey}; -pub use manager::RegionMigrationProcedureTask; -use manager::{RegionMigrationProcedureGuard, RegionMigrationProcedureTracker}; +use manager::RegionMigrationProcedureGuard; +pub use manager::{ + RegionMigrationManagerRef, RegionMigrationProcedureTask, RegionMigrationProcedureTracker, +}; use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; use store_api::storage::RegionId; diff --git a/src/meta-srv/src/procedure/region_migration/manager.rs b/src/meta-srv/src/procedure/region_migration/manager.rs index 01ea887ca9..1059c1794a 100644 --- a/src/meta-srv/src/procedure/region_migration/manager.rs +++ b/src/meta-srv/src/procedure/region_migration/manager.rs @@ -44,7 +44,7 @@ pub struct RegionMigrationManager { } #[derive(Default, Clone)] -pub(crate) struct RegionMigrationProcedureTracker { +pub struct RegionMigrationProcedureTracker { running_procedures: Arc>>, } @@ -149,7 +149,7 @@ impl RegionMigrationManager { } /// Returns the [`RegionMigrationProcedureTracker`]. - pub(crate) fn tracker(&self) -> &RegionMigrationProcedureTracker { + pub fn tracker(&self) -> &RegionMigrationProcedureTracker { &self.tracker } diff --git a/src/meta-srv/src/region/supervisor.rs b/src/meta-srv/src/region/supervisor.rs index 8367acbb26..80f6c84865 100644 --- a/src/meta-srv/src/region/supervisor.rs +++ b/src/meta-srv/src/region/supervisor.rs @@ -223,7 +223,7 @@ impl RegionFailureDetectorController for RegionFailureDetectorControl { .send(Event::RegisterFailureDetectors(detecting_regions)) .await { - error!(err; "RegionSupervisor is stop receiving heartbeat"); + error!(err; "RegionSupervisor has stop receiving heartbeat."); } } @@ -233,7 +233,7 @@ impl RegionFailureDetectorController for RegionFailureDetectorControl { .send(Event::DeregisterFailureDetectors(detecting_regions)) .await { - error!(err; "RegionSupervisor is stop receiving heartbeat"); + error!(err; "RegionSupervisor has stop receiving heartbeat."); } } } @@ -251,13 +251,13 @@ impl HeartbeatAcceptor { /// Accepts heartbeats from datanodes. pub(crate) async fn accept(&self, heartbeat: DatanodeHeartbeat) { if let Err(err) = self.sender.send(Event::HeartbeatArrived(heartbeat)).await { - error!(err; "RegionSupervisor is stop receiving heartbeat"); + error!(err; "RegionSupervisor has stop receiving heartbeat."); } } } impl RegionSupervisor { - /// Returns a a mpsc channel with a buffer capacity of 1024 for sending and receiving `Event` messages. + /// Returns a mpsc channel with a buffer capacity of 1024 for sending and receiving `Event` messages. pub(crate) fn channel() -> (Sender, Receiver) { tokio::sync::mpsc::channel(1024) } From 4b34f610aa35d8847c7d713cd7bb588e571f5df3 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Fri, 11 Oct 2024 11:13:49 +0800 Subject: [PATCH 064/128] feat: information extension (#4811) * feat: information extension * Update manager.rs Co-authored-by: Weny Xu * chore: by comment --------- Co-authored-by: Weny Xu --- src/catalog/src/error.rs | 7 +- src/catalog/src/kvbackend/manager.rs | 26 ++-- .../src/system_schema/information_schema.rs | 42 ++++++- .../information_schema/cluster_info.rs | 69 ++--------- .../information_schema/procedure_info.rs | 83 ++----------- .../information_schema/region_statistics.rs | 30 +---- src/catalog/src/system_schema/utils.rs | 55 ++------- src/catalog/src/table_source.rs | 6 +- src/cmd/src/cli/repl.rs | 6 +- src/cmd/src/flownode.rs | 7 +- src/cmd/src/frontend.rs | 8 +- src/cmd/src/lib.rs | 76 ++++++++++++ src/cmd/src/standalone.rs | 112 ++++++++++++++++-- tests-integration/src/cluster.rs | 6 +- tests-integration/src/standalone.rs | 4 +- 15 files changed, 284 insertions(+), 253 deletions(-) diff --git a/src/catalog/src/error.rs b/src/catalog/src/error.rs index dd7071b095..0d9e96ab6a 100644 --- a/src/catalog/src/error.rs +++ b/src/catalog/src/error.rs @@ -89,9 +89,8 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to get procedure client in {mode} mode"))] - GetProcedureClient { - mode: String, + #[snafu(display("Failed to get information extension client"))] + GetInformationExtension { #[snafu(implicit)] location: Location, }, @@ -301,7 +300,7 @@ impl ErrorExt for Error { | Error::CacheNotFound { .. } | Error::CastManager { .. } | Error::Json { .. } - | Error::GetProcedureClient { .. } + | Error::GetInformationExtension { .. } | Error::ProcedureIdNotFound { .. } => StatusCode::Unexpected, Error::ViewPlanColumnsChanged { .. } => StatusCode::InvalidArguments, diff --git a/src/catalog/src/kvbackend/manager.rs b/src/catalog/src/kvbackend/manager.rs index e3377afbe1..ca20805d37 100644 --- a/src/catalog/src/kvbackend/manager.rs +++ b/src/catalog/src/kvbackend/manager.rs @@ -21,7 +21,6 @@ use common_catalog::consts::{ DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, INFORMATION_SCHEMA_NAME, NUMBERS_TABLE_ID, PG_CATALOG_NAME, }; -use common_config::Mode; use common_error::ext::BoxedError; use common_meta::cache::{LayeredCacheRegistryRef, ViewInfoCacheRef}; use common_meta::key::catalog_name::CatalogNameKey; @@ -34,7 +33,6 @@ use common_meta::kv_backend::KvBackendRef; use common_procedure::ProcedureManagerRef; use futures_util::stream::BoxStream; use futures_util::{StreamExt, TryStreamExt}; -use meta_client::client::MetaClient; use moka::sync::Cache; use partition::manager::{PartitionRuleManager, PartitionRuleManagerRef}; use session::context::{Channel, QueryContext}; @@ -50,7 +48,7 @@ use crate::error::{ CacheNotFoundSnafu, GetTableCacheSnafu, InvalidTableInfoInCatalogSnafu, ListCatalogsSnafu, ListSchemasSnafu, ListTablesSnafu, Result, TableMetadataManagerSnafu, }; -use crate::information_schema::InformationSchemaProvider; +use crate::information_schema::{InformationExtensionRef, InformationSchemaProvider}; use crate::kvbackend::TableCacheRef; use crate::system_schema::pg_catalog::PGCatalogProvider; use crate::system_schema::SystemSchemaProvider; @@ -63,9 +61,8 @@ use crate::CatalogManager; /// comes from `SystemCatalog`, which is static and read-only. #[derive(Clone)] pub struct KvBackendCatalogManager { - mode: Mode, - /// Only available in `Distributed` mode. - meta_client: Option>, + /// Provides the extension methods for the `information_schema` tables + information_extension: InformationExtensionRef, /// Manages partition rules. partition_manager: PartitionRuleManagerRef, /// Manages table metadata. @@ -82,15 +79,13 @@ const CATALOG_CACHE_MAX_CAPACITY: u64 = 128; impl KvBackendCatalogManager { pub fn new( - mode: Mode, - meta_client: Option>, + information_extension: InformationExtensionRef, backend: KvBackendRef, cache_registry: LayeredCacheRegistryRef, procedure_manager: Option, ) -> Arc { Arc::new_cyclic(|me| Self { - mode, - meta_client, + information_extension, partition_manager: Arc::new(PartitionRuleManager::new( backend.clone(), cache_registry @@ -118,20 +113,15 @@ impl KvBackendCatalogManager { }) } - /// Returns the server running mode. - pub fn running_mode(&self) -> &Mode { - &self.mode - } - pub fn view_info_cache(&self) -> Result { self.cache_registry.get().context(CacheNotFoundSnafu { name: "view_info_cache", }) } - /// Returns the `[MetaClient]`. - pub fn meta_client(&self) -> Option> { - self.meta_client.clone() + /// Returns the [`InformationExtension`]. + pub fn information_extension(&self) -> InformationExtensionRef { + self.information_extension.clone() } pub fn partition_manager(&self) -> PartitionRuleManagerRef { diff --git a/src/catalog/src/system_schema/information_schema.rs b/src/catalog/src/system_schema/information_schema.rs index 9fa31b85fd..4101887cb4 100644 --- a/src/catalog/src/system_schema/information_schema.rs +++ b/src/catalog/src/system_schema/information_schema.rs @@ -32,7 +32,11 @@ use std::collections::HashMap; use std::sync::{Arc, Weak}; use common_catalog::consts::{self, DEFAULT_CATALOG_NAME, INFORMATION_SCHEMA_NAME}; +use common_error::ext::ErrorExt; +use common_meta::cluster::NodeInfo; +use common_meta::datanode::RegionStat; use common_meta::key::flow::FlowMetadataManager; +use common_procedure::ProcedureInfo; use common_recordbatch::SendableRecordBatchStream; use datatypes::schema::SchemaRef; use lazy_static::lazy_static; @@ -45,7 +49,7 @@ use views::InformationSchemaViews; use self::columns::InformationSchemaColumns; use super::{SystemSchemaProviderInner, SystemTable, SystemTableRef}; -use crate::error::Result; +use crate::error::{Error, Result}; use crate::system_schema::information_schema::cluster_info::InformationSchemaClusterInfo; use crate::system_schema::information_schema::flows::InformationSchemaFlows; use crate::system_schema::information_schema::information_memory_table::get_schema_columns; @@ -318,3 +322,39 @@ where InformationTable::to_stream(self, request) } } + +pub type InformationExtensionRef = Arc + Send + Sync>; + +/// The `InformationExtension` trait provides the extension methods for the `information_schema` tables. +#[async_trait::async_trait] +pub trait InformationExtension { + type Error: ErrorExt; + + /// Gets the nodes information. + async fn nodes(&self) -> std::result::Result, Self::Error>; + + /// Gets the procedures information. + async fn procedures(&self) -> std::result::Result, Self::Error>; + + /// Gets the region statistics. + async fn region_stats(&self) -> std::result::Result, Self::Error>; +} + +pub struct NoopInformationExtension; + +#[async_trait::async_trait] +impl InformationExtension for NoopInformationExtension { + type Error = Error; + + async fn nodes(&self) -> std::result::Result, Self::Error> { + Ok(vec![]) + } + + async fn procedures(&self) -> std::result::Result, Self::Error> { + Ok(vec![]) + } + + async fn region_stats(&self) -> std::result::Result, Self::Error> { + Ok(vec![]) + } +} diff --git a/src/catalog/src/system_schema/information_schema/cluster_info.rs b/src/catalog/src/system_schema/information_schema/cluster_info.rs index e85e210304..1ab700497c 100644 --- a/src/catalog/src/system_schema/information_schema/cluster_info.rs +++ b/src/catalog/src/system_schema/information_schema/cluster_info.rs @@ -17,13 +17,10 @@ use std::time::Duration; use arrow_schema::SchemaRef as ArrowSchemaRef; use common_catalog::consts::INFORMATION_SCHEMA_CLUSTER_INFO_TABLE_ID; -use common_config::Mode; use common_error::ext::BoxedError; -use common_meta::cluster::{ClusterInfo, NodeInfo, NodeStatus}; -use common_meta::peer::Peer; +use common_meta::cluster::NodeInfo; use common_recordbatch::adapter::RecordBatchStreamAdapter; use common_recordbatch::{RecordBatch, SendableRecordBatchStream}; -use common_telemetry::warn; use common_time::timestamp::Timestamp; use datafusion::execution::TaskContext; use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter; @@ -40,7 +37,7 @@ use snafu::ResultExt; use store_api::storage::{ScanRequest, TableId}; use super::CLUSTER_INFO; -use crate::error::{CreateRecordBatchSnafu, InternalSnafu, ListNodesSnafu, Result}; +use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result}; use crate::system_schema::information_schema::{InformationTable, Predicates}; use crate::system_schema::utils; use crate::CatalogManager; @@ -70,7 +67,6 @@ const INIT_CAPACITY: usize = 42; pub(super) struct InformationSchemaClusterInfo { schema: SchemaRef, catalog_manager: Weak, - start_time_ms: u64, } impl InformationSchemaClusterInfo { @@ -78,7 +74,6 @@ impl InformationSchemaClusterInfo { Self { schema: Self::schema(), catalog_manager, - start_time_ms: common_time::util::current_time_millis() as u64, } } @@ -100,11 +95,7 @@ impl InformationSchemaClusterInfo { } fn builder(&self) -> InformationSchemaClusterInfoBuilder { - InformationSchemaClusterInfoBuilder::new( - self.schema.clone(), - self.catalog_manager.clone(), - self.start_time_ms, - ) + InformationSchemaClusterInfoBuilder::new(self.schema.clone(), self.catalog_manager.clone()) } } @@ -144,7 +135,6 @@ impl InformationTable for InformationSchemaClusterInfo { struct InformationSchemaClusterInfoBuilder { schema: SchemaRef, - start_time_ms: u64, catalog_manager: Weak, peer_ids: Int64VectorBuilder, @@ -158,11 +148,7 @@ struct InformationSchemaClusterInfoBuilder { } impl InformationSchemaClusterInfoBuilder { - fn new( - schema: SchemaRef, - catalog_manager: Weak, - start_time_ms: u64, - ) -> Self { + fn new(schema: SchemaRef, catalog_manager: Weak) -> Self { Self { schema, catalog_manager, @@ -174,56 +160,17 @@ impl InformationSchemaClusterInfoBuilder { start_times: TimestampMillisecondVectorBuilder::with_capacity(INIT_CAPACITY), uptimes: StringVectorBuilder::with_capacity(INIT_CAPACITY), active_times: StringVectorBuilder::with_capacity(INIT_CAPACITY), - start_time_ms, } } /// Construct the `information_schema.cluster_info` virtual table async fn make_cluster_info(&mut self, request: Option) -> Result { let predicates = Predicates::from_scan_request(&request); - let mode = utils::running_mode(&self.catalog_manager)?.unwrap_or(Mode::Standalone); - - match mode { - Mode::Standalone => { - let build_info = common_version::build_info(); - - self.add_node_info( - &predicates, - NodeInfo { - // For the standalone: - // - id always 0 - // - empty string for peer_addr - peer: Peer { - id: 0, - addr: "".to_string(), - }, - last_activity_ts: -1, - status: NodeStatus::Standalone, - version: build_info.version.to_string(), - git_commit: build_info.commit_short.to_string(), - // Use `self.start_time_ms` instead. - // It's not precise but enough. - start_time_ms: self.start_time_ms, - }, - ); - } - Mode::Distributed => { - if let Some(meta_client) = utils::meta_client(&self.catalog_manager)? { - let node_infos = meta_client - .list_nodes(None) - .await - .map_err(BoxedError::new) - .context(ListNodesSnafu)?; - - for node_info in node_infos { - self.add_node_info(&predicates, node_info); - } - } else { - warn!("Could not find meta client in distributed mode."); - } - } + let information_extension = utils::information_extension(&self.catalog_manager)?; + let node_infos = information_extension.nodes().await?; + for node_info in node_infos { + self.add_node_info(&predicates, node_info); } - self.finish() } diff --git a/src/catalog/src/system_schema/information_schema/procedure_info.rs b/src/catalog/src/system_schema/information_schema/procedure_info.rs index 56c36c2210..6e3c0b1f46 100644 --- a/src/catalog/src/system_schema/information_schema/procedure_info.rs +++ b/src/catalog/src/system_schema/information_schema/procedure_info.rs @@ -14,14 +14,10 @@ use std::sync::{Arc, Weak}; -use api::v1::meta::{ProcedureMeta, ProcedureStatus}; use arrow_schema::SchemaRef as ArrowSchemaRef; use common_catalog::consts::INFORMATION_SCHEMA_PROCEDURE_INFO_TABLE_ID; -use common_config::Mode; use common_error::ext::BoxedError; -use common_meta::ddl::{ExecutorContext, ProcedureExecutor}; -use common_meta::rpc::procedure; -use common_procedure::{ProcedureInfo, ProcedureState}; +use common_procedure::ProcedureInfo; use common_recordbatch::adapter::RecordBatchStreamAdapter; use common_recordbatch::{RecordBatch, SendableRecordBatchStream}; use common_time::timestamp::Timestamp; @@ -38,10 +34,7 @@ use snafu::ResultExt; use store_api::storage::{ScanRequest, TableId}; use super::PROCEDURE_INFO; -use crate::error::{ - ConvertProtoDataSnafu, CreateRecordBatchSnafu, GetProcedureClientSnafu, InternalSnafu, - ListProceduresSnafu, ProcedureIdNotFoundSnafu, Result, -}; +use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result}; use crate::system_schema::information_schema::{InformationTable, Predicates}; use crate::system_schema::utils; use crate::CatalogManager; @@ -167,45 +160,11 @@ impl InformationSchemaProcedureInfoBuilder { /// Construct the `information_schema.procedure_info` virtual table async fn make_procedure_info(&mut self, request: Option) -> Result { let predicates = Predicates::from_scan_request(&request); - let mode = utils::running_mode(&self.catalog_manager)?.unwrap_or(Mode::Standalone); - match mode { - Mode::Standalone => { - if let Some(procedure_manager) = utils::procedure_manager(&self.catalog_manager)? { - let procedures = procedure_manager - .list_procedures() - .await - .map_err(BoxedError::new) - .context(ListProceduresSnafu)?; - for procedure in procedures { - self.add_procedure( - &predicates, - procedure.state.as_str_name().to_string(), - procedure, - ); - } - } else { - return GetProcedureClientSnafu { mode: "standalone" }.fail(); - } - } - Mode::Distributed => { - if let Some(meta_client) = utils::meta_client(&self.catalog_manager)? { - let procedures = meta_client - .list_procedures(&ExecutorContext::default()) - .await - .map_err(BoxedError::new) - .context(ListProceduresSnafu)?; - for procedure in procedures.procedures { - self.add_procedure_info(&predicates, procedure)?; - } - } else { - return GetProcedureClientSnafu { - mode: "distributed", - } - .fail(); - } - } - }; - + let information_extension = utils::information_extension(&self.catalog_manager)?; + let procedures = information_extension.procedures().await?; + for (status, procedure_info) in procedures { + self.add_procedure(&predicates, status, procedure_info); + } self.finish() } @@ -247,34 +206,6 @@ impl InformationSchemaProcedureInfoBuilder { self.lock_keys.push(Some(&lock_keys)); } - fn add_procedure_info( - &mut self, - predicates: &Predicates, - procedure: ProcedureMeta, - ) -> Result<()> { - let pid = match procedure.id { - Some(pid) => pid, - None => return ProcedureIdNotFoundSnafu {}.fail(), - }; - let pid = procedure::pb_pid_to_pid(&pid) - .map_err(BoxedError::new) - .context(ConvertProtoDataSnafu)?; - let status = ProcedureStatus::try_from(procedure.status) - .map(|v| v.as_str_name()) - .unwrap_or("Unknown") - .to_string(); - let procedure_info = ProcedureInfo { - id: pid, - type_name: procedure.type_name, - start_time_ms: procedure.start_time_ms, - end_time_ms: procedure.end_time_ms, - state: ProcedureState::Running, - lock_keys: procedure.lock_keys, - }; - self.add_procedure(predicates, status, procedure_info); - Ok(()) - } - fn finish(&mut self) -> Result { let columns: Vec = vec![ Arc::new(self.procedure_ids.finish()), diff --git a/src/catalog/src/system_schema/information_schema/region_statistics.rs b/src/catalog/src/system_schema/information_schema/region_statistics.rs index 07b94ede54..e92558acd0 100644 --- a/src/catalog/src/system_schema/information_schema/region_statistics.rs +++ b/src/catalog/src/system_schema/information_schema/region_statistics.rs @@ -16,13 +16,10 @@ use std::sync::{Arc, Weak}; use arrow_schema::SchemaRef as ArrowSchemaRef; use common_catalog::consts::INFORMATION_SCHEMA_REGION_STATISTICS_TABLE_ID; -use common_config::Mode; use common_error::ext::BoxedError; -use common_meta::cluster::ClusterInfo; use common_meta::datanode::RegionStat; use common_recordbatch::adapter::RecordBatchStreamAdapter; use common_recordbatch::{DfSendableRecordBatchStream, RecordBatch, SendableRecordBatchStream}; -use common_telemetry::tracing::warn; use datafusion::execution::TaskContext; use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter; use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream; @@ -34,7 +31,7 @@ use snafu::ResultExt; use store_api::storage::{ScanRequest, TableId}; use super::{InformationTable, REGION_STATISTICS}; -use crate::error::{CreateRecordBatchSnafu, InternalSnafu, ListRegionStatsSnafu, Result}; +use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result}; use crate::information_schema::Predicates; use crate::system_schema::utils; use crate::CatalogManager; @@ -167,28 +164,11 @@ impl InformationSchemaRegionStatisticsBuilder { request: Option, ) -> Result { let predicates = Predicates::from_scan_request(&request); - let mode = utils::running_mode(&self.catalog_manager)?.unwrap_or(Mode::Standalone); - - match mode { - Mode::Standalone => { - // TODO(weny): implement it - } - Mode::Distributed => { - if let Some(meta_client) = utils::meta_client(&self.catalog_manager)? { - let region_stats = meta_client - .list_region_stats() - .await - .map_err(BoxedError::new) - .context(ListRegionStatsSnafu)?; - for region_stat in region_stats { - self.add_region_statistic(&predicates, region_stat); - } - } else { - warn!("Meta client is not available"); - } - } + let information_extension = utils::information_extension(&self.catalog_manager)?; + let region_stats = information_extension.region_stats().await?; + for region_stat in region_stats { + self.add_region_statistic(&predicates, region_stat); } - self.finish() } diff --git a/src/catalog/src/system_schema/utils.rs b/src/catalog/src/system_schema/utils.rs index b9786bc260..8d8af84bf0 100644 --- a/src/catalog/src/system_schema/utils.rs +++ b/src/catalog/src/system_schema/utils.rs @@ -12,48 +12,33 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod tables; +use std::sync::Weak; -use std::sync::{Arc, Weak}; - -use common_config::Mode; use common_meta::key::TableMetadataManagerRef; -use common_procedure::ProcedureManagerRef; -use meta_client::client::MetaClient; use snafu::OptionExt; -use crate::error::{Result, UpgradeWeakCatalogManagerRefSnafu}; +use crate::error::{GetInformationExtensionSnafu, Result, UpgradeWeakCatalogManagerRefSnafu}; +use crate::information_schema::InformationExtensionRef; use crate::kvbackend::KvBackendCatalogManager; use crate::CatalogManager; -/// Try to get the server running mode from `[CatalogManager]` weak reference. -pub fn running_mode(catalog_manager: &Weak) -> Result> { +pub mod tables; + +/// Try to get the `[InformationExtension]` from `[CatalogManager]` weak reference. +pub fn information_extension( + catalog_manager: &Weak, +) -> Result { let catalog_manager = catalog_manager .upgrade() .context(UpgradeWeakCatalogManagerRefSnafu)?; - Ok(catalog_manager + let information_extension = catalog_manager .as_any() .downcast_ref::() - .map(|manager| manager.running_mode()) - .copied()) -} + .map(|manager| manager.information_extension()) + .context(GetInformationExtensionSnafu)?; -/// Try to get the `[MetaClient]` from `[CatalogManager]` weak reference. -pub fn meta_client(catalog_manager: &Weak) -> Result>> { - let catalog_manager = catalog_manager - .upgrade() - .context(UpgradeWeakCatalogManagerRefSnafu)?; - - let meta_client = match catalog_manager - .as_any() - .downcast_ref::() - { - None => None, - Some(manager) => manager.meta_client(), - }; - - Ok(meta_client) + Ok(information_extension) } /// Try to get the `[TableMetadataManagerRef]` from `[CatalogManager]` weak reference. @@ -69,17 +54,3 @@ pub fn table_meta_manager( .downcast_ref::() .map(|manager| manager.table_metadata_manager_ref().clone())) } - -/// Try to get the `[ProcedureManagerRef]` from `[CatalogManager]` weak reference. -pub fn procedure_manager( - catalog_manager: &Weak, -) -> Result> { - let catalog_manager = catalog_manager - .upgrade() - .context(UpgradeWeakCatalogManagerRefSnafu)?; - - Ok(catalog_manager - .as_any() - .downcast_ref::() - .and_then(|manager| manager.procedure_manager())) -} diff --git a/src/catalog/src/table_source.rs b/src/catalog/src/table_source.rs index 09d3d9d2fd..ca9c5b89d3 100644 --- a/src/catalog/src/table_source.rs +++ b/src/catalog/src/table_source.rs @@ -259,7 +259,6 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use cache::{build_fundamental_cache_registry, with_default_composite_cache_registry}; - use common_config::Mode; use common_meta::cache::{CacheRegistryBuilder, LayeredCacheRegistryBuilder}; use common_meta::key::TableMetadataManager; use common_meta::kv_backend::memory::MemoryKvBackend; @@ -269,6 +268,8 @@ mod tests { use datafusion::logical_expr::builder::LogicalTableSource; use datafusion::logical_expr::{col, lit, LogicalPlan, LogicalPlanBuilder}; + use crate::information_schema::NoopInformationExtension; + struct MockDecoder; impl MockDecoder { pub fn arc() -> Arc { @@ -323,8 +324,7 @@ mod tests { ); let catalog_manager = KvBackendCatalogManager::new( - Mode::Standalone, - None, + Arc::new(NoopInformationExtension), backend.clone(), layered_cache_registry, None, diff --git a/src/cmd/src/cli/repl.rs b/src/cmd/src/cli/repl.rs index b55b1c44d6..0138471227 100644 --- a/src/cmd/src/cli/repl.rs +++ b/src/cmd/src/cli/repl.rs @@ -46,12 +46,12 @@ use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan}; use crate::cli::cmd::ReplCommand; use crate::cli::helper::RustylineHelper; use crate::cli::AttachCommand; -use crate::error; use crate::error::{ CollectRecordBatchesSnafu, ParseSqlSnafu, PlanStatementSnafu, PrettyPrintRecordBatchesSnafu, ReadlineSnafu, ReplCreationSnafu, RequestDatabaseSnafu, Result, StartMetaClientSnafu, SubstraitEncodeLogicalPlanSnafu, }; +use crate::{error, DistributedInformationExtension}; /// Captures the state of the repl, gathers commands and executes them one by one pub struct Repl { @@ -275,9 +275,9 @@ async fn create_query_engine(meta_addr: &str) -> Result { .build(), ); + let information_extension = Arc::new(DistributedInformationExtension::new(meta_client.clone())); let catalog_manager = KvBackendCatalogManager::new( - Mode::Distributed, - Some(meta_client.clone()), + information_extension, cached_meta_backend.clone(), layered_cache_registry, None, diff --git a/src/cmd/src/flownode.rs b/src/cmd/src/flownode.rs index 78dfc90607..d2a8454085 100644 --- a/src/cmd/src/flownode.rs +++ b/src/cmd/src/flownode.rs @@ -41,7 +41,7 @@ use crate::error::{ MissingConfigSnafu, Result, ShutdownFlownodeSnafu, StartFlownodeSnafu, }; use crate::options::{GlobalOptions, GreptimeOptions}; -use crate::{log_versions, App}; +use crate::{log_versions, App, DistributedInformationExtension}; pub const APP_NAME: &str = "greptime-flownode"; @@ -269,9 +269,10 @@ impl StartCommand { .build(), ); + let information_extension = + Arc::new(DistributedInformationExtension::new(meta_client.clone())); let catalog_manager = KvBackendCatalogManager::new( - opts.mode, - Some(meta_client.clone()), + information_extension, cached_meta_backend.clone(), layered_cache_registry.clone(), None, diff --git a/src/cmd/src/frontend.rs b/src/cmd/src/frontend.rs index 7678e90c88..2f10b722c7 100644 --- a/src/cmd/src/frontend.rs +++ b/src/cmd/src/frontend.rs @@ -38,7 +38,6 @@ use frontend::server::Services; use meta_client::{MetaClientOptions, MetaClientType}; use query::stats::StatementStatistics; use servers::tls::{TlsMode, TlsOption}; -use servers::Mode; use snafu::{OptionExt, ResultExt}; use tracing_appender::non_blocking::WorkerGuard; @@ -47,7 +46,7 @@ use crate::error::{ Result, StartFrontendSnafu, }; use crate::options::{GlobalOptions, GreptimeOptions}; -use crate::{log_versions, App}; +use crate::{log_versions, App, DistributedInformationExtension}; type FrontendOptions = GreptimeOptions; @@ -316,9 +315,10 @@ impl StartCommand { .build(), ); + let information_extension = + Arc::new(DistributedInformationExtension::new(meta_client.clone())); let catalog_manager = KvBackendCatalogManager::new( - Mode::Distributed, - Some(meta_client.clone()), + information_extension, cached_meta_backend.clone(), layered_cache_registry.clone(), None, diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index 5797ef5a85..80e1628bc0 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -15,7 +15,17 @@ #![feature(assert_matches, let_chains)] use async_trait::async_trait; +use catalog::information_schema::InformationExtension; +use client::api::v1::meta::ProcedureStatus; +use common_error::ext::BoxedError; +use common_meta::cluster::{ClusterInfo, NodeInfo}; +use common_meta::datanode::RegionStat; +use common_meta::ddl::{ExecutorContext, ProcedureExecutor}; +use common_meta::rpc::procedure; +use common_procedure::{ProcedureInfo, ProcedureState}; use common_telemetry::{error, info}; +use meta_client::MetaClientRef; +use snafu::ResultExt; use crate::error::Result; @@ -94,3 +104,69 @@ fn log_env_flags() { info!("argument: {}", argument); } } + +pub struct DistributedInformationExtension { + meta_client: MetaClientRef, +} + +impl DistributedInformationExtension { + pub fn new(meta_client: MetaClientRef) -> Self { + Self { meta_client } + } +} + +#[async_trait::async_trait] +impl InformationExtension for DistributedInformationExtension { + type Error = catalog::error::Error; + + async fn nodes(&self) -> std::result::Result, Self::Error> { + self.meta_client + .list_nodes(None) + .await + .map_err(BoxedError::new) + .context(catalog::error::ListNodesSnafu) + } + + async fn procedures(&self) -> std::result::Result, Self::Error> { + let procedures = self + .meta_client + .list_procedures(&ExecutorContext::default()) + .await + .map_err(BoxedError::new) + .context(catalog::error::ListProceduresSnafu)? + .procedures; + let mut result = Vec::with_capacity(procedures.len()); + for procedure in procedures { + let pid = match procedure.id { + Some(pid) => pid, + None => return catalog::error::ProcedureIdNotFoundSnafu {}.fail(), + }; + let pid = procedure::pb_pid_to_pid(&pid) + .map_err(BoxedError::new) + .context(catalog::error::ConvertProtoDataSnafu)?; + let status = ProcedureStatus::try_from(procedure.status) + .map(|v| v.as_str_name()) + .unwrap_or("Unknown") + .to_string(); + let procedure_info = ProcedureInfo { + id: pid, + type_name: procedure.type_name, + start_time_ms: procedure.start_time_ms, + end_time_ms: procedure.end_time_ms, + state: ProcedureState::Running, + lock_keys: procedure.lock_keys, + }; + result.push((status, procedure_info)); + } + + Ok(result) + } + + async fn region_stats(&self) -> std::result::Result, Self::Error> { + self.meta_client + .list_region_stats() + .await + .map_err(BoxedError::new) + .context(catalog::error::ListRegionStatsSnafu) + } +} diff --git a/src/cmd/src/standalone.rs b/src/cmd/src/standalone.rs index 4335bd5447..d4a0d823c9 100644 --- a/src/cmd/src/standalone.rs +++ b/src/cmd/src/standalone.rs @@ -17,14 +17,18 @@ use std::{fs, path}; use async_trait::async_trait; use cache::{build_fundamental_cache_registry, with_default_composite_cache_registry}; +use catalog::information_schema::InformationExtension; use catalog::kvbackend::KvBackendCatalogManager; use clap::Parser; +use client::api::v1::meta::RegionRole; use common_base::Plugins; use common_catalog::consts::{MIN_USER_FLOW_ID, MIN_USER_TABLE_ID}; use common_config::{metadata_store_dir, Configurable, KvBackendConfig}; use common_error::ext::BoxedError; use common_meta::cache::LayeredCacheRegistryBuilder; use common_meta::cache_invalidator::CacheInvalidatorRef; +use common_meta::cluster::{NodeInfo, NodeStatus}; +use common_meta::datanode::RegionStat; use common_meta::ddl::flow_meta::{FlowMetadataAllocator, FlowMetadataAllocatorRef}; use common_meta::ddl::table_meta::{TableMetadataAllocator, TableMetadataAllocatorRef}; use common_meta::ddl::{DdlContext, NoopRegionFailureDetectorControl, ProcedureExecutorRef}; @@ -33,10 +37,11 @@ use common_meta::key::flow::{FlowMetadataManager, FlowMetadataManagerRef}; use common_meta::key::{TableMetadataManager, TableMetadataManagerRef}; use common_meta::kv_backend::KvBackendRef; use common_meta::node_manager::NodeManagerRef; +use common_meta::peer::Peer; use common_meta::region_keeper::MemoryRegionKeeper; use common_meta::sequence::SequenceBuilder; use common_meta::wal_options_allocator::{WalOptionsAllocator, WalOptionsAllocatorRef}; -use common_procedure::ProcedureManagerRef; +use common_procedure::{ProcedureInfo, ProcedureManagerRef}; use common_telemetry::info; use common_telemetry::logging::{LoggingOptions, TracingOptions}; use common_time::timezone::set_default_timezone; @@ -44,6 +49,7 @@ use common_version::{short_version, version}; use common_wal::config::DatanodeWalConfig; use datanode::config::{DatanodeOptions, ProcedureConfig, RegionEngineConfig, StorageConfig}; use datanode::datanode::{Datanode, DatanodeBuilder}; +use datanode::region_server::RegionServer; use file_engine::config::EngineConfig as FileEngineConfig; use flow::{FlowWorkerManager, FlownodeBuilder, FrontendInvoker}; use frontend::frontend::FrontendOptions; @@ -478,9 +484,18 @@ impl StartCommand { .build(), ); + let datanode = DatanodeBuilder::new(dn_opts, plugins.clone()) + .with_kv_backend(kv_backend.clone()) + .build() + .await + .context(StartDatanodeSnafu)?; + + let information_extension = Arc::new(StandaloneInformationExtension::new( + datanode.region_server(), + procedure_manager.clone(), + )); let catalog_manager = KvBackendCatalogManager::new( - dn_opts.mode, - None, + information_extension, kv_backend.clone(), layered_cache_registry.clone(), Some(procedure_manager.clone()), @@ -489,12 +504,6 @@ impl StartCommand { let table_metadata_manager = Self::create_table_metadata_manager(kv_backend.clone()).await?; - let datanode = DatanodeBuilder::new(dn_opts, plugins.clone()) - .with_kv_backend(kv_backend.clone()) - .build() - .await - .context(StartDatanodeSnafu)?; - let flow_metadata_manager = Arc::new(FlowMetadataManager::new(kv_backend.clone())); let flow_builder = FlownodeBuilder::new( Default::default(), @@ -644,6 +653,91 @@ impl StartCommand { } } +struct StandaloneInformationExtension { + region_server: RegionServer, + procedure_manager: ProcedureManagerRef, + start_time_ms: u64, +} + +impl StandaloneInformationExtension { + pub fn new(region_server: RegionServer, procedure_manager: ProcedureManagerRef) -> Self { + Self { + region_server, + procedure_manager, + start_time_ms: common_time::util::current_time_millis() as u64, + } + } +} + +#[async_trait::async_trait] +impl InformationExtension for StandaloneInformationExtension { + type Error = catalog::error::Error; + + async fn nodes(&self) -> std::result::Result, Self::Error> { + let build_info = common_version::build_info(); + let node_info = NodeInfo { + // For the standalone: + // - id always 0 + // - empty string for peer_addr + peer: Peer { + id: 0, + addr: "".to_string(), + }, + last_activity_ts: -1, + status: NodeStatus::Standalone, + version: build_info.version.to_string(), + git_commit: build_info.commit_short.to_string(), + // Use `self.start_time_ms` instead. + // It's not precise but enough. + start_time_ms: self.start_time_ms, + }; + Ok(vec![node_info]) + } + + async fn procedures(&self) -> std::result::Result, Self::Error> { + self.procedure_manager + .list_procedures() + .await + .map_err(BoxedError::new) + .map(|procedures| { + procedures + .into_iter() + .map(|procedure| { + let status = procedure.state.as_str_name().to_string(); + (status, procedure) + }) + .collect::>() + }) + .context(catalog::error::ListProceduresSnafu) + } + + async fn region_stats(&self) -> std::result::Result, Self::Error> { + let stats = self + .region_server + .reportable_regions() + .into_iter() + .map(|stat| { + let region_stat = self + .region_server + .region_statistic(stat.region_id) + .unwrap_or_default(); + RegionStat { + id: stat.region_id, + rcus: 0, + wcus: 0, + approximate_bytes: region_stat.estimated_disk_size() as i64, + engine: stat.engine, + role: RegionRole::from(stat.role).into(), + memtable_size: region_stat.memtable_size, + manifest_size: region_stat.manifest_size, + sst_size: region_stat.sst_size, + } + }) + .collect::>(); + Ok(stats) + } +} + #[cfg(test)] mod tests { use std::default::Default; diff --git a/tests-integration/src/cluster.rs b/tests-integration/src/cluster.rs index ad2c3e369f..44e5946769 100644 --- a/tests-integration/src/cluster.rs +++ b/tests-integration/src/cluster.rs @@ -23,6 +23,7 @@ use cache::{build_fundamental_cache_registry, with_default_composite_cache_regis use catalog::kvbackend::{CachedMetaKvBackendBuilder, KvBackendCatalogManager, MetaKvBackend}; use client::client_manager::NodeClients; use client::Client; +use cmd::DistributedInformationExtension; use common_base::Plugins; use common_grpc::channel_manager::{ChannelConfig, ChannelManager}; use common_meta::cache::{CacheRegistryBuilder, LayeredCacheRegistryBuilder}; @@ -366,9 +367,10 @@ impl GreptimeDbClusterBuilder { .build(), ); + let information_extension = + Arc::new(DistributedInformationExtension::new(meta_client.clone())); let catalog_manager = KvBackendCatalogManager::new( - Mode::Distributed, - Some(meta_client.clone()), + information_extension, cached_meta_backend.clone(), cache_registry.clone(), None, diff --git a/tests-integration/src/standalone.rs b/tests-integration/src/standalone.rs index 123614e436..47095ecc65 100644 --- a/tests-integration/src/standalone.rs +++ b/tests-integration/src/standalone.rs @@ -15,6 +15,7 @@ use std::sync::Arc; use cache::{build_fundamental_cache_registry, with_default_composite_cache_registry}; +use catalog::information_schema::NoopInformationExtension; use catalog::kvbackend::KvBackendCatalogManager; use cmd::error::StartFlownodeSnafu; use cmd::standalone::StandaloneOptions; @@ -146,8 +147,7 @@ impl GreptimeDbStandaloneBuilder { ); let catalog_manager = KvBackendCatalogManager::new( - Mode::Standalone, - None, + Arc::new(NoopInformationExtension), kv_backend.clone(), cache_registry.clone(), Some(procedure_manager.clone()), From d168bde22622f8f7ac90ca8e0a31a05cc03e5aae Mon Sep 17 00:00:00 2001 From: Yingwen Date: Fri, 11 Oct 2024 12:16:37 +0800 Subject: [PATCH 065/128] feat!: move v1/prof API to debug/prof (#4810) * feat!: move v1/prof to debug/prof * docs: update readme * docs: move prof docs to docs dir * chore: update message * feat!: remove v1/prof * docs: update mem prof docs --- .../how-to/how-to-profile-cpu.md | 6 ++--- .../how-to/how-to-profile-memory.md | 13 ++++++---- src/servers/src/http.rs | 25 ++++++++++--------- 3 files changed, 24 insertions(+), 20 deletions(-) rename src/servers/src/http/pprof/README.md => docs/how-to/how-to-profile-cpu.md (71%) rename src/common/mem-prof/README.md => docs/how-to/how-to-profile-memory.md (67%) diff --git a/src/servers/src/http/pprof/README.md b/docs/how-to/how-to-profile-cpu.md similarity index 71% rename from src/servers/src/http/pprof/README.md rename to docs/how-to/how-to-profile-cpu.md index b8393ae3f5..b73c85ea2f 100644 --- a/src/servers/src/http/pprof/README.md +++ b/docs/how-to/how-to-profile-cpu.md @@ -9,7 +9,7 @@ cargo build --features=pprof ## HTTP API Sample at 99 Hertz, for 5 seconds, output report in [protobuf format](https://github.com/google/pprof/blob/master/proto/profile.proto). ```bash -curl -s '0:4000/v1/prof/cpu' > /tmp/pprof.out +curl -s '0:4000/debug/prof/cpu' > /tmp/pprof.out ``` Then you can use `pprof` command with the protobuf file. @@ -19,10 +19,10 @@ go tool pprof -top /tmp/pprof.out Sample at 99 Hertz, for 60 seconds, output report in flamegraph format. ```bash -curl -s '0:4000/v1/prof/cpu?seconds=60&output=flamegraph' > /tmp/pprof.svg +curl -s '0:4000/debug/prof/cpu?seconds=60&output=flamegraph' > /tmp/pprof.svg ``` Sample at 49 Hertz, for 10 seconds, output report in text format. ```bash -curl -s '0:4000/v1/prof/cpu?seconds=10&frequency=49&output=text' > /tmp/pprof.txt +curl -s '0:4000/debug/prof/cpu?seconds=10&frequency=49&output=text' > /tmp/pprof.txt ``` diff --git a/src/common/mem-prof/README.md b/docs/how-to/how-to-profile-memory.md similarity index 67% rename from src/common/mem-prof/README.md rename to docs/how-to/how-to-profile-memory.md index da1bbae9f0..7211683190 100644 --- a/src/common/mem-prof/README.md +++ b/docs/how-to/how-to-profile-memory.md @@ -12,10 +12,10 @@ brew install jemalloc sudo apt install libjemalloc-dev ``` -### [flamegraph](https://github.com/brendangregg/FlameGraph) +### [flamegraph](https://github.com/brendangregg/FlameGraph) ```bash -curl https://raw.githubusercontent.com/brendangregg/FlameGraph/master/flamegraph.pl > ./flamegraph.pl +curl https://raw.githubusercontent.com/brendangregg/FlameGraph/master/flamegraph.pl > ./flamegraph.pl ``` ### Build GreptimeDB with `mem-prof` feature. @@ -35,7 +35,7 @@ MALLOC_CONF=prof:true,lg_prof_interval:28 ./target/debug/greptime standalone sta Dump memory profiling data through HTTP API: ```bash -curl localhost:4000/v1/prof/mem > greptime.hprof +curl localhost:4000/debug/prof/mem > greptime.hprof ``` You can periodically dump profiling data and compare them to find the delta memory usage. @@ -45,6 +45,9 @@ You can periodically dump profiling data and compare them to find the delta memo To create flamegraph according to dumped profiling data: ```bash -jeprof --svg --base= > output.svg -``` +sudo apt install -y libjemalloc-dev +jeprof --collapse | ./flamegraph.pl > mem-prof.svg + +jeprof --base --collapse | ./flamegraph.pl > output.svg +``` diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 953ff9e73a..67309b7244 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -730,6 +730,7 @@ impl HttpServer { authorize::check_http_auth, )), ) + // Handlers for debug, we don't expect a timeout. .nest( "/debug", Router::new() @@ -737,19 +738,19 @@ impl HttpServer { .route( "/log_level", routing::get(dyn_log::dyn_log_handler).post(dyn_log::dyn_log_handler), - ), - ) - // Handlers for debug, we don't expect a timeout. - .nest( - &format!("/{HTTP_API_VERSION}/prof"), - Router::new() - .route( - "/cpu", - routing::get(pprof::pprof_handler).post(pprof::pprof_handler), ) - .route( - "/mem", - routing::get(mem_prof::mem_prof_handler).post(mem_prof::mem_prof_handler), + .nest( + "/prof", + Router::new() + .route( + "/cpu", + routing::get(pprof::pprof_handler).post(pprof::pprof_handler), + ) + .route( + "/mem", + routing::get(mem_prof::mem_prof_handler) + .post(mem_prof::mem_prof_handler), + ), ), ) } From 7dd0e3ab37264d11a7deec619ecb3845eb249705 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Fri, 11 Oct 2024 16:23:01 +0800 Subject: [PATCH 066/128] fix: Panic in UNION ALL queries (#4796) * fix/union_all_panic: Improve MetricCollector by incrementing level and fix underflow issue; add tests for UNION ALL queries * chore: remove useless documentation * fix/union_all_panic: Add order by clause to UNION ALL select queries in tests --- src/common/recordbatch/src/adapter.rs | 4 ++-- tests/cases/standalone/common/select/union_all.result | 10 ++++++++++ tests/cases/standalone/common/select/union_all.sql | 1 + 3 files changed, 13 insertions(+), 2 deletions(-) create mode 100644 tests/cases/standalone/common/select/union_all.result create mode 100644 tests/cases/standalone/common/select/union_all.sql diff --git a/src/common/recordbatch/src/adapter.rs b/src/common/recordbatch/src/adapter.rs index 85236381b2..98f4d0a38a 100644 --- a/src/common/recordbatch/src/adapter.rs +++ b/src/common/recordbatch/src/adapter.rs @@ -329,6 +329,7 @@ impl ExecutionPlanVisitor for MetricCollector { level: self.current_level, metrics: vec![], }); + self.current_level += 1; return Ok(true); }; @@ -365,8 +366,7 @@ impl ExecutionPlanVisitor for MetricCollector { } fn post_visit(&mut self, _plan: &dyn ExecutionPlan) -> std::result::Result { - // the last minus will underflow - self.current_level = self.current_level.wrapping_sub(1); + self.current_level -= 1; Ok(true) } } diff --git a/tests/cases/standalone/common/select/union_all.result b/tests/cases/standalone/common/select/union_all.result new file mode 100644 index 0000000000..54332e1254 --- /dev/null +++ b/tests/cases/standalone/common/select/union_all.result @@ -0,0 +1,10 @@ +SELECT 123 as a, 'h' as b UNION ALL SELECT 456 as a, 'e' as b UNION ALL SELECT 789 as a, 'l' as b order by a; + ++-----+---+ +| a | b | ++-----+---+ +| 123 | h | +| 456 | e | +| 789 | l | ++-----+---+ + diff --git a/tests/cases/standalone/common/select/union_all.sql b/tests/cases/standalone/common/select/union_all.sql new file mode 100644 index 0000000000..207d8463fc --- /dev/null +++ b/tests/cases/standalone/common/select/union_all.sql @@ -0,0 +1 @@ +SELECT 123 as a, 'h' as b UNION ALL SELECT 456 as a, 'e' as b UNION ALL SELECT 789 as a, 'l' as b order by a; From a61c0bd1d8a16b208266bbac92f3b8fe2cbdf594 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Fri, 11 Oct 2024 18:02:45 +0800 Subject: [PATCH 067/128] fix: error in admin function is not formatted properly (#4820) Signed-off-by: Ruihang Xia --- src/operator/src/statement/admin.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/operator/src/statement/admin.rs b/src/operator/src/statement/admin.rs index 37e6acf966..6c0413c2aa 100644 --- a/src/operator/src/statement/admin.rs +++ b/src/operator/src/statement/admin.rs @@ -59,7 +59,7 @@ impl StatementExecutor { .map(|arg| { let FunctionArg::Unnamed(FunctionArgExpr::Expr(Expr::Value(value))) = arg else { return error::BuildAdminFunctionArgsSnafu { - msg: "unsupported function arg {arg}", + msg: format!("unsupported function arg {arg}"), } .fail(); }; @@ -200,7 +200,7 @@ fn values_to_vectors_by_valid_types( } error::BuildAdminFunctionArgsSnafu { - msg: "failed to cast {value}", + msg: format!("failed to cast {value}"), } .fail() }) From 0f907ef99e5d2cc2a8ea40ea72b564173ad3a0d2 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 11 Oct 2024 19:32:15 +0800 Subject: [PATCH 068/128] fix: correct table name formatting (#4819) --- src/common/meta/src/ddl/utils.rs | 9 +++++---- src/common/meta/src/key/table_name.rs | 11 +++++++++++ 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/common/meta/src/ddl/utils.rs b/src/common/meta/src/ddl/utils.rs index fd1737a3e6..b1608d40e3 100644 --- a/src/common/meta/src/ddl/utils.rs +++ b/src/common/meta/src/ddl/utils.rs @@ -18,6 +18,7 @@ use common_procedure::error::Error as ProcedureError; use snafu::{ensure, OptionExt, ResultExt}; use store_api::metric_engine_consts::LOGICAL_TABLE_METADATA_KEY; use table::metadata::TableId; +use table::table_reference::TableReference; use crate::ddl::DetectingRegion; use crate::error::{Error, OperateDatanodeSnafu, Result, TableNotFoundSnafu, UnsupportedSnafu}; @@ -109,8 +110,8 @@ pub async fn check_and_get_physical_table_id( .table_name_manager() .get(physical_table_name) .await? - .context(TableNotFoundSnafu { - table_name: physical_table_name.to_string(), + .with_context(|| TableNotFoundSnafu { + table_name: TableReference::from(physical_table_name).to_string(), }) .map(|table| table.table_id()) } @@ -123,8 +124,8 @@ pub async fn get_physical_table_id( .table_name_manager() .get(logical_table_name) .await? - .context(TableNotFoundSnafu { - table_name: logical_table_name.to_string(), + .with_context(|| TableNotFoundSnafu { + table_name: TableReference::from(logical_table_name).to_string(), }) .map(|table| table.table_id())?; diff --git a/src/common/meta/src/key/table_name.rs b/src/common/meta/src/key/table_name.rs index a632e3a233..e508c5e877 100644 --- a/src/common/meta/src/key/table_name.rs +++ b/src/common/meta/src/key/table_name.rs @@ -21,6 +21,7 @@ use serde::{Deserialize, Serialize}; use snafu::OptionExt; use table::metadata::TableId; use table::table_name::TableName; +use table::table_reference::TableReference; use super::{MetadataKey, MetadataValue, TABLE_NAME_KEY_PATTERN, TABLE_NAME_KEY_PREFIX}; use crate::error::{Error, InvalidMetadataSnafu, Result}; @@ -122,6 +123,16 @@ impl From> for TableName { } } +impl<'a> From> for TableReference<'a> { + fn from(value: TableNameKey<'a>) -> Self { + Self { + catalog: value.catalog, + schema: value.schema, + table: value.table, + } + } +} + impl<'a> TryFrom<&'a str> for TableNameKey<'a> { type Error = Error; From 4bb1f4f18447685fd1536e4fddafb0c9a6bb116e Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 11 Oct 2024 20:48:53 +0800 Subject: [PATCH 069/128] feat: introduce `LeadershipChangeNotifier` and `LeadershipChangeListener` (#4817) * feat: introduce `LeadershipChangeNotifier` * refactor: use `LeadershipChangeNotifier` * chore: apply suggestions from CR * chore: apply suggestions from CR * chore: adjust log styling --- src/common/meta/src/error.rs | 18 ++- src/common/meta/src/leadership_notifier.rs | 156 +++++++++++++++++++ src/common/meta/src/lib.rs | 1 + src/common/meta/src/wal_options_allocator.rs | 17 ++ src/meta-srv/src/lib.rs | 1 - src/meta-srv/src/metasrv.rs | 62 ++++---- src/meta-srv/src/procedure.rs | 30 ++++ src/meta-srv/src/region/supervisor.rs | 19 +++ 8 files changed, 270 insertions(+), 34 deletions(-) create mode 100644 src/common/meta/src/leadership_notifier.rs diff --git a/src/common/meta/src/error.rs b/src/common/meta/src/error.rs index 849ee28948..0e7709df0b 100644 --- a/src/common/meta/src/error.rs +++ b/src/common/meta/src/error.rs @@ -147,6 +147,20 @@ pub enum Error { source: common_procedure::Error, }, + #[snafu(display("Failed to start procedure manager"))] + StartProcedureManager { + #[snafu(implicit)] + location: Location, + source: common_procedure::Error, + }, + + #[snafu(display("Failed to stop procedure manager"))] + StopProcedureManager { + #[snafu(implicit)] + location: Location, + source: common_procedure::Error, + }, + #[snafu(display( "Failed to get procedure output, procedure id: {procedure_id}, error: {err_msg}" ))] @@ -715,7 +729,9 @@ impl ErrorExt for Error { SubmitProcedure { source, .. } | QueryProcedure { source, .. } - | WaitProcedure { source, .. } => source.status_code(), + | WaitProcedure { source, .. } + | StartProcedureManager { source, .. } + | StopProcedureManager { source, .. } => source.status_code(), RegisterProcedureLoader { source, .. } => source.status_code(), External { source, .. } => source.status_code(), OperateDatanode { source, .. } => source.status_code(), diff --git a/src/common/meta/src/leadership_notifier.rs b/src/common/meta/src/leadership_notifier.rs new file mode 100644 index 0000000000..4ba65d1adc --- /dev/null +++ b/src/common/meta/src/leadership_notifier.rs @@ -0,0 +1,156 @@ +// 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::sync::Arc; + +use async_trait::async_trait; +use common_telemetry::error; + +use crate::error::Result; + +pub type LeadershipChangeNotifierCustomizerRef = Arc; + +/// A trait for customizing the leadership change notifier. +pub trait LeadershipChangeNotifierCustomizer: Send + Sync { + fn customize(&self, notifier: &mut LeadershipChangeNotifier); +} + +/// A trait for handling leadership change events in a distributed system. +#[async_trait] +pub trait LeadershipChangeListener: Send + Sync { + /// Returns the listener name. + fn name(&self) -> &str; + + /// Called when the node transitions to the leader role. + async fn on_leader_start(&self) -> Result<()>; + + /// Called when the node transitions to the follower role. + async fn on_leader_stop(&self) -> Result<()>; +} + +/// A notifier for leadership change events. +#[derive(Default)] +pub struct LeadershipChangeNotifier { + listeners: Vec>, +} + +impl LeadershipChangeNotifier { + /// Adds a listener to the notifier. + pub fn add_listener(&mut self, listener: Arc) { + self.listeners.push(listener); + } + + /// Notify all listeners that the node has become a leader. + pub async fn notify_on_leader_start(&self) { + for listener in &self.listeners { + if let Err(err) = listener.on_leader_start().await { + error!( + err; + "Failed to notify listener: {}, event 'on_leader_start'", + listener.name() + ); + } + } + } + + /// Notify all listeners that the node has become a follower. + pub async fn notify_on_leader_stop(&self) { + for listener in &self.listeners { + if let Err(err) = listener.on_leader_stop().await { + error!( + err; + "Failed to notify listener: {}, event: 'on_follower_start'", + listener.name() + ); + } + } + } +} + +#[cfg(test)] +mod tests { + use std::sync::atomic::{AtomicBool, Ordering}; + use std::sync::Arc; + + use super::*; + + struct MockListener { + name: String, + on_leader_start_fn: Option Result<()> + Send + Sync>>, + on_follower_start_fn: Option Result<()> + Send + Sync>>, + } + + #[async_trait::async_trait] + impl LeadershipChangeListener for MockListener { + fn name(&self) -> &str { + &self.name + } + + async fn on_leader_start(&self) -> Result<()> { + if let Some(f) = &self.on_leader_start_fn { + return f(); + } + Ok(()) + } + + async fn on_leader_stop(&self) -> Result<()> { + if let Some(f) = &self.on_follower_start_fn { + return f(); + } + Ok(()) + } + } + + #[tokio::test] + async fn test_leadership_change_notifier() { + let mut notifier = LeadershipChangeNotifier::default(); + let listener1 = Arc::new(MockListener { + name: "listener1".to_string(), + on_leader_start_fn: None, + on_follower_start_fn: None, + }); + let called_on_leader_start = Arc::new(AtomicBool::new(false)); + let called_on_follower_start = Arc::new(AtomicBool::new(false)); + let called_on_leader_start_moved = called_on_leader_start.clone(); + let called_on_follower_start_moved = called_on_follower_start.clone(); + let listener2 = Arc::new(MockListener { + name: "listener2".to_string(), + on_leader_start_fn: Some(Box::new(move || { + called_on_leader_start_moved.store(true, Ordering::Relaxed); + Ok(()) + })), + on_follower_start_fn: Some(Box::new(move || { + called_on_follower_start_moved.store(true, Ordering::Relaxed); + Ok(()) + })), + }); + + notifier.add_listener(listener1); + notifier.add_listener(listener2); + + let listener1 = notifier.listeners.first().unwrap(); + let listener2 = notifier.listeners.get(1).unwrap(); + + assert_eq!(listener1.name(), "listener1"); + assert_eq!(listener2.name(), "listener2"); + + notifier.notify_on_leader_start().await; + assert!(!called_on_follower_start.load(Ordering::Relaxed)); + assert!(called_on_leader_start.load(Ordering::Relaxed)); + + notifier.notify_on_leader_stop().await; + assert!(called_on_follower_start.load(Ordering::Relaxed)); + assert!(called_on_leader_start.load(Ordering::Relaxed)); + } +} diff --git a/src/common/meta/src/lib.rs b/src/common/meta/src/lib.rs index 1dd658890c..158350bc32 100644 --- a/src/common/meta/src/lib.rs +++ b/src/common/meta/src/lib.rs @@ -32,6 +32,7 @@ pub mod heartbeat; pub mod instruction; pub mod key; pub mod kv_backend; +pub mod leadership_notifier; pub mod lock_key; pub mod metrics; pub mod node_manager; diff --git a/src/common/meta/src/wal_options_allocator.rs b/src/common/meta/src/wal_options_allocator.rs index ba0c6f407f..283f43b9a8 100644 --- a/src/common/meta/src/wal_options_allocator.rs +++ b/src/common/meta/src/wal_options_allocator.rs @@ -17,6 +17,7 @@ pub mod kafka; use std::collections::HashMap; use std::sync::Arc; +use async_trait::async_trait; use common_wal::config::MetasrvWalConfig; use common_wal::options::{KafkaWalOptions, WalOptions, WAL_OPTIONS_KEY}; use snafu::ResultExt; @@ -24,6 +25,7 @@ use store_api::storage::{RegionId, RegionNumber}; use crate::error::{EncodeWalOptionsSnafu, Result}; use crate::kv_backend::KvBackendRef; +use crate::leadership_notifier::LeadershipChangeListener; use crate::wal_options_allocator::kafka::topic_manager::TopicManager as KafkaTopicManager; /// Allocates wal options in region granularity. @@ -94,6 +96,21 @@ impl WalOptionsAllocator { } } +#[async_trait] +impl LeadershipChangeListener for WalOptionsAllocator { + fn name(&self) -> &str { + "WalOptionsAllocator" + } + + async fn on_leader_start(&self) -> Result<()> { + self.start().await + } + + async fn on_leader_stop(&self) -> Result<()> { + Ok(()) + } +} + /// Allocates a wal options for each region. The allocated wal options is encoded immediately. pub fn allocate_region_wal_options( regions: Vec, diff --git a/src/meta-srv/src/lib.rs b/src/meta-srv/src/lib.rs index 717edc92b2..01b48f1da0 100644 --- a/src/meta-srv/src/lib.rs +++ b/src/meta-srv/src/lib.rs @@ -40,7 +40,6 @@ pub mod selector; pub mod service; pub mod state; pub mod table_meta_alloc; - pub use crate::error::Result; mod greptimedb_telemetry; diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index de7d54aa65..3cdb64e1e0 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -29,6 +29,9 @@ use common_meta::cache_invalidator::CacheInvalidatorRef; use common_meta::ddl::ProcedureExecutorRef; use common_meta::key::TableMetadataManagerRef; use common_meta::kv_backend::{KvBackendRef, ResettableKvBackend, ResettableKvBackendRef}; +use common_meta::leadership_notifier::{ + LeadershipChangeNotifier, LeadershipChangeNotifierCustomizerRef, +}; use common_meta::peer::Peer; use common_meta::region_keeper::MemoryRegionKeeperRef; use common_meta::wal_options_allocator::WalOptionsAllocatorRef; @@ -56,6 +59,7 @@ use crate::handler::HeartbeatHandlerGroupRef; use crate::lease::lookup_datanode_peer; use crate::lock::DistLockRef; use crate::procedure::region_migration::manager::RegionMigrationManagerRef; +use crate::procedure::ProcedureManagerListenerAdapter; use crate::pubsub::{PublisherRef, SubscriptionManagerRef}; use crate::region::supervisor::RegionSupervisorTickerRef; use crate::selector::{Selector, SelectorType}; @@ -291,17 +295,15 @@ pub type SelectorRef = Arc>; pub struct MetaStateHandler { - procedure_manager: ProcedureManagerRef, - wal_options_allocator: WalOptionsAllocatorRef, subscribe_manager: Option, greptimedb_telemetry_task: Arc, leader_cached_kv_backend: Arc, - region_supervisor_ticker: Option, + leadership_change_notifier: LeadershipChangeNotifier, state: StateRef, } impl MetaStateHandler { - pub async fn on_become_leader(&self) { + pub async fn on_leader_start(&self) { self.state.write().unwrap().next_state(become_leader(false)); if let Err(e) = self.leader_cached_kv_backend.load().await { @@ -310,33 +312,19 @@ impl MetaStateHandler { self.state.write().unwrap().next_state(become_leader(true)); } - if let Some(ticker) = self.region_supervisor_ticker.as_ref() { - ticker.start(); - } - - if let Err(e) = self.procedure_manager.start().await { - error!(e; "Failed to start procedure manager"); - } - - if let Err(e) = self.wal_options_allocator.start().await { - error!(e; "Failed to start wal options allocator"); - } + self.leadership_change_notifier + .notify_on_leader_start() + .await; self.greptimedb_telemetry_task.should_report(true); } - pub async fn on_become_follower(&self) { + pub async fn on_leader_stop(&self) { self.state.write().unwrap().next_state(become_follower()); - // Stops the procedures. - if let Err(e) = self.procedure_manager.stop().await { - error!(e; "Failed to stop procedure manager"); - } - - if let Some(ticker) = self.region_supervisor_ticker.as_ref() { - // Stops the supervisor ticker. - ticker.stop(); - } + self.leadership_change_notifier + .notify_on_leader_stop() + .await; // Suspends reporting. self.greptimedb_telemetry_task.should_report(false); @@ -410,15 +398,25 @@ impl Metasrv { greptimedb_telemetry_task .start() .context(StartTelemetryTaskSnafu)?; - let region_supervisor_ticker = self.region_supervisor_ticker.clone(); + + // Builds leadership change notifier. + let mut leadership_change_notifier = LeadershipChangeNotifier::default(); + leadership_change_notifier.add_listener(self.wal_options_allocator.clone()); + leadership_change_notifier + .add_listener(Arc::new(ProcedureManagerListenerAdapter(procedure_manager))); + if let Some(region_supervisor_ticker) = &self.region_supervisor_ticker { + leadership_change_notifier.add_listener(region_supervisor_ticker.clone() as _); + } + if let Some(customizer) = self.plugins.get::() { + customizer.customize(&mut leadership_change_notifier); + } + let state_handler = MetaStateHandler { greptimedb_telemetry_task, subscribe_manager, - procedure_manager, - wal_options_allocator: self.wal_options_allocator.clone(), state: self.state.clone(), leader_cached_kv_backend: leader_cached_kv_backend.clone(), - region_supervisor_ticker, + leadership_change_notifier, }; let _handle = common_runtime::spawn_global(async move { loop { @@ -429,12 +427,12 @@ impl Metasrv { info!("Leader's cache has bean cleared on leader change: {msg}"); match msg { LeaderChangeMessage::Elected(_) => { - state_handler.on_become_leader().await; + state_handler.on_leader_start().await; } LeaderChangeMessage::StepDown(leader) => { error!("Leader :{:?} step down", leader); - state_handler.on_become_follower().await; + state_handler.on_leader_stop().await; } } } @@ -448,7 +446,7 @@ impl Metasrv { } } - state_handler.on_become_follower().await; + state_handler.on_leader_stop().await; }); // Register candidate and keep lease in background. diff --git a/src/meta-srv/src/procedure.rs b/src/meta-srv/src/procedure.rs index 1f430654d2..8e696723e8 100644 --- a/src/meta-srv/src/procedure.rs +++ b/src/meta-srv/src/procedure.rs @@ -12,7 +12,37 @@ // See the License for the specific language governing permissions and // limitations under the License. +use async_trait::async_trait; +use common_meta::error::{self, Result}; +use common_meta::leadership_notifier::LeadershipChangeListener; +use common_procedure::ProcedureManagerRef; +use snafu::ResultExt; + pub mod region_migration; #[cfg(test)] mod tests; pub mod utils; + +#[derive(Clone)] +pub struct ProcedureManagerListenerAdapter(pub ProcedureManagerRef); + +#[async_trait] +impl LeadershipChangeListener for ProcedureManagerListenerAdapter { + fn name(&self) -> &str { + "ProcedureManager" + } + + async fn on_leader_start(&self) -> Result<()> { + self.0 + .start() + .await + .context(error::StartProcedureManagerSnafu) + } + + async fn on_leader_stop(&self) -> Result<()> { + self.0 + .stop() + .await + .context(error::StopProcedureManagerSnafu) + } +} diff --git a/src/meta-srv/src/region/supervisor.rs b/src/meta-srv/src/region/supervisor.rs index 80f6c84865..32da666f2b 100644 --- a/src/meta-srv/src/region/supervisor.rs +++ b/src/meta-srv/src/region/supervisor.rs @@ -16,10 +16,12 @@ use std::fmt::Debug; use std::sync::{Arc, Mutex}; use std::time::Duration; +use async_trait::async_trait; use common_meta::datanode::Stat; use common_meta::ddl::{DetectingRegion, RegionFailureDetectorController}; use common_meta::key::MAINTENANCE_KEY; use common_meta::kv_backend::KvBackendRef; +use common_meta::leadership_notifier::LeadershipChangeListener; use common_meta::peer::PeerLookupServiceRef; use common_meta::{ClusterId, DatanodeId}; use common_runtime::JoinHandle; @@ -129,6 +131,23 @@ pub struct RegionSupervisorTicker { sender: Sender, } +#[async_trait] +impl LeadershipChangeListener for RegionSupervisorTicker { + fn name(&self) -> &'static str { + "RegionSupervisorTicker" + } + + async fn on_leader_start(&self) -> common_meta::error::Result<()> { + self.start(); + Ok(()) + } + + async fn on_leader_stop(&self) -> common_meta::error::Result<()> { + self.stop(); + Ok(()) + } +} + impl RegionSupervisorTicker { pub(crate) fn new(tick_interval: Duration, sender: Sender) -> Self { Self { From aaa9b329085de638e5122bf132c239359dba65d6 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Fri, 11 Oct 2024 10:57:54 -0700 Subject: [PATCH 070/128] feat: add more h3 functions (#4770) * feat: add more h3 grid functions * feat: add more traversal functions * refactor: update some function definitions * style: format * refactor: avoid creating slice in nested loop * feat: ensure column number and length * refactor: fix lint warnings * refactor: merge main * Apply suggestions from code review Co-authored-by: LFC <990479+MichaelScofield@users.noreply.github.com> * Update src/common/function/src/scalars/geo/h3.rs Co-authored-by: LFC <990479+MichaelScofield@users.noreply.github.com> * style: format --------- Co-authored-by: LFC <990479+MichaelScofield@users.noreply.github.com> --- src/common/function/src/scalars/geo.rs | 29 +- src/common/function/src/scalars/geo/h3.rs | 728 +++++++++++++----- .../standalone/common/function/geo.result | 59 +- .../cases/standalone/common/function/geo.sql | 22 +- 4 files changed, 595 insertions(+), 243 deletions(-) diff --git a/src/common/function/src/scalars/geo.rs b/src/common/function/src/scalars/geo.rs index 37b6c0704b..e47a1de9f2 100644 --- a/src/common/function/src/scalars/geo.rs +++ b/src/common/function/src/scalars/geo.rs @@ -29,18 +29,31 @@ impl GeoFunctions { // geohash registry.register(Arc::new(GeohashFunction)); registry.register(Arc::new(GeohashNeighboursFunction)); - // h3 family + + // h3 index registry.register(Arc::new(h3::H3LatLngToCell)); registry.register(Arc::new(h3::H3LatLngToCellString)); + + // h3 index inspection registry.register(Arc::new(h3::H3CellBase)); - registry.register(Arc::new(h3::H3CellCenterChild)); - registry.register(Arc::new(h3::H3CellCenterLat)); - registry.register(Arc::new(h3::H3CellCenterLng)); registry.register(Arc::new(h3::H3CellIsPentagon)); - registry.register(Arc::new(h3::H3CellParent)); - registry.register(Arc::new(h3::H3CellResolution)); - registry.register(Arc::new(h3::H3CellToString)); - registry.register(Arc::new(h3::H3IsNeighbour)); registry.register(Arc::new(h3::H3StringToCell)); + registry.register(Arc::new(h3::H3CellToString)); + registry.register(Arc::new(h3::H3CellCenterLatLng)); + registry.register(Arc::new(h3::H3CellResolution)); + + // h3 hierarchical grid + registry.register(Arc::new(h3::H3CellCenterChild)); + registry.register(Arc::new(h3::H3CellParent)); + registry.register(Arc::new(h3::H3CellToChildren)); + registry.register(Arc::new(h3::H3CellToChildrenSize)); + registry.register(Arc::new(h3::H3CellToChildPos)); + registry.register(Arc::new(h3::H3ChildPosToCell)); + + // h3 grid traversal + registry.register(Arc::new(h3::H3GridDisk)); + registry.register(Arc::new(h3::H3GridDiskDistances)); + registry.register(Arc::new(h3::H3GridDistance)); + registry.register(Arc::new(h3::H3GridPathCells)); } } diff --git a/src/common/function/src/scalars/geo/h3.rs b/src/common/function/src/scalars/geo/h3.rs index 672fbfd714..00c567d8d7 100644 --- a/src/common/function/src/scalars/geo/h3.rs +++ b/src/common/function/src/scalars/geo/h3.rs @@ -20,18 +20,71 @@ use common_query::error::{self, InvalidFuncArgsSnafu, Result}; use common_query::prelude::{Signature, TypeSignature}; use datafusion::logical_expr::Volatility; use datatypes::prelude::ConcreteDataType; -use datatypes::scalars::ScalarVectorBuilder; -use datatypes::value::Value; +use datatypes::scalars::{Scalar, ScalarVectorBuilder}; +use datatypes::value::{ListValue, Value}; use datatypes::vectors::{ - BooleanVectorBuilder, Float64VectorBuilder, MutableVector, StringVectorBuilder, - UInt64VectorBuilder, UInt8VectorBuilder, VectorRef, + BooleanVectorBuilder, Int32VectorBuilder, ListVectorBuilder, MutableVector, + StringVectorBuilder, UInt64VectorBuilder, UInt8VectorBuilder, VectorRef, }; use derive_more::Display; use h3o::{CellIndex, LatLng, Resolution}; +use once_cell::sync::Lazy; use snafu::{ensure, ResultExt}; +use super::helpers::{ensure_columns_len, ensure_columns_n}; use crate::function::{Function, FunctionContext}; +static CELL_TYPES: Lazy> = Lazy::new(|| { + vec![ + ConcreteDataType::int64_datatype(), + ConcreteDataType::uint64_datatype(), + ] +}); + +static COORDINATE_TYPES: Lazy> = Lazy::new(|| { + vec![ + ConcreteDataType::float32_datatype(), + ConcreteDataType::float64_datatype(), + ] +}); +static RESOLUTION_TYPES: Lazy> = Lazy::new(|| { + vec![ + ConcreteDataType::int8_datatype(), + ConcreteDataType::int16_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int64_datatype(), + ConcreteDataType::uint8_datatype(), + ConcreteDataType::uint16_datatype(), + ConcreteDataType::uint32_datatype(), + ConcreteDataType::uint64_datatype(), + ] +}); +static DISTANCE_TYPES: Lazy> = Lazy::new(|| { + vec![ + ConcreteDataType::int8_datatype(), + ConcreteDataType::int16_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int64_datatype(), + ConcreteDataType::uint8_datatype(), + ConcreteDataType::uint16_datatype(), + ConcreteDataType::uint32_datatype(), + ConcreteDataType::uint64_datatype(), + ] +}); + +static POSITION_TYPES: Lazy> = Lazy::new(|| { + vec![ + ConcreteDataType::int8_datatype(), + ConcreteDataType::int16_datatype(), + ConcreteDataType::int32_datatype(), + ConcreteDataType::int64_datatype(), + ConcreteDataType::uint8_datatype(), + ConcreteDataType::uint16_datatype(), + ConcreteDataType::uint32_datatype(), + ConcreteDataType::uint64_datatype(), + ] +}); + /// Function that returns [h3] encoding cellid for a given geospatial coordinate. /// /// [h3]: https://h3geo.org/ @@ -50,20 +103,8 @@ impl Function for H3LatLngToCell { fn signature(&self) -> Signature { let mut signatures = Vec::new(); - for coord_type in &[ - ConcreteDataType::float32_datatype(), - ConcreteDataType::float64_datatype(), - ] { - for resolution_type in &[ - ConcreteDataType::int8_datatype(), - ConcreteDataType::int16_datatype(), - ConcreteDataType::int32_datatype(), - ConcreteDataType::int64_datatype(), - ConcreteDataType::uint8_datatype(), - ConcreteDataType::uint16_datatype(), - ConcreteDataType::uint32_datatype(), - ConcreteDataType::uint64_datatype(), - ] { + for coord_type in COORDINATE_TYPES.as_slice() { + for resolution_type in RESOLUTION_TYPES.as_slice() { signatures.push(TypeSignature::Exact(vec![ // latitude coord_type.clone(), @@ -78,15 +119,7 @@ impl Function for H3LatLngToCell { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 3, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 3, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 3); let lat_vec = &columns[0]; let lon_vec = &columns[1]; @@ -142,20 +175,8 @@ impl Function for H3LatLngToCellString { fn signature(&self) -> Signature { let mut signatures = Vec::new(); - for coord_type in &[ - ConcreteDataType::float32_datatype(), - ConcreteDataType::float64_datatype(), - ] { - for resolution_type in &[ - ConcreteDataType::int8_datatype(), - ConcreteDataType::int16_datatype(), - ConcreteDataType::int32_datatype(), - ConcreteDataType::int64_datatype(), - ConcreteDataType::uint8_datatype(), - ConcreteDataType::uint16_datatype(), - ConcreteDataType::uint32_datatype(), - ConcreteDataType::uint64_datatype(), - ] { + for coord_type in COORDINATE_TYPES.as_slice() { + for resolution_type in RESOLUTION_TYPES.as_slice() { signatures.push(TypeSignature::Exact(vec![ // latitude coord_type.clone(), @@ -170,15 +191,7 @@ impl Function for H3LatLngToCellString { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 3, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 3, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 3); let lat_vec = &columns[0]; let lon_vec = &columns[1]; @@ -234,15 +247,7 @@ impl Function for H3CellToString { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 1, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 1, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 1); let cell_vec = &columns[0]; let size = cell_vec.len(); @@ -280,15 +285,7 @@ impl Function for H3StringToCell { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 1, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 1, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 1); let string_vec = &columns[0]; let size = string_vec.len(); @@ -319,18 +316,20 @@ impl Function for H3StringToCell { } } -/// Function that returns centroid latitude of given cell id +/// Function that returns centroid latitude and longitude of given cell id #[derive(Clone, Debug, Default, Display)] #[display("{}", self.name())] -pub struct H3CellCenterLat; +pub struct H3CellCenterLatLng; -impl Function for H3CellCenterLat { +impl Function for H3CellCenterLatLng { fn name(&self) -> &str { - "h3_cell_center_lat" + "h3_cell_center_latlng" } fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { - Ok(ConcreteDataType::float64_datatype()) + Ok(ConcreteDataType::list_datatype( + ConcreteDataType::float64_datatype(), + )) } fn signature(&self) -> Signature { @@ -338,69 +337,26 @@ impl Function for H3CellCenterLat { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 1, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 1, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 1); let cell_vec = &columns[0]; let size = cell_vec.len(); - let mut results = Float64VectorBuilder::with_capacity(size); + let mut results = + ListVectorBuilder::with_type_capacity(ConcreteDataType::float64_datatype(), size); for i in 0..size { let cell = cell_from_value(cell_vec.get(i))?; - let lat = cell.map(|cell| LatLng::from(cell).lat()); + let latlng = cell.map(LatLng::from); - results.push(lat); - } - - Ok(results.to_vector()) - } -} - -/// Function that returns centroid longitude of given cell id -#[derive(Clone, Debug, Default, Display)] -#[display("{}", self.name())] -pub struct H3CellCenterLng; - -impl Function for H3CellCenterLng { - fn name(&self) -> &str { - "h3_cell_center_lng" - } - - fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { - Ok(ConcreteDataType::float64_datatype()) - } - - fn signature(&self) -> Signature { - signature_of_cell() - } - - fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 1, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 1, provided : {}", - columns.len() - ), + if let Some(latlng) = latlng { + let result = ListValue::new( + vec![latlng.lat().into(), latlng.lng().into()], + ConcreteDataType::float64_datatype(), + ); + results.push(Some(result.as_scalar_ref())); + } else { + results.push(None); } - ); - - let cell_vec = &columns[0]; - let size = cell_vec.len(); - let mut results = Float64VectorBuilder::with_capacity(size); - - for i in 0..size { - let cell = cell_from_value(cell_vec.get(i))?; - let lat = cell.map(|cell| LatLng::from(cell).lng()); - - results.push(lat); } Ok(results.to_vector()) @@ -470,15 +426,7 @@ impl Function for H3CellBase { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 1, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 1, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 1); let cell_vec = &columns[0]; let size = cell_vec.len(); @@ -514,15 +462,7 @@ impl Function for H3CellIsPentagon { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 1, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 1, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 1); let cell_vec = &columns[0]; let size = cell_vec.len(); @@ -558,15 +498,7 @@ impl Function for H3CellCenterChild { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 2, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 2, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 2); let cell_vec = &columns[0]; let res_vec = &columns[1]; @@ -606,15 +538,7 @@ impl Function for H3CellParent { } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 2, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 2, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 2); let cell_vec = &columns[0]; let res_vec = &columns[1]; @@ -633,48 +557,323 @@ impl Function for H3CellParent { } } -/// Function that checks if two cells are neighbour +/// Function that returns children cell list #[derive(Clone, Debug, Default, Display)] #[display("{}", self.name())] -pub struct H3IsNeighbour; +pub struct H3CellToChildren; -impl Function for H3IsNeighbour { +impl Function for H3CellToChildren { fn name(&self) -> &str { - "h3_is_neighbour" + "h3_cell_to_children" } fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { - Ok(ConcreteDataType::boolean_datatype()) + Ok(ConcreteDataType::list_datatype( + ConcreteDataType::uint64_datatype(), + )) } fn signature(&self) -> Signature { - signature_of_double_cell() + signature_of_cell_and_resolution() } fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { - ensure!( - columns.len() == 2, - InvalidFuncArgsSnafu { - err_msg: format!( - "The length of the args is not correct, expect 2, provided : {}", - columns.len() - ), - } - ); + ensure_columns_n!(columns, 2); let cell_vec = &columns[0]; - let cell2_vec = &columns[1]; + let res_vec = &columns[1]; let size = cell_vec.len(); - let mut results = BooleanVectorBuilder::with_capacity(size); + let mut results = + ListVectorBuilder::with_type_capacity(ConcreteDataType::uint64_datatype(), size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = value_to_resolution(res_vec.get(i))?; + let result = cell.map(|cell| { + let children: Vec = cell + .children(res) + .map(|child| Value::from(u64::from(child))) + .collect(); + ListValue::new(children, ConcreteDataType::uint64_datatype()) + }); + + if let Some(list_value) = result { + results.push(Some(list_value.as_scalar_ref())); + } else { + results.push(None); + } + } + + Ok(results.to_vector()) + } +} + +/// Function that returns children cell count +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellToChildrenSize; + +impl Function for H3CellToChildrenSize { + fn name(&self) -> &str { + "h3_cell_to_children_size" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint64_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell_and_resolution() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure_columns_n!(columns, 2); + + let cell_vec = &columns[0]; + let res_vec = &columns[1]; + let size = cell_vec.len(); + let mut results = UInt64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = value_to_resolution(res_vec.get(i))?; + let result = cell.map(|cell| cell.children_count(res)); + results.push(result); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns the cell position if its parent at given resolution +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3CellToChildPos; + +impl Function for H3CellToChildPos { + fn name(&self) -> &str { + "h3_cell_to_child_pos" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint64_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_cell_and_resolution() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure_columns_n!(columns, 2); + + let cell_vec = &columns[0]; + let res_vec = &columns[1]; + let size = cell_vec.len(); + let mut results = UInt64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let res = value_to_resolution(res_vec.get(i))?; + let result = cell.and_then(|cell| cell.child_position(res)); + results.push(result); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns the cell at given position of the parent at given resolution +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3ChildPosToCell; + +impl Function for H3ChildPosToCell { + fn name(&self) -> &str { + "h3_child_pos_to_cell" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::uint64_datatype()) + } + + fn signature(&self) -> Signature { + let mut signatures = + Vec::with_capacity(POSITION_TYPES.len() * CELL_TYPES.len() * RESOLUTION_TYPES.len()); + for position_type in POSITION_TYPES.as_slice() { + for cell_type in CELL_TYPES.as_slice() { + for resolution_type in RESOLUTION_TYPES.as_slice() { + signatures.push(TypeSignature::Exact(vec![ + position_type.clone(), + cell_type.clone(), + resolution_type.clone(), + ])); + } + } + } + Signature::one_of(signatures, Volatility::Stable) + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure_columns_n!(columns, 3); + + let pos_vec = &columns[0]; + let cell_vec = &columns[1]; + let res_vec = &columns[2]; + let size = cell_vec.len(); + let mut results = UInt64VectorBuilder::with_capacity(size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let pos = value_to_position(pos_vec.get(i))?; + let res = value_to_resolution(res_vec.get(i))?; + let result = cell.and_then(|cell| cell.child_at(pos, res).map(u64::from)); + results.push(result); + } + + Ok(results.to_vector()) + } +} + +/// Function that returns cells with k distances of given cell +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3GridDisk; + +impl Function for H3GridDisk { + fn name(&self) -> &str { + "h3_grid_disk" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::list_datatype( + ConcreteDataType::uint64_datatype(), + )) + } + + fn signature(&self) -> Signature { + signature_of_cell_and_distance() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure_columns_n!(columns, 2); + + let cell_vec = &columns[0]; + let k_vec = &columns[1]; + let size = cell_vec.len(); + let mut results = + ListVectorBuilder::with_type_capacity(ConcreteDataType::uint64_datatype(), size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let k = value_to_distance(k_vec.get(i))?; + + let result = cell.map(|cell| { + let children: Vec = cell + .grid_disk::>(k) + .into_iter() + .map(|child| Value::from(u64::from(child))) + .collect(); + ListValue::new(children, ConcreteDataType::uint64_datatype()) + }); + + if let Some(list_value) = result { + results.push(Some(list_value.as_scalar_ref())); + } else { + results.push(None); + } + } + + Ok(results.to_vector()) + } +} + +/// Function that returns all cells within k distances of given cell +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3GridDiskDistances; + +impl Function for H3GridDiskDistances { + fn name(&self) -> &str { + "h3_grid_disk_distances" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::list_datatype( + ConcreteDataType::uint64_datatype(), + )) + } + + fn signature(&self) -> Signature { + signature_of_cell_and_distance() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure_columns_n!(columns, 2); + + let cell_vec = &columns[0]; + let k_vec = &columns[1]; + let size = cell_vec.len(); + let mut results = + ListVectorBuilder::with_type_capacity(ConcreteDataType::uint64_datatype(), size); + + for i in 0..size { + let cell = cell_from_value(cell_vec.get(i))?; + let k = value_to_distance(k_vec.get(i))?; + + let result = cell.map(|cell| { + let children: Vec = cell + .grid_disk_distances::>(k) + .into_iter() + .map(|(child, _distance)| Value::from(u64::from(child))) + .collect(); + ListValue::new(children, ConcreteDataType::uint64_datatype()) + }); + + if let Some(list_value) = result { + results.push(Some(list_value.as_scalar_ref())); + } else { + results.push(None); + } + } + + Ok(results.to_vector()) + } +} + +/// Function that returns distance between two cells +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3GridDistance; + +impl Function for H3GridDistance { + fn name(&self) -> &str { + "h3_grid_distance" + } + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::int32_datatype()) + } + + fn signature(&self) -> Signature { + signature_of_double_cells() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure_columns_n!(columns, 2); + + let cell_this_vec = &columns[0]; + let cell_that_vec = &columns[1]; + let size = cell_this_vec.len(); + + let mut results = Int32VectorBuilder::with_capacity(size); for i in 0..size { let result = match ( - cell_from_value(cell_vec.get(i))?, - cell_from_value(cell2_vec.get(i))?, + cell_from_value(cell_this_vec.get(i))?, + cell_from_value(cell_that_vec.get(i))?, ) { (Some(cell_this), Some(cell_that)) => { - let is_neighbour = cell_this - .is_neighbor_with(cell_that) + let dist = cell_this + .grid_distance(cell_that) .map_err(|e| { BoxedError::new(PlainError::new( format!("H3 error: {}", e), @@ -682,7 +881,7 @@ impl Function for H3IsNeighbour { )) }) .context(error::ExecuteSnafu)?; - Some(is_neighbour) + Some(dist) } _ => None, }; @@ -694,6 +893,73 @@ impl Function for H3IsNeighbour { } } +/// Function that returns path cells between two cells +#[derive(Clone, Debug, Default, Display)] +#[display("{}", self.name())] +pub struct H3GridPathCells; + +impl Function for H3GridPathCells { + fn name(&self) -> &str { + "h3_grid_path_cells" + } + + fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result { + Ok(ConcreteDataType::list_datatype( + ConcreteDataType::uint64_datatype(), + )) + } + + fn signature(&self) -> Signature { + signature_of_double_cells() + } + + fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result { + ensure_columns_n!(columns, 2); + + let cell_this_vec = &columns[0]; + let cell_that_vec = &columns[1]; + let size = cell_this_vec.len(); + let mut results = + ListVectorBuilder::with_type_capacity(ConcreteDataType::uint64_datatype(), size); + + for i in 0..size { + let result = match ( + cell_from_value(cell_this_vec.get(i))?, + cell_from_value(cell_that_vec.get(i))?, + ) { + (Some(cell_this), Some(cell_that)) => { + let cells = cell_this + .grid_path_cells(cell_that) + .and_then(|x| x.collect::, _>>()) + .map_err(|e| { + BoxedError::new(PlainError::new( + format!("H3 error: {}", e), + StatusCode::EngineExecuteQuery, + )) + }) + .context(error::ExecuteSnafu)?; + Some(ListValue::new( + cells + .into_iter() + .map(|c| Value::from(u64::from(c))) + .collect(), + ConcreteDataType::uint64_datatype(), + )) + } + _ => None, + }; + + if let Some(list_value) = result { + results.push(Some(list_value.as_scalar_ref())); + } else { + results.push(None); + } + } + + Ok(results.to_vector()) + } +} + fn value_to_resolution(v: Value) -> Result { let r = match v { Value::Int8(v) => v as u8, @@ -716,26 +982,59 @@ fn value_to_resolution(v: Value) -> Result { .context(error::ExecuteSnafu) } +macro_rules! ensure_and_coerce { + ($compare:expr, $coerce:expr) => {{ + ensure!( + $compare, + InvalidFuncArgsSnafu { + err_msg: "Argument was outside of acceptable range " + } + ); + Ok($coerce) + }}; +} + +fn value_to_position(v: Value) -> Result { + match v { + Value::Int8(v) => ensure_and_coerce!(v >= 0, v as u64), + Value::Int16(v) => ensure_and_coerce!(v >= 0, v as u64), + Value::Int32(v) => ensure_and_coerce!(v >= 0, v as u64), + Value::Int64(v) => ensure_and_coerce!(v >= 0, v as u64), + Value::UInt8(v) => Ok(v as u64), + Value::UInt16(v) => Ok(v as u64), + Value::UInt32(v) => Ok(v as u64), + Value::UInt64(v) => Ok(v), + _ => unreachable!(), + } +} + +fn value_to_distance(v: Value) -> Result { + match v { + Value::Int8(v) => ensure_and_coerce!(v >= 0, v as u32), + Value::Int16(v) => ensure_and_coerce!(v >= 0, v as u32), + Value::Int32(v) => ensure_and_coerce!(v >= 0, v as u32), + Value::Int64(v) => ensure_and_coerce!(v >= 0, v as u32), + Value::UInt8(v) => Ok(v as u32), + Value::UInt16(v) => Ok(v as u32), + Value::UInt32(v) => Ok(v), + Value::UInt64(v) => Ok(v as u32), + _ => unreachable!(), + } +} + fn signature_of_cell() -> Signature { - let mut signatures = Vec::new(); - for cell_type in &[ - ConcreteDataType::uint64_datatype(), - ConcreteDataType::int64_datatype(), - ] { + let mut signatures = Vec::with_capacity(CELL_TYPES.len()); + for cell_type in CELL_TYPES.as_slice() { signatures.push(TypeSignature::Exact(vec![cell_type.clone()])); } Signature::one_of(signatures, Volatility::Stable) } -fn signature_of_double_cell() -> Signature { - let mut signatures = Vec::new(); - let cell_types = &[ - ConcreteDataType::uint64_datatype(), - ConcreteDataType::int64_datatype(), - ]; - for cell_type in cell_types { - for cell_type2 in cell_types { +fn signature_of_double_cells() -> Signature { + let mut signatures = Vec::with_capacity(CELL_TYPES.len() * CELL_TYPES.len()); + for cell_type in CELL_TYPES.as_slice() { + for cell_type2 in CELL_TYPES.as_slice() { signatures.push(TypeSignature::Exact(vec![ cell_type.clone(), cell_type2.clone(), @@ -747,21 +1046,9 @@ fn signature_of_double_cell() -> Signature { } fn signature_of_cell_and_resolution() -> Signature { - let mut signatures = Vec::new(); - for cell_type in &[ - ConcreteDataType::uint64_datatype(), - ConcreteDataType::int64_datatype(), - ] { - for resolution_type in &[ - ConcreteDataType::int8_datatype(), - ConcreteDataType::int16_datatype(), - ConcreteDataType::int32_datatype(), - ConcreteDataType::int64_datatype(), - ConcreteDataType::uint8_datatype(), - ConcreteDataType::uint16_datatype(), - ConcreteDataType::uint32_datatype(), - ConcreteDataType::uint64_datatype(), - ] { + let mut signatures = Vec::with_capacity(CELL_TYPES.len() * RESOLUTION_TYPES.len()); + for cell_type in CELL_TYPES.as_slice() { + for resolution_type in RESOLUTION_TYPES.as_slice() { signatures.push(TypeSignature::Exact(vec![ cell_type.clone(), resolution_type.clone(), @@ -771,6 +1058,19 @@ fn signature_of_cell_and_resolution() -> Signature { Signature::one_of(signatures, Volatility::Stable) } +fn signature_of_cell_and_distance() -> Signature { + let mut signatures = Vec::with_capacity(CELL_TYPES.len() * DISTANCE_TYPES.len()); + for cell_type in CELL_TYPES.as_slice() { + for distance_type in DISTANCE_TYPES.as_slice() { + signatures.push(TypeSignature::Exact(vec![ + cell_type.clone(), + distance_type.clone(), + ])); + } + } + Signature::one_of(signatures, Volatility::Stable) +} + fn cell_from_value(v: Value) -> Result> { let cell = match v { Value::Int64(v) => Some( diff --git a/tests/cases/standalone/common/function/geo.result b/tests/cases/standalone/common/function/geo.result index 4f9d168ac0..75caeb886b 100644 --- a/tests/cases/standalone/common/function/geo.result +++ b/tests/cases/standalone/common/function/geo.result @@ -102,35 +102,58 @@ SELECT h3_cell_to_string(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS c | 88283082e7fffff | 613196570438926335 | +-----------------+--------------------+ -SELECT h3_cell_center_lat(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lat, h3_cell_center_lng(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lng; +SELECT h3_cell_center_latlng(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center; -+-------------------+---------------------+ -| cell_center_lat | cell_center_lng | -+-------------------+---------------------+ -| 37.77246152245501 | -122.39010997087324 | -+-------------------+---------------------+ ++------------------------------------------+ +| cell_center | ++------------------------------------------+ +| [37.77246152245501, -122.39010997087324] | ++------------------------------------------+ SELECT h3_cell_resolution(cell) AS resolution, h3_cell_base(cell) AS base, h3_cell_is_pentagon(cell) AS pentagon, h3_cell_parent(cell, 6::UInt64) AS parent, + h3_cell_to_children(cell, 10::UInt64) AS children, + h3_cell_to_children_size(cell, 10) AS children_count, + h3_cell_to_child_pos(cell, 6) AS child_pos, + h3_child_pos_to_cell(25, cell, 11) AS child FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell); -+------------+------+----------+--------------------+ -| resolution | base | pentagon | parent | -+------------+------+----------+--------------------+ -| 8 | 20 | false | 604189371209351167 | -+------------+------+----------+--------------------+ ++------------+------+----------+--------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------+-----------+--------------------+ +| resolution | base | pentagon | parent | children | children_count | child_pos | child | ++------------+------+----------+--------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------+-----------+--------------------+ +| 8 | 20 | false | 604189371209351167 | [622203769691602943, 622203769691635711, 622203769691668479, 622203769691701247, 622203769691734015, 622203769691766783, 622203769691799551, 622203769691865087, 622203769691897855, 622203769691930623, 622203769691963391, 622203769691996159, 622203769692028927, 622203769692061695, 622203769692127231, 622203769692159999, 622203769692192767, 622203769692225535, 622203769692258303, 622203769692291071, 622203769692323839, 622203769692389375, 622203769692422143, 622203769692454911, 622203769692487679, 622203769692520447, 622203769692553215, 622203769692585983, 622203769692651519, 622203769692684287, 622203769692717055, 622203769692749823, 622203769692782591, 622203769692815359, 622203769692848127, 622203769692913663, 622203769692946431, 622203769692979199, 622203769693011967, 622203769693044735, 622203769693077503, 622203769693110271, 622203769693175807, 622203769693208575, 622203769693241343, 622203769693274111, 622203769693306879, 622203769693339647, 622203769693372415] | 49 | 45 | 626707369319059455 | ++------------+------+----------+--------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------+-----------+--------------------+ -SELECT h3_is_neighbour(cell1, cell2) -FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell1, h3_latlng_to_cell(36.76938, -122.3889, 8::UInt64) AS cell2); +SELECT + h3_grid_disk(cell, 0) AS current_cell, + h3_grid_disk(cell, 3) AS grids, + h3_grid_disk_distances(cell, 3) AS all_grids, +FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell); -+------------------------------+ -| h3_is_neighbour(cell1,cell2) | -+------------------------------+ -| false | -+------------------------------+ ++----------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| current_cell | grids | all_grids | ++----------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| [613196570438926335] | [613196570438926335, 613196570436829183, 613196569755254783, 613196570378108927, 613196570373914623, 613196570434732031, 613196570432634879, 613196570445217791, 613196570250182655, 613196569753157631, 613196569744769023, 613196569746866175, 613196570369720319, 613196570365526015, 613196570376011775, 613196570336165887, 613196570344554495, 613196570443120639, 613196570441023487, 613196570220822527, 613196570258571263, 613196570248085503, 613196570254376959, 613196569757351935, 613196569748963327, 613196569751060479, 613196569686048767, 613196569688145919, 613196570371817471, 613196570367623167, 613196570394886143, 613196570338263039, 613196570331971583, 613196570340360191, 613196570405371903, 613196570403274751, 613196570216628223] | [613196570438926335, 613196570436829183, 613196569755254783, 613196570378108927, 613196570373914623, 613196570434732031, 613196570432634879, 613196570445217791, 613196570250182655, 613196569753157631, 613196569744769023, 613196569746866175, 613196570369720319, 613196570365526015, 613196570376011775, 613196570336165887, 613196570344554495, 613196570443120639, 613196570441023487, 613196570220822527, 613196570258571263, 613196570248085503, 613196570254376959, 613196569757351935, 613196569748963327, 613196569751060479, 613196569686048767, 613196569688145919, 613196570371817471, 613196570367623167, 613196570394886143, 613196570338263039, 613196570331971583, 613196570340360191, 613196570405371903, 613196570403274751, 613196570216628223] | ++----------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +SELECT + h3_grid_distance(cell1, cell2) AS distance, + h3_grid_path_cells(cell1, cell2) AS path_cells, +FROM + ( + SELECT + h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell1, + h3_latlng_to_cell(39.634, -104.999, 8::UInt64) AS cell2 + ); + ++----------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| distance | path_cells | ++----------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| 1612 | [613196570438926335, 613196569755254783, 613196569744769023, 613196569748963327, 613196569669271551, 613196569673465855, 613196569763643391, 613196569767837695, 613196570023690239, 613196570021593087, 613196570025787391, 613196569998524415, 613196570002718719, 613196570040467455, 613196570029981695, 613196570034175999, 613196572437512191, 613196572441706495, 613196572414443519, 613196572418637823, 613196572456386559, 613196572445900799, 613196572450095103, 613196572705947647, 613196572710141951, 613196572691267583, 613196572680781823, 613196572684976127, 613196572722724863, 613196572726919167, 613196592932978687, 613196592937172991, 613196592421273599, 613196592410787839, 613196592414982143, 613196592452730879, 613196592456925183, 613196592261890047, 613196592266084351, 613196592689709055, 613196592679223295, 613196592683417599, 613196592496771071, 613196592500965375, 613196592538714111, 613196592528228351, 613196592532422655, 613196587587338239, 613196587591532543, 613196587396497407, 613196587400691711, 613196587438440447, 613196587427954687, 613196587432148991, 613196586916249599, 613196586920443903, 613196587664932863, 613196587669127167, 613196587706875903, 613196587704778751, 613196587708973055, 613196587681710079, 613196587685904383, 613196593444683775, 613196593434198015, 613196593438392319, 613196593476141055, 613196593480335359, 613196593453072383, 613196593457266687, 613196593713119231, 613196593702633471, 613196593706827775, 613196593744576511, 613196593748770815, 613196593729896447, 613196593719410687, 613196593723604991, 613196962272903167, 613196962277097471, 613196962249834495, 613196962254028799, 613196962291777535, 613196962281291775, 613196962285486079, 613196961601814527, 613196961606008831, 613196961578745855, 613196961582940159, 613196961620688895, 613196961610203135, 613196961614397439, 613196961931067391, 613196961935261695, 613196961855569919, 613196961845084159, 613196961849278463, 613196961887027199, 613196961891221503, 613196956830793727, 613196956834988031, 613196956755296255, 613196956744810495, 613196956749004799, 613196956786753535, 613196956790947839, 613196957099229183, 613196957103423487, 613196957023731711, 613196957021634559, 613196957025828863, 613196957116006399, 613196957120200703, 613196962878980095, 613196962868494335, 613196962872688639, 613196962792996863, 613196962797191167, 613196962887368703, 613196962891563007, 613196963147415551, 613196963136929791, 613196963141124095, 613196963061432319, 613196963065626623, 613196963164192767, 613196963153707007, 613196963157901311, 613196979519881215, 613196979524075519, 613196979496812543, 613196979501006847, 613196979538755583, 613196979528269823, 613196979532464127, 613196978848792575, 613196978852986879, 613196978825723903, 613196978829918207, 613196978867666943, 613196978857181183, 613196978861375487, 613196979614253055, 613196979618447359, 613196979102547967, 613196979092062207, 613196979096256511, 613196979134005247, 613196979138199551, 613196977869422591, 613196977873616895, 613196974002274303, 613196973991788543, 613196973995982847, 613196974033731583, 613196974037925887, 613196978137858047, 613196978142052351, 613196978139955199, 613196974268612607, 613196974272806911, 613196980520222719, 613196980524417023, 613196980562165759, 613196980551679999, 613196980555874303, 613196980039974911, 613196980044169215, 613196979849134079, 613196979853328383, 613196979891077119, 613196979880591359, 613196979884785663, 613196980308410367, 613196980312604671, 613196980125958143, 613196980115472383, 613196980119666687, 613196980157415423, 613196980161609727, 613196980134346751, 613196980138541055, 613220567281041407, 613220567270555647, 613220567274749951, 613220567312498687, 613220567316692991, 613220567289430015, 613220567293624319, 613220567549476863, 613220567538991103, 613220567543185407, 613220567524311039, 613220567528505343, 613220567566254079, 613220567555768319, 613220567559962623, 613220565802549247, 613220565806743551, 613220565779480575, 613220565783674879, 613220565821423615, 613220565810937855, 613220565815132159, 613220566070984703, 613220566075179007, 613220566047916031, 613220566052110335, 613220566050013183, 613220566087761919, 613220566091956223, 613220568547721215, 613220568551915519, 613220568472223743, 613220568461737983, 613220568465932287, 613220568503681023, 613220568507875327, 613220567876632575, 613220567880826879, 613220567801135103, 613220567790649343, 613220567794843647, 613220567832592383, 613220567836786687, 613220568153456639, 613220568142970879, 613220568147165183, 613220568067473407, 613220568071667711, 613220568161845247, 613220568166039551, 613220524398477311, 613220524387991551, 613220524392185855, 613220524312494079, 613220524316688383, 613220524406865919, 613220524411060223, 613220524666912767, 613220524656427007, 613220524669009919, 613220524641746943, 613220524645941247, 613220524683689983, 613220524673204223, 613220524677398527, 613220522919985151, 613220522924179455, 613220522896916479, 613220522901110783, 613220522938859519, 613220522928373759, 613220522932568063, 613220523188420607, 613220523192614911, 613220523165351935, 613220523169546239, 613220523167449087, 613220523205197823, 613220523209392127, 613220523014356991, 613220523018551295, 613220525589659647, 613220525579173887, 613220525583368191, 613220525621116927, 613220525625311231, 613220525430276095, 613220525434470399, 613220524918571007, 613220524908085247, 613220524912279551, 613220524950028287, 613220524954222591, 613220525707100159, 613220525696614399, 613220525700808703, 613220525184909311, 613220525189103615, 613220573312450559, 613220573316644863, 613220573354393599, 613220573343907839, 613220573348102143, 613220541559472127, 613220541563666431, 613220573580886015, 613220573585080319, 613220573622829055, 613220573612343295, 613220573624926207, 613220573597663231, 613220573601857535, 613220540571713535, 613220540561227775, 613220540565422079, 613220540603170815, 613220540607365119, 613220540580102143, 613220540584296447, 613220539900624895, 613220539890139135, 613220539894333439, 613220539932082175, 613220539936276479, 613220539909013503, 613220539913207807, 613220539911110655, 613220540166963199, 613220540171157503, 613220540143894527, 613220540148088831, 613220540185837567, 613220540175351807, 613220540179546111, 613220542582882303, 613220542587076607, 613220542559813631, 613220542564007935, 613220542601756671, 613220542591270911, 613220542595465215, 613220542851317759, 613220542855512063, 613220542836637695, 613220542826151935, 613220542830346239, 613220542868094975, 613220542872289279, 613220461710409727, 613220461714604031, 613220461634912255, 613220461624426495, 613220461628620799, 613220461666369535, 613220461670563839, 613220461978845183, 613220461983039487, 613220461903347711, 613220461892861951, 613220461905444863, 613220461995622399, 613220461999816703, 613220214221307903, 613220214210822143, 613220214215016447, 613220214135324671, 613220214139518975, 613220214229696511, 613220214233890815, 613220213550219263, 613220213539733503, 613220213543927807, 613220213464236031, 613220213468430335, 613220213558607871, 613220213562802175, 613220213560705023, 613220213816557567, 613220213820751871, 613220213793488895, 613220213797683199, 613220213835431935, 613220213824946175, 613220213829140479, 613220216232476671, 613220216236670975, 613220216209407999, 613220216213602303, 613220216251351039, 613220216240865279, 613220216245059583, 613220216500912127, 613220476981870591, 613220216486232063, 613220216475746303, 613220216479940607, 613220216517689343, 613220216521883647, 613220479393595391, 613220479397789695, 613220478881890303, 613220478871404543, 613220478875598847, 613220478913347583, 613220478917541887, 613220478722506751, 613220478726701055, 613220479150325759, 613220479139839999, 613220479152422911, 613220478957387775, 613220478961582079, 613220478999330815, 613220478988845055, 613220478993039359, 613220231382302719, 613220231386497023, 613220231191461887, 613220231195656191, 613220231233404927, 613220231222919167, 613220231227113471, 613220230711214079, 613220230715408383, 613220231459897343, 613220231464091647, 613220231461994495, 613220231499743231, 613220231503937535, 613220231476674559, 613220231480868863, 613220631302897663, 613220631292411903, 613220631296606207, 613220631334354943, 613220631338549247, 613220631311286271, 613220631315480575, 613220631571333119, 613220631560847359, 613220631565041663, 613220631602790399, 613220631550361599, 613220631588110335, 613220631577624575, 613220631581818879, 613220633985155071, 613220633989349375, 613220633962086399, 613220633966280703, 613220634004029439, 613220633993543679, 613220633997737983, 613220633314066431, 613220633318260735, 613220633290997759, 613220633295192063, 613220633332940799, 613220633322455039, 613220633335037951, 613220633643319295, 613220633647513599, 613220633567821823, 613220633557336063, 613220633561530367, 613220633599279103, 613220633603473407, 613220600625758207, 613220600629952511, 613220600550260735, 613220600539774975, 613220600543969279, 613220600581718015, 613220600585912319, 613220600894193663, 613220600898387967, 613220600896290815, 613220600816599039, 613220600820793343, 613220600910970879, 613220600915165183, 613220588420333567, 613220588409847807, 613220588414042111, 613220588334350335, 613220588338544639, 613220588428722175, 613220588432916479, 613220588688769023, 613220588678283263, 613220588682477567, 613220588602785791, 613220588667797503, 613220588705546239, 613220588695060479, 613220588699254783, 613220591102590975, 613220591106785279, 613220591079522303, 613220591083716607, 613220591121465343, 613220591110979583, 613220591115173887, 613220590431502335, 613220590435696639, 613220590408433663, 613220590412627967, 613220590450376703, 613220590439890943, 613220590452473855, 613220591196962815, 613220591201157119, 613220590685257727, 613220590674771967, 613220590678966271, 613220590716715007, 613220590720909311, 613220589452132351, 613220589456326655, 613220617797238783, 613220617786753023, 613220617790947327, 613220617828696063, 613220617832890367, 613220589720567807, 613220589718470655, 613220589722664959, 613220618063577087, 613220618067771391, 613220606061576191, 613220606065770495, 613220606103519231, 613220606093033471, 613220606097227775, 613220605581328383, 613220605585522687, 613220605390487551, 613220605394681855, 613220605432430591, 613220605421944831, 613220605426139135, 613220605849763839, 613220605411459071, 613220605667311615, 613220605656825855, 613220605661020159, 613220605698768895, 613220605702963199, 613220605675700223, 613220605679894527, 613220608083230719, 613220608072744959, 613220608076939263, 613220608114687999, 613220608118882303, 613220608091619327, 613220608095813631, 613220608351666175, 613220608341180415, 613220608353763327, 613220608326500351, 613220608330694655, 613220608368443391, 613220608357957631, 613220608362151935, 613220606604738559, 613220606608932863, 613220606581669887, 613220606585864191, 613220606623612927, 613220606613127167, 613220606617321471, 613220606873174015, 613220606877368319, 613220606850105343, 613220606848008191, 613220606852202495, 613220606889951231, 613220606894145535, 613221654100705279, 613221654104899583, 613221654025207807, 613221654014722047, 613221654018916351, 613221654056665087, 613221654060859391, 613221653429616639, 613221653433810943, 613221653354119167, 613221653343633407, 613221653347827711, 613221653385576447, 613221653450588159, 613221653706440703, 613221653695954943, 613221653700149247, 613221653620457471, 613221653624651775, 613221653714829311, 613221653719023615, 613221656122359807, 613221656111874047, 613221656116068351, 613221656036376575, 613221656040570879, 613221656130748415, 613221656134942719, 613221656390795263, 613221656388698111, 613221656392892415, 613221656365629439, 613221656369823743, 613221656407572479, 613221656397086719, 613221656401281023, 613221654643867647, 613221654648061951, 613221654620798975, 613221654624993279, 613221654662742015, 613221654652256255, 613221654656450559, 613221654912303103, 613221654916497407, 613221654889234431, 613221654887137279, 613221654891331583, 613221654929080319, 613221654933274623, 613221654738239487, 613221654742433791, 613221671272185855, 613221671261700095, 613221671265894399, 613221671303643135, 613221671307837439, 613221671112802303, 613221671116996607, 613221670601097215, 613221670590611455, 613221670594805759, 613221670632554495, 613221671351877631, 613221671389626367, 613221671379140607, 613221671383334911, 613221670867435519, 613221670871629823, 613221669602852863, 613221669607047167, 613221669644795903, 613221669634310143, 613221669638504447, 613221673283354623, 613221673287548927, 613221669871288319, 613221669875482623, 613221669913231359, 613221669911134207, 613221669915328511, 613221669888065535, 613221669892259839, 613221672295596031, 613221672285110271, 613221672289304575, 613221672327053311, 613221672331247615, 613221672303984639, 613221672308178943, 613221671624507391, 613221671614021631, 613221671618215935, 613221671655964671, 613221671660158975, 613221671632895999, 613221671630798847, 613221671634993151, 613221671890845695, 613221671895039999, 613221671867777023, 613221671871971327, 613221671909720063, 613221671899234303, 613221671903428607, 613221559416389631, 613221559420583935, 613221559393320959, 613221559397515263, 613221559435263999, 613221559424778239, 613221559428972543, 613221559684825087, 613221559749836799, 613221559670145023, 613221559659659263, 613221559663853567, 613221559701602303, 613221559705796607, 613221558000812031, 613221558005006335, 613221557925314559, 613221557914828799, 613221557919023103, 613221557956771839, 613221557960966143, 613221558269247487, 613221558273441791, 613221558193750015, 613221558191652863, 613221558195847167, 613221558286024703, 613221558290219007, 613221560693555199, 613221560683069439, 613221560687263743, 613221560607571967, 613221560611766271, 613221560701943807, 613221560706138111, 613221560022466559, 613221560011980799, 613221560016175103, 613221559936483327, 613221559940677631, 613221560030855167, 613221560028758015, 613221560032952319, 613221560288804863, 613221560292999167, 613221560265736191, 613221560269930495, 613221560307679231, 613221560297193471, 613221560301387775, 613221576663367679, 613221576667561983, 613221576640299007, 613221576644493311, 613221576682242047, 613221576671756287, 613221576675950591, 613221580784271359, 613221580788465663, 613221576917123071, 613221576906637311, 613221576910831615, 613221576948580351, 613221576952774655, 613221575683997695, 613221575688191999, 613221575172292607, 613221575161806847, 613221575166001151, 613221575203749887, 613221575207944191, 613221575012909055, 613221575017103359, 613221575440728063, 613221575438630911, 613221575442825215, 613221575247790079, 613221575251984383, 613221575289733119, 613221575279247359, 613221575283441663, 613221577854550015, 613221577858744319, 613221577663709183, 613221577667903487, 613221577705652223, 613221577695166463, 613221577699360767, 613221577183461375, 613221577187655679, 613221577932144639, 613221577930047487, 613221577934241791, 613221577971990527, 613221577976184831, 613221577948921855, 613221577953116159, 613221735109492735, 613221735099006975, 613221735103201279, 613221735140950015, 613221735145144319, 613221735117881343, 613221735122075647, 613221735377928191, 613221735367442431, 613221735371636735, 613221735352762367, 613221735356956671, 613221735394705407, 613221735384219647, 613221735388413951, 613221730275557375, 613221730279751679, 613221730252488703, 613221730256683007, 613221730294431743, 613221730283945983, 613221730288140287, 613221729604468735, 613221729608663039, 613221729581400063, 613221729585594367, 613221729623343103, 613221729621245951, 613221729625440255, 613221729933721599, 613221729937915903, 613221729858224127, 613221729847738367, 613221729851932671, 613221729889681407, 613221729893875711, 613221732349640703, 613221732353835007, 613221732274143231, 613221732263657471, 613221732267851775, 613221732305600511, 613221732309794815, 613221732626464767, 613221732615979007, 613221732620173311, 613221732540481535, 613221732544675839, 613221732634853375, 613221732639047679, 613221692226928639, 613221692216442879, 613221692220637183, 613221692140945407, 613221692145139711, 613221692235317247, 613221692239511551, 613221692495364095, 613221692484878335, 613221692489072639, 613221692470198271, 613221692474392575, 613221692512141311, 613221692501655551, 613221692505849855, 613221687392993279, 613221687397187583, 613221687369924607, 613221687374118911, 613221687411867647, 613221687401381887, 613221687405576191, 613221686721904639, 613221686726098943, 613221686698835967, 613221686703030271, 613221686740779007, 613221686738681855, 613221686742876159, 613221687487365119, 613221687491559423, 613221686975660031, 613221686965174271, 613221686969368575, 613221687007117311, 613221687011311615, 613221693258727423, 613221693262921727, 613221689391579135, 613221689381093375, 613221689385287679, 613221689423036415, 613221689427230719, 613221693535551487, 613221693525065727, 613221693529260031, 613221689657917439, 613221689662111743, 613221709868171263, 613221709872365567, 613221709910114303, 613221709899628543, 613221709903822847, 613221709387923455, 613221709392117759, 613221709197082623, 613221709201276927, 613221709239025663, 613221709228539903, 613221709232734207, 613221709213859839, 613221709218054143, 613221709473906687, 613221709463420927, 613221709467615231, 613221709505363967, 613221709509558271, 613221709482295295, 613221709486489599, 613221704373633023, 613221704363147263, 613221704367341567, 613221704405090303, 613221704409284607, 613221704382021631, 613221704386215935, 613221704642068479, 613221704639971327, 613221704644165631, 613221704616902655, 613221704621096959, 613221704658845695, 613221704648359935, 613221704652554239, 613221710411333631, 613221710415527935, 613221710388264959, 613221710392459263, 613221710430207999, 613221710419722239, 613221710423916543, 613221710679769087, 613221710683963391, 613221710665089023, 613221710654603263, 613221710658797567, 613221710696546303, 613221710700740607, 613168113669636095, 613168113665441791, 613168113627693055, 613168113623498751, 613168113633984511, 613168113713676287, 613168113709481983, 613168113401200639, 613168113397006335, 613168113359257599, 613168113355063295, 613168113365549055, 613168113384423423, 613168113380229119, 613168114063900671, 613168114059706367, 613168114070192127, 613168114032443391, 613168114028249087, 613168114055512063, 613168114051317759, 613168111647981567, 613168111643787263, 613168111654273023, 613168111616524287, 613168111612329983, 613168111639592959, 613168111635398655, 613168111637495807, 613168111381643263, 613168111377448959, 613168111404711935, 613168111400517631, 613168111362768895, 613168111358574591, 613168111369060351, 613168113126473727, 613168113122279423, 613168113149542399, 613168113145348095, 613168113107599359, 613168113103405055, 613168113113890815, 613168112858038271, 613168112853843967, 613168112872718335, 613168112868524031, 613168112879009791, 613168112841261055, 613168112837066751, 613168113032101887, 613168113027907583, 613168096498155519, 613168096493961215, 613168096504446975, 613168096466698239, 613168096462503935, 613168096657539071, 613168096653344767, 613168096229720063, 613168096225525759, 613168096236011519, 613168096422658047, 613168096418463743, 613168096380715007, 613168096376520703, 613168096387006463, 613168096902905855, 613168096898711551, 613168098167488511, 613168098163294207, 613168098125545471, 613168098121351167, 613168098131836927, 613168094486986751, 613168094482792447, 613168097899053055, 613168097894858751, 613168097896955903, 613168097859207167, 613168097855012863, 613168097882275839, 613168097878081535, 613168095474745343, 613168095470551039, 613168095481036799, 613168095443288063, 613168095439093759, 613168095466356735, 613168095462162431, 613168095206309887, 613168095202115583, 613168095212601343, 613168095174852607, 613168095170658303, 613168095189532671, 613168095185338367, 613168095195824127, 613168095879495679, 613168095875301375, 613168095785123839, 613168095780929535, 613168095860621311, 613168095856427007, 613168095866912767, 613168208353951743, 613168208349757439, 613168208259579903, 613168208255385599, 613168208335077375, 613168208330883071, 613168208341368831, 613168208024698879, 613168208020504575, 613168207982755839, 613168207978561535, 613168207989047295, 613168208068739071, 613168208064544767, 613168209769529343, 613168209765335039, 613168209727586303, 613168209723391999, 613168209733877759, 613168209813569535, 613168209809375231, 613168209501093887, 613168209496899583, 613168209498996735, 613168209461247999, 613168209457053695, 613168209484316671, 613168209480122367, 613168207076786175, 613168207072591871, 613168207083077631, 613168207045328895, 613168207041134591, 613168207068397567, 613168207064203263, 613168206808350719, 613168206804156415, 613168206814642175, 613168206776893439, 613168206772699135, 613168206791573503, 613168206787379199, 613168206797864959, 613168207481536511, 613168207477342207, 613168207504605183, 613168207500410879, 613168207462662143, 613168207458467839, 613168207468953599, 613168191106973695, 613168191102779391, 613168191130042367, 613168191125848063, 613168191088099327, 613168191083905023, 613168191086002175, 613168159068782591, 613168159064588287, 613168190853218303, 613168190849023999, 613168190859509759, 613168190821761023, 613168190817566719, 613168192086343679, 613168192082149375, 613168192598048767, 613168192593854463, 613168192604340223, 613168192566591487, 613168192562397183, 613168191817908223, 613168191813713919, 613168191815811071, 613168192331710463, 613168192327516159, 613168192522551295, 613168192518356991, 613168192480608255, 613168192476413951, 613168192486899711, 613168189915791359, 613168189911597055, 613168190106632191, 613168190102437887, 613168190064689151, 613168190060494847, 613168190070980607, 613168189647355903, 613168189643161599, 613168189829808127, 613168189825613823, 613168189836099583, 613168189798350847, 613168189794156543, 613168189821419519, 613168189817225215, 613168202312056831, 613168202307862527, 613168202318348287, 613168202280599551, 613168202276405247, 613168202303668223, 613168202299473919, 613168202043621375, 613168202039427071, 613168202041524223, 613168201951346687, 613168201947152383, 613168202026844159, 613168202022649855, 613168202033135615, 613168037494783999, 613168037490589695, 613168037400412159, 613168037396217855, 613168037475909631, 613168037471715327, 613168037482201087, 613168037226348543, 613168037222154239, 613168037131976703, 613168037127782399, 613168037129879551, 613168037209571327, 613168037205377023, 613168037836619775, 613168037832425471, 613168037794676735, 613168037790482431, 613168037800968191, 613168037880659967, 613168037876465663, 613168035420700671, 613168035416506367, 613168035378757631, 613168035374563327, 613168035385049087, 613168035464740863, 613168035399729151, 613168035143876607, 613168035139682303, 613168035150168063, 613168035112419327, 613168035108225023, 613168035135487999, 613168035131293695, 613168047626125311, 613168047621931007, 613168047632416767, 613168047594668031, 613168047590473727, 613168047617736703, 613168047613542399, 613168047357689855, 613168047353495551, 613168047355592703, 613168047382855679, 613168047378661375, 613168047340912639, 613168047336718335, 613168047347204095, 613168020247805951, 613168020243611647, 613168020270874623, 613168020266680319, 613168020228931583, 613168020224737279, 613168020235223039, 613168019979370495, 613168019975176191, 613168020002439167, 613168019998244863, 613168020000342015, 613168019962593279, 613168019958398975, 613168020153434111, 613168020149239807, 613168020665139199, 613168020660944895, 613168020671430655, 613168020633681919, 613168020629487615, 613168032635682815, 613168032631488511, 613168018249220095, 613168018245025791, 613168018255511551, 613168018217762815, 613168032396607487, 613168032358858751, 613168032354664447, 613168032365150207, 613168017982881791, 613168017978687487, 613168029984882687, 613168029980688383, 613168029942939647, 613168029938745343, 613168029949231103, 613168030465130495, 613168030460936191, 613168029716447231, 613168029712252927, 613168029674504191, 613168029670309887, 613168029672407039, 613168029699670015, 613168029695475711, 613168030379147263, 613168030374952959, 613168030385438719, 613168030347689983, 613168030343495679, 613168030370758655, 613168030366564351, 613168063396708351, 613168063392514047, 613168063402999807, 613168063365251071, 613168063361056767, 613168063388319743, 613168063384125439, 613168063386222591, 613168063130370047, 613168063126175743, 613168063035998207, 613168063031803903, 613168063111495679, 613168063107301375, 613168063117787135, 613168831526862847, 613168831522668543, 613168831432491007, 613168831428296703, 613168831507988479, 613168831503794175, 613168831514279935, 613168831258427391, 613168831193415679, 613168831155666943, 613168831151472639, 613168831161958399, 613168831241650175, 613168831237455871, 613168828781690879, 613168828777496575, 613168828739747839, 613168828735553535, 613168828746039295, 613168828825731071, 613168828821536767, 613168828513255423, 613168828509061119, 613168828471312383, 613168828467118079, 613168828469215231, 613168828496478207, 613168828492283903, 613168829175955455, 613168829171761151, 613168829182246911, 613168829144498175, 613168829140303871, 613168829167566847, 613168829163372543, 613167015357579263, 613167015353384959, 613167015363870719, 613167015326121983, 613167015321927679, 613167015349190655, 613167015336607743, 613167015347093503, 613167015091240959, 613167015087046655, 613167015114309631, 613167015110115327, 613167015072366591, 613167015068172287, 613167015078658047, 613168814279884799, 613168814275690495, 613168814302953471, 613168814298759167, 613168814261010431, 613168814256816127, 613168814267301887, 613168814011449343, 613168813510230015, 613168814026129407, 613168814021935103, 613168814032420863, 613168813994672127, 613168813990477823, 613168814185512959, 613168814181318655, 613168811610210303, 613168811606015999, 613168811616501759, 613168811578753023, 613168811574558719, 613168811769593855, 613168811765399551, 613168811341774847, 613168811337580543, 613168811339677695, 613168811534712831, 613168811530518527, 613168811492769791, 613168811488575487, 613168811499061247, 613168812014960639, 613168812010766335, 613168813279543295, 613168813275348991, 613168813237600255, 613168813233405951, 613168813243891711, 613166998196584447, 613166998192390143, 613168813011107839, 613168812998524927, 613168813009010687, 613168812971261951, 613168812967067647, 613168812994330623, 613168812990136319, 613167078641238015, 613167078637043711, 613167078647529471, 613167078609780735, 613167078605586431, 613167078632849407, 613167078628655103, 613167078372802559, 613167078368608255, 613167078379094015, 613167078341345279, 613167078276333567, 613167078356025343, 613167078351831039, 613167078362316799, 613167079045988351, 613167079041794047, 613167078951616511, 613167078947422207, 613167079027113983, 613167079022919679, 613167079033405439, 613167076630069247, 613167076625874943, 613167076535697407, 613167076531503103, 613167076611194879, 613167076607000575, 613167076609097727, 613167076300816383, 613167076296622079, 613167076258873343, 613167076254679039, 613167076265164799, 613167076344856575, 613167076340662271, 613167078045646847, 613167078041452543, 613167078003703807, 613167077999509503, 613167078009995263, 613167078089687039, 613167078085492735, 613167077777211391, 613167077764628479, 613167077775114239, 613167077737365503, 613167077733171199, 613167077760434175, 613167077756239871, 613167061394259967, 613167061390065663, 613167061400551423, 613167061362802687, 613167061358608383, 613167061385871359, 613167061381677055, 613167061125824511, 613167061121630207, 613167061132115967, 613167061094367231, 613167061146796031, 613167061109047295, 613167061104852991, 613167061115338751, 613167061799010303, 613167061794815999, 613167061822078975, 613167061817884671, 613167061780135935, 613167061775941631, 613167061786427391, 613167059383091199, 613167059378896895, 613167059406159871, 613167059401965567, 613167059364216831, 613167059366313983, 613167059362119679, 613167055262187519, 613167055257993215, 613167059129335807, 613167059125141503, 613167059135627263, 613167059097878527, 613167059093684223, 613167060362461183, 613167060358266879, 613167060874166271, 613167060869971967, 613167060880457727, 613167060842708991, 613167060838514687, 613167060094025727, 613167060081442815, 613167060091928575, 613167060607827967, 613167060603633663, 613167060798668799, 613167060794474495, 613167060756725759, 613167060752531455, 613167060763017215, 613167104362807295, 613167104358612991, 613167104553648127, 613167104549453823, 613167104511705087, 613167104507510783, 613167104517996543, 613167104094371839, 613167104532676607, 613167104276824063, 613167104272629759, 613167104283115519, 613167104245366783, 613167104241172479, 613167104268435455, 613167104264241151, 613167098505461759, 613167098501267455, 613167098511753215, 613167098474004479, 613167098469810175, 613167098497073151, 613167098492878847, 613167098237026303, 613167098239123455, 613167098234929151, 613167098144751615, 613167098140557311, 613167098220249087, 613167098216054783, 613167098226540543, 613167103339397119, 613167103335202815, 613167103245025279, 613167103240830975, 613167103320522751, 613167103316328447, 613167103326814207, 613167103070961663, 613167103066767359, 613167102976589823, 613167102964006911, 613167102974492671, 613167103054184447, 613167103049990143, 613167103681232895, 613167103677038591, 613167103639289855, 613167103635095551, 613167103645581311, 613167103725273087, 613167103721078783, 613166949867716607, 613166949863522303, 613166949825773567, 613166949821579263, 613166949832065023, 613166949850939391, 613166949846745087, 613166949590892543, 613166949586698239, 613166949597183999, 613166949559435263, 613166949555240959, 613166949582503935, 613166949578309631, 613166943819530239, 613166943815335935, 613166943825821695, 613166943788072959, 613166943783878655, 613166943811141631, 613166943806947327, 613166943551094783, 613166943553191935, 613166943548997631, 613166943576260607, 613166943572066303, 613166943534317567, 613166943530123263, 613166943540609023, 613166948653465599, 613166948649271295, 613166948676534271, 613166948672339967, 613166948634591231, 613166948630396927, 613166948640882687, 613166948385030143, 613166948380835839, 613166948408098815, 613166948395515903, 613166948406001663, 613166948368252927, 613166948364058623, 613166948559093759, 613166948554899455, 613166949070798847, 613166949066604543, 613166949077090303, 613166949039341567, 613166949035147263, 613166928829087743, 613166928824893439, 613166932696236031, 613166932692041727, 613166932702527487, 613166928594206719, 613166928590012415, 613166928552263679, 613166928548069375, 613166928558555135, 613166932429897727, 613166932425703423, 613166926178287615, 613166926174093311, 613166926136344575, 613166926132150271, 613166926142636031, 613166926658535423, 613166926654341119, 613166925909852159, 613166925905657855, 613166925867909119, 613166925870006271, 613166925865811967, 613166925893074943, 613166925888880639, 613166926572552191, 613166926568357887, 613166926578843647, 613166926541094911, 613166926536900607, 613166926564163583, 613166926559969279, 613166931672825855, 613166931668631551, 613166931679117311, 613166931641368575, 613166931637174271, 613166931664437247, 613166931651854335, 613166931662340095, 613166931406487551, 613166931402293247, 613166931312115711, 613166931307921407, 613166931387613183, 613166931383418879, 613166931393904639, 613167727720267775, 613167727716073471, 613167727625895935, 613167727621701631, 613167727701393407, 613167727697199103, 613167727707684863, 613167727391014911, 613167727386820607, 613167727349071871, 613167727344877567, 613167727355363327, 613167727435055103, 613167727430860799, 613167724975095807, 613167724970901503] | ++----------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ SELECT geohash(37.76938, -122.3889, 9); diff --git a/tests/cases/standalone/common/function/geo.sql b/tests/cases/standalone/common/function/geo.sql index cd9d403e6e..af2a16517d 100644 --- a/tests/cases/standalone/common/function/geo.sql +++ b/tests/cases/standalone/common/function/geo.sql @@ -26,18 +26,34 @@ SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64), h3_latlng_to_cell_stri SELECT h3_cell_to_string(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_str, h3_string_to_cell(h3_latlng_to_cell_string(37.76938, -122.3889, 8::UInt64)) AS cell_index; -SELECT h3_cell_center_lat(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lat, h3_cell_center_lng(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center_lng; +SELECT h3_cell_center_latlng(h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64)) AS cell_center; SELECT h3_cell_resolution(cell) AS resolution, h3_cell_base(cell) AS base, h3_cell_is_pentagon(cell) AS pentagon, h3_cell_parent(cell, 6::UInt64) AS parent, + h3_cell_to_children(cell, 10::UInt64) AS children, + h3_cell_to_children_size(cell, 10) AS children_count, + h3_cell_to_child_pos(cell, 6) AS child_pos, + h3_child_pos_to_cell(25, cell, 11) AS child FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell); -SELECT h3_is_neighbour(cell1, cell2) -FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell1, h3_latlng_to_cell(36.76938, -122.3889, 8::UInt64) AS cell2); +SELECT + h3_grid_disk(cell, 0) AS current_cell, + h3_grid_disk(cell, 3) AS grids, + h3_grid_disk_distances(cell, 3) AS all_grids, +FROM (SELECT h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell); +SELECT + h3_grid_distance(cell1, cell2) AS distance, + h3_grid_path_cells(cell1, cell2) AS path_cells, +FROM + ( + SELECT + h3_latlng_to_cell(37.76938, -122.3889, 8::UInt64) AS cell1, + h3_latlng_to_cell(39.634, -104.999, 8::UInt64) AS cell2 + ); SELECT geohash(37.76938, -122.3889, 9); From 856c0280f53d3d70ee6f5b0e71568f549f6a5479 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Sat, 12 Oct 2024 17:04:22 +0800 Subject: [PATCH 071/128] feat: remove the distributed lock (#4825) * feat: remove the distributed lock as we do not need it any more * chore: delete todo comment * chore: remove unused error --- src/meta-client/examples/lock.rs | 129 -------------------- src/meta-client/src/client.rs | 35 ------ src/meta-client/src/client/lock.rs | 178 ---------------------------- src/meta-srv/src/bootstrap.rs | 22 +--- src/meta-srv/src/error.rs | 27 ----- src/meta-srv/src/lib.rs | 1 - src/meta-srv/src/lock.rs | 99 ---------------- src/meta-srv/src/lock/etcd.rs | 93 --------------- src/meta-srv/src/lock/memory.rs | 112 ----------------- src/meta-srv/src/metasrv.rs | 6 - src/meta-srv/src/metasrv/builder.rs | 12 -- src/meta-srv/src/service.rs | 1 - src/meta-srv/src/service/lock.rs | 51 -------- 13 files changed, 4 insertions(+), 762 deletions(-) delete mode 100644 src/meta-client/examples/lock.rs delete mode 100644 src/meta-client/src/client/lock.rs delete mode 100644 src/meta-srv/src/lock.rs delete mode 100644 src/meta-srv/src/lock/etcd.rs delete mode 100644 src/meta-srv/src/lock/memory.rs delete mode 100644 src/meta-srv/src/service/lock.rs diff --git a/src/meta-client/examples/lock.rs b/src/meta-client/examples/lock.rs deleted file mode 100644 index c8a8b61d60..0000000000 --- a/src/meta-client/examples/lock.rs +++ /dev/null @@ -1,129 +0,0 @@ -// 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::sync::Arc; -use std::time::Duration; - -use api::v1::meta::Role; -use common_grpc::channel_manager::{ChannelConfig, ChannelManager}; -use common_meta::rpc::lock::{LockRequest, UnlockRequest}; -use meta_client::client::MetaClientBuilder; -use meta_client::MetaClientRef; -use tracing::{info, subscriber}; -use tracing_subscriber::FmtSubscriber; - -fn main() { - subscriber::set_global_default(FmtSubscriber::builder().finish()).unwrap(); - run(); -} - -#[tokio::main] -async fn run() { - let id = (1000u64, 2000u64); - let config = ChannelConfig::new() - .timeout(Duration::from_secs(30)) - .connect_timeout(Duration::from_secs(5)) - .tcp_nodelay(true); - let channel_manager = ChannelManager::with_config(config); - let mut meta_client = MetaClientBuilder::new(id.0, id.1, Role::Datanode) - .enable_lock() - .channel_manager(channel_manager) - .build(); - meta_client.start(&["127.0.0.1:3002"]).await.unwrap(); - let meta_client = Arc::new(meta_client); - - run_normal(meta_client.clone()).await; - - run_multi_thread(meta_client.clone()).await; - - run_multi_thread_with_one_timeout(meta_client).await; -} - -async fn run_normal(meta_client: MetaClientRef) { - let name = "lock_name".as_bytes().to_vec(); - let expire_secs = 60; - - let lock_req = LockRequest { name, expire_secs }; - - let lock_result = meta_client.lock(lock_req).await.unwrap(); - let key = lock_result.key; - info!( - "lock success! Returned key: {}", - String::from_utf8(key.clone()).unwrap() - ); - - // It is recommended that time of holding lock is less than the timeout of the grpc channel - info!("do some work, take 3 seconds"); - tokio::time::sleep(Duration::from_secs(3)).await; - - let unlock_req = UnlockRequest { key }; - - meta_client.unlock(unlock_req).await.unwrap(); - info!("unlock success!"); -} - -async fn run_multi_thread(meta_client: MetaClientRef) { - let meta_client_clone = meta_client.clone(); - let join1 = tokio::spawn(async move { - run_normal(meta_client_clone.clone()).await; - }); - - tokio::time::sleep(Duration::from_secs(1)).await; - - let join2 = tokio::spawn(async move { - run_normal(meta_client).await; - }); - - join1.await.unwrap(); - join2.await.unwrap(); -} - -async fn run_multi_thread_with_one_timeout(meta_client: MetaClientRef) { - let meta_client_clone = meta_client.clone(); - let join1 = tokio::spawn(async move { - run_with_timeout(meta_client_clone.clone()).await; - }); - - tokio::time::sleep(Duration::from_secs(1)).await; - - let join2 = tokio::spawn(async move { - run_normal(meta_client).await; - }); - - join1.await.unwrap(); - join2.await.unwrap(); -} - -async fn run_with_timeout(meta_client: MetaClientRef) { - let name = "lock_name".as_bytes().to_vec(); - let expire_secs = 5; - - let lock_req = LockRequest { name, expire_secs }; - - let lock_result = meta_client.lock(lock_req).await.unwrap(); - let key = lock_result.key; - info!( - "lock success! Returned key: {}", - String::from_utf8(key.clone()).unwrap() - ); - - // It is recommended that time of holding lock is less than the timeout of the grpc channel - info!("do some work, take 20 seconds"); - tokio::time::sleep(Duration::from_secs(20)).await; - - let unlock_req = UnlockRequest { key }; - - meta_client.unlock(unlock_req).await.unwrap(); - info!("unlock success!"); -} diff --git a/src/meta-client/src/client.rs b/src/meta-client/src/client.rs index 1c5b96a684..ed6fdf13fb 100644 --- a/src/meta-client/src/client.rs +++ b/src/meta-client/src/client.rs @@ -15,7 +15,6 @@ mod ask_leader; mod heartbeat; mod load_balance; -mod lock; mod procedure; mod cluster; @@ -33,7 +32,6 @@ use common_meta::datanode::{DatanodeStatKey, DatanodeStatValue, RegionStat}; use common_meta::ddl::{ExecutorContext, ProcedureExecutor}; use common_meta::error::{self as meta_error, Result as MetaResult}; use common_meta::rpc::ddl::{SubmitDdlTaskRequest, SubmitDdlTaskResponse}; -use common_meta::rpc::lock::{LockRequest, LockResponse, UnlockRequest}; use common_meta::rpc::procedure::{ MigrateRegionRequest, MigrateRegionResponse, ProcedureStateResponse, }; @@ -45,7 +43,6 @@ use common_meta::rpc::store::{ use common_meta::ClusterId; use common_telemetry::info; use heartbeat::Client as HeartbeatClient; -use lock::Client as LockClient; use procedure::Client as ProcedureClient; use snafu::{OptionExt, ResultExt}; use store::Client as StoreClient; @@ -67,7 +64,6 @@ pub struct MetaClientBuilder { role: Role, enable_heartbeat: bool, enable_store: bool, - enable_lock: bool, enable_procedure: bool, enable_access_cluster_info: bool, channel_manager: Option, @@ -123,13 +119,6 @@ impl MetaClientBuilder { } } - pub fn enable_lock(self) -> Self { - Self { - enable_lock: true, - ..self - } - } - pub fn enable_procedure(self) -> Self { Self { enable_procedure: true, @@ -188,10 +177,6 @@ impl MetaClientBuilder { client.store = Some(StoreClient::new(self.id, self.role, mgr.clone())); } - if self.enable_lock { - client.lock = Some(LockClient::new(self.id, self.role, mgr.clone())); - } - if self.enable_procedure { let mgr = self.ddl_channel_manager.unwrap_or(mgr.clone()); client.procedure = Some(ProcedureClient::new( @@ -221,7 +206,6 @@ pub struct MetaClient { channel_manager: ChannelManager, heartbeat: Option, store: Option, - lock: Option, procedure: Option, cluster: Option, } @@ -383,10 +367,6 @@ impl MetaClient { client.start(urls.clone()).await?; info!("Store client started"); } - if let Some(client) = &mut self.lock { - client.start(urls.clone()).await?; - info!("Lock client started"); - } if let Some(client) = &mut self.procedure { client.start(urls.clone()).await?; info!("DDL client started"); @@ -482,15 +462,6 @@ impl MetaClient { .context(ConvertMetaResponseSnafu) } - pub async fn lock(&self, req: LockRequest) -> Result { - self.lock_client()?.lock(req.into()).await.map(Into::into) - } - - pub async fn unlock(&self, req: UnlockRequest) -> Result<()> { - let _ = self.lock_client()?.unlock(req.into()).await?; - Ok(()) - } - /// Query the procedure state by its id. pub async fn query_procedure_state(&self, pid: &str) -> Result { self.procedure_client()?.query_procedure_state(pid).await @@ -538,12 +509,6 @@ impl MetaClient { }) } - pub fn lock_client(&self) -> Result { - self.lock.clone().context(NotStartedSnafu { - name: "lock_client", - }) - } - pub fn procedure_client(&self) -> Result { self.procedure.clone().context(NotStartedSnafu { name: "procedure_client", diff --git a/src/meta-client/src/client/lock.rs b/src/meta-client/src/client/lock.rs deleted file mode 100644 index 66fe077c22..0000000000 --- a/src/meta-client/src/client/lock.rs +++ /dev/null @@ -1,178 +0,0 @@ -// 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::HashSet; -use std::sync::Arc; - -use api::v1::meta::lock_client::LockClient; -use api::v1::meta::{LockRequest, LockResponse, Role, UnlockRequest, UnlockResponse}; -use common_grpc::channel_manager::ChannelManager; -use common_telemetry::tracing_context::TracingContext; -use snafu::{ensure, OptionExt, ResultExt}; -use tokio::sync::RwLock; -use tonic::transport::Channel; - -use crate::client::{load_balance, Id}; -use crate::error; -use crate::error::Result; - -#[derive(Clone, Debug)] -pub struct Client { - inner: Arc>, -} - -impl Client { - pub fn new(id: Id, role: Role, channel_manager: ChannelManager) -> Self { - let inner = Arc::new(RwLock::new(Inner { - id, - role, - channel_manager, - peers: vec![], - })); - - Self { inner } - } - - pub async fn start(&mut self, urls: A) -> Result<()> - where - U: AsRef, - A: AsRef<[U]>, - { - let mut inner = self.inner.write().await; - inner.start(urls).await - } - - pub async fn lock(&self, req: LockRequest) -> Result { - let inner = self.inner.read().await; - inner.lock(req).await - } - - pub async fn unlock(&self, req: UnlockRequest) -> Result { - let inner = self.inner.read().await; - inner.unlock(req).await - } -} - -#[derive(Debug)] -struct Inner { - id: Id, - role: Role, - channel_manager: ChannelManager, - peers: Vec, -} - -impl Inner { - async fn start(&mut self, urls: A) -> Result<()> - where - U: AsRef, - A: AsRef<[U]>, - { - ensure!( - !self.is_started(), - error::IllegalGrpcClientStateSnafu { - err_msg: "Lock client already started", - } - ); - - self.peers = urls - .as_ref() - .iter() - .map(|url| url.as_ref().to_string()) - .collect::>() - .drain() - .collect::>(); - - Ok(()) - } - - fn random_client(&self) -> Result> { - let len = self.peers.len(); - let peer = load_balance::random_get(len, |i| Some(&self.peers[i])).context( - error::IllegalGrpcClientStateSnafu { - err_msg: "Empty peers, lock client may not start yet", - }, - )?; - - self.make_client(peer) - } - - fn make_client(&self, addr: impl AsRef) -> Result> { - let channel = self - .channel_manager - .get(addr) - .context(error::CreateChannelSnafu)?; - - Ok(LockClient::new(channel)) - } - - #[inline] - fn is_started(&self) -> bool { - !self.peers.is_empty() - } - - async fn lock(&self, mut req: LockRequest) -> Result { - let mut client = self.random_client()?; - req.set_header( - self.id, - self.role, - TracingContext::from_current_span().to_w3c(), - ); - let res = client.lock(req).await.map_err(error::Error::from)?; - - Ok(res.into_inner()) - } - - async fn unlock(&self, mut req: UnlockRequest) -> Result { - let mut client = self.random_client()?; - req.set_header( - self.id, - self.role, - TracingContext::from_current_span().to_w3c(), - ); - let res = client.unlock(req).await.map_err(error::Error::from)?; - - Ok(res.into_inner()) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[tokio::test] - async fn test_already_start() { - let mut client = Client::new((0, 0), Role::Datanode, ChannelManager::default()); - client - .start(&["127.0.0.1:1000", "127.0.0.1:1001"]) - .await - .unwrap(); - let res = client.start(&["127.0.0.1:1002"]).await; - assert!(res.is_err()); - assert!(matches!( - res.err(), - Some(error::Error::IllegalGrpcClientState { .. }) - )); - } - - #[tokio::test] - async fn test_start_with_duplicate_peers() { - let mut client = Client::new((0, 0), Role::Datanode, ChannelManager::default()); - client - .start(&["127.0.0.1:1000", "127.0.0.1:1000", "127.0.0.1:1000"]) - .await - .unwrap(); - - assert_eq!(1, client.inner.write().await.peers.len()); - } -} diff --git a/src/meta-srv/src/bootstrap.rs b/src/meta-srv/src/bootstrap.rs index dcfac253f3..d10b22c610 100644 --- a/src/meta-srv/src/bootstrap.rs +++ b/src/meta-srv/src/bootstrap.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use api::v1::meta::cluster_server::ClusterServer; use api::v1::meta::heartbeat_server::HeartbeatServer; -use api::v1::meta::lock_server::LockServer; use api::v1::meta::procedure_service_server::ProcedureServiceServer; use api::v1::meta::store_server::StoreServer; use common_base::Plugins; @@ -48,8 +47,6 @@ use crate::election::etcd::EtcdElection; #[cfg(feature = "pg_kvbackend")] use crate::error::InvalidArgumentsSnafu; use crate::error::{InitExportMetricsTaskSnafu, TomlFormatSnafu}; -use crate::lock::etcd::EtcdLock; -use crate::lock::memory::MemLock; use crate::metasrv::builder::MetasrvBuilder; use crate::metasrv::{BackendImpl, Metasrv, MetasrvOptions, SelectorRef}; use crate::selector::lease_based::LeaseBasedSelector; @@ -183,7 +180,6 @@ pub fn router(metasrv: Metasrv) -> Router { .add_service(HeartbeatServer::new(metasrv.clone())) .add_service(StoreServer::new(metasrv.clone())) .add_service(ClusterServer::new(metasrv.clone())) - .add_service(LockServer::new(metasrv.clone())) .add_service(ProcedureServiceServer::new(metasrv.clone())) .add_service(admin::make_admin_service(metasrv)) } @@ -193,13 +189,9 @@ pub async fn metasrv_builder( plugins: Plugins, kv_backend: Option, ) -> Result { - let (kv_backend, election, lock) = match (kv_backend, &opts.backend) { - (Some(kv_backend), _) => (kv_backend, None, Some(Arc::new(MemLock::default()) as _)), - (None, BackendImpl::MemoryStore) => ( - Arc::new(MemoryKvBackend::new()) as _, - None, - Some(Arc::new(MemLock::default()) as _), - ), + let (kv_backend, election) = match (kv_backend, &opts.backend) { + (Some(kv_backend), _) => (kv_backend, None), + (None, BackendImpl::MemoryStore) => (Arc::new(MemoryKvBackend::new()) as _, None), (None, BackendImpl::EtcdStore) => { let etcd_client = create_etcd_client(opts).await?; let kv_backend = { @@ -224,18 +216,13 @@ pub async fn metasrv_builder( ) .await?, ), - Some(EtcdLock::with_etcd_client( - etcd_client, - opts.store_key_prefix.clone(), - )?), ) } #[cfg(feature = "pg_kvbackend")] (None, BackendImpl::PostgresStore) => { let pg_client = create_postgres_client(opts).await?; let kv_backend = PgStore::with_pg_client(pg_client).await.unwrap(); - // TODO: implement locking and leader election for pg backend. - (kv_backend, None, None) + (kv_backend, None) } }; @@ -253,7 +240,6 @@ pub async fn metasrv_builder( .in_memory(in_memory) .selector(selector) .election(election) - .lock(lock) .plugins(plugins)) } diff --git a/src/meta-srv/src/error.rs b/src/meta-srv/src/error.rs index 4c3d974c0a..6bcac5db2a 100644 --- a/src/meta-srv/src/error.rs +++ b/src/meta-srv/src/error.rs @@ -448,30 +448,6 @@ pub enum Error { location: Location, }, - #[snafu(display("Failed to lock based on etcd"))] - Lock { - #[snafu(source)] - error: etcd_client::Error, - #[snafu(implicit)] - location: Location, - }, - - #[snafu(display("Failed to unlock based on etcd"))] - Unlock { - #[snafu(source)] - error: etcd_client::Error, - #[snafu(implicit)] - location: Location, - }, - - #[snafu(display("Failed to grant lease"))] - LeaseGrant { - #[snafu(source)] - error: etcd_client::Error, - #[snafu(implicit)] - location: Location, - }, - #[snafu(display("Invalid utf-8 value"))] InvalidUtf8Value { #[snafu(source)] @@ -770,9 +746,6 @@ impl ErrorExt for Error { | Error::ResponseHeaderNotFound { .. } | Error::IsNotLeader { .. } | Error::InvalidHttpBody { .. } - | Error::Lock { .. } - | Error::Unlock { .. } - | Error::LeaseGrant { .. } | Error::ExceededRetryLimit { .. } | Error::SendShutdownSignal { .. } | Error::PusherNotFound { .. } diff --git a/src/meta-srv/src/lib.rs b/src/meta-srv/src/lib.rs index 01b48f1da0..a438f4255f 100644 --- a/src/meta-srv/src/lib.rs +++ b/src/meta-srv/src/lib.rs @@ -28,7 +28,6 @@ pub mod flow_meta_alloc; pub mod handler; pub mod key; pub mod lease; -pub mod lock; pub mod metasrv; mod metrics; #[cfg(feature = "mock")] diff --git a/src/meta-srv/src/lock.rs b/src/meta-srv/src/lock.rs deleted file mode 100644 index 53451591da..0000000000 --- a/src/meta-srv/src/lock.rs +++ /dev/null @@ -1,99 +0,0 @@ -// Copyright 2023 Greptime Team -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -pub mod etcd; -pub(crate) mod memory; - -use std::sync::Arc; - -use common_telemetry::error; - -use crate::error::Result; - -pub type Key = Vec; - -pub const DEFAULT_EXPIRE_TIME_SECS: u64 = 10; - -pub struct Opts { - // If the expiration time is exceeded and currently holds the lock, the lock is - // automatically released. - pub expire_secs: Option, -} - -impl Default for Opts { - fn default() -> Self { - Opts { - expire_secs: Some(DEFAULT_EXPIRE_TIME_SECS), - } - } -} - -#[async_trait::async_trait] -pub trait DistLock: Send + Sync { - // Lock acquires a distributed shared lock on a given named lock. On success, it - // will return a unique key that exists so long as the lock is held by the caller. - async fn lock(&self, name: Vec, opts: Opts) -> Result; - - // Unlock takes a key returned by Lock and releases the hold on lock. - async fn unlock(&self, key: Vec) -> Result<()>; -} - -pub type DistLockRef = Arc; - -pub struct DistLockGuard<'a> { - lock: &'a DistLockRef, - name: Vec, - key: Option, -} - -impl<'a> DistLockGuard<'a> { - pub fn new(lock: &'a DistLockRef, name: Vec) -> Self { - Self { - lock, - name, - key: None, - } - } - - pub async fn lock(&mut self) -> Result<()> { - if self.key.is_some() { - return Ok(()); - } - let key = self - .lock - .lock( - self.name.clone(), - Opts { - expire_secs: Some(2), - }, - ) - .await?; - self.key = Some(key); - Ok(()) - } -} - -impl Drop for DistLockGuard<'_> { - fn drop(&mut self) { - if let Some(key) = self.key.take() { - let lock = self.lock.clone(); - let name = self.name.clone(); - let _handle = common_runtime::spawn_global(async move { - if let Err(e) = lock.unlock(key).await { - error!(e; "Failed to unlock '{}'", String::from_utf8_lossy(&name)); - } - }); - } - } -} diff --git a/src/meta-srv/src/lock/etcd.rs b/src/meta-srv/src/lock/etcd.rs deleted file mode 100644 index 3f53b40e15..0000000000 --- a/src/meta-srv/src/lock/etcd.rs +++ /dev/null @@ -1,93 +0,0 @@ -// 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::sync::Arc; - -use etcd_client::{Client, LockOptions}; -use snafu::ResultExt; - -use super::{DistLock, DistLockRef, Opts, DEFAULT_EXPIRE_TIME_SECS}; -use crate::error; -use crate::error::Result; - -/// A implementation of distributed lock based on etcd. The Clone of EtcdLock is cheap. -#[derive(Clone)] -pub struct EtcdLock { - client: Client, - store_key_prefix: String, -} - -impl EtcdLock { - pub async fn with_endpoints(endpoints: S, store_key_prefix: String) -> Result - where - E: AsRef, - S: AsRef<[E]>, - { - let client = Client::connect(endpoints, None) - .await - .context(error::ConnectEtcdSnafu)?; - - Self::with_etcd_client(client, store_key_prefix) - } - - pub fn with_etcd_client(client: Client, store_key_prefix: String) -> Result { - Ok(Arc::new(EtcdLock { - client, - store_key_prefix, - })) - } - - fn lock_key(&self, key: Vec) -> Vec { - if self.store_key_prefix.is_empty() { - key - } else { - let mut prefix = self.store_key_prefix.as_bytes().to_vec(); - prefix.extend_from_slice(&key); - prefix - } - } -} - -#[async_trait::async_trait] -impl DistLock for EtcdLock { - async fn lock(&self, key: Vec, opts: Opts) -> Result> { - let expire = opts.expire_secs.unwrap_or(DEFAULT_EXPIRE_TIME_SECS) as i64; - - let mut client = self.client.clone(); - - let resp = client - .lease_grant(expire, None) - .await - .context(error::LeaseGrantSnafu)?; - - let lease_id = resp.id(); - let lock_opts = LockOptions::new().with_lease(lease_id); - - let resp = client - .lock(self.lock_key(key), Some(lock_opts)) - .await - .context(error::LockSnafu)?; - - Ok(resp.key().to_vec()) - } - - async fn unlock(&self, key: Vec) -> Result<()> { - let mut client = self.client.clone(); - let _ = client - .unlock(self.lock_key(key)) - .await - .context(error::UnlockSnafu)?; - Ok(()) - } -} diff --git a/src/meta-srv/src/lock/memory.rs b/src/meta-srv/src/lock/memory.rs deleted file mode 100644 index d312700d12..0000000000 --- a/src/meta-srv/src/lock/memory.rs +++ /dev/null @@ -1,112 +0,0 @@ -// 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::sync::Arc; - -use async_trait::async_trait; -use dashmap::DashMap; -use tokio::sync::{Mutex, OwnedMutexGuard}; - -use crate::error::Result; -use crate::lock::{DistLock, Key, Opts}; - -#[derive(Default)] -pub(crate) struct MemLock { - mutexes: DashMap>>, - guards: DashMap>, -} - -#[async_trait] -impl DistLock for MemLock { - async fn lock(&self, key: Vec, _opts: Opts) -> Result { - let mutex = self - .mutexes - .entry(key.clone()) - .or_insert_with(|| Arc::new(Mutex::new(()))) - .clone(); - - let guard = mutex.lock_owned().await; - - let _ = self.guards.insert(key.clone(), guard); - Ok(key) - } - - async fn unlock(&self, key: Vec) -> Result<()> { - // drop the guard, so that the mutex can be unlocked, - // effectively make the `mutex.lock_owned` in `lock` method to proceed - let _ = self.guards.remove(&key); - Ok(()) - } -} - -#[cfg(test)] -mod tests { - use std::collections::HashMap; - use std::sync::atomic::{AtomicU32, Ordering}; - - use rand::seq::SliceRandom; - - use super::*; - - #[tokio::test(flavor = "multi_thread")] - async fn test_mem_lock_concurrently() { - let lock = Arc::new(MemLock::default()); - - let keys = (0..10) - .map(|i| format!("my-lock-{i}").into_bytes()) - .collect::>(); - let counters: [(Key, AtomicU32); 10] = keys - .iter() - .map(|x| (x.clone(), AtomicU32::new(0))) - .collect::>() - .try_into() - .unwrap(); - let counters = Arc::new(HashMap::from(counters)); - - let tasks = (0..100) - .map(|_| { - let mut keys = keys.clone(); - keys.shuffle(&mut rand::thread_rng()); - - let lock_clone = lock.clone(); - let counters_clone = counters.clone(); - tokio::spawn(async move { - // every key counter will be added by 1 for 10 times - for i in 0..100 { - let key = &keys[i % keys.len()]; - assert!(lock_clone - .lock(key.clone(), Opts { expire_secs: None }) - .await - .is_ok()); - - // Intentionally create a critical section: - // if our MemLock is flawed, the resulting counter is wrong. - // - // Note that AtomicU32 is only used to enable the updates from multiple tasks, - // does not make any guarantee about the correctness of the result. - - let counter = counters_clone.get(key).unwrap(); - let v = counter.load(Ordering::Relaxed); - counter.store(v + 1, Ordering::Relaxed); - - lock_clone.unlock(key.clone()).await.unwrap(); - } - }) - }) - .collect::>(); - let _ = futures::future::join_all(tasks).await; - - assert!(counters.values().all(|x| x.load(Ordering::Relaxed) == 1000)); - } -} diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index 3cdb64e1e0..c43eec60d4 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -57,7 +57,6 @@ use crate::error::{ use crate::failure_detector::PhiAccrualFailureDetectorOptions; use crate::handler::HeartbeatHandlerGroupRef; use crate::lease::lookup_datanode_peer; -use crate::lock::DistLockRef; use crate::procedure::region_migration::manager::RegionMigrationManagerRef; use crate::procedure::ProcedureManagerListenerAdapter; use crate::pubsub::{PublisherRef, SubscriptionManagerRef}; @@ -356,7 +355,6 @@ pub struct Metasrv { flow_selector: SelectorRef, handler_group: HeartbeatHandlerGroupRef, election: Option, - lock: DistLockRef, procedure_manager: ProcedureManagerRef, mailbox: MailboxRef, procedure_executor: ProcedureExecutorRef, @@ -568,10 +566,6 @@ impl Metasrv { self.election.as_ref() } - pub fn lock(&self) -> &DistLockRef { - &self.lock - } - pub fn mailbox(&self) -> &MailboxRef { &self.mailbox } diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 050f825174..db01b3ec9d 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -52,8 +52,6 @@ use crate::handler::{ HeartbeatHandlerGroup, HeartbeatHandlerGroupBuilder, HeartbeatMailbox, Pushers, }; use crate::lease::MetaPeerLookupService; -use crate::lock::memory::MemLock; -use crate::lock::DistLockRef; use crate::metasrv::{ ElectionRef, Metasrv, MetasrvInfo, MetasrvOptions, SelectorContext, SelectorRef, TABLE_ID_SEQ, }; @@ -79,7 +77,6 @@ pub struct MetasrvBuilder { handler_group: Option, election: Option, meta_peer_client: Option, - lock: Option, node_manager: Option, plugins: Option, table_metadata_allocator: Option, @@ -95,7 +92,6 @@ impl MetasrvBuilder { meta_peer_client: None, election: None, options: None, - lock: None, node_manager: None, plugins: None, table_metadata_allocator: None, @@ -137,11 +133,6 @@ impl MetasrvBuilder { self } - pub fn lock(mut self, lock: Option) -> Self { - self.lock = lock; - self - } - pub fn node_manager(mut self, node_manager: NodeManagerRef) -> Self { self.node_manager = Some(node_manager); self @@ -171,7 +162,6 @@ impl MetasrvBuilder { in_memory, selector, handler_group, - lock, node_manager, plugins, table_metadata_allocator, @@ -205,7 +195,6 @@ impl MetasrvBuilder { let flow_metadata_manager = Arc::new(FlowMetadataManager::new( leader_cached_kv_backend.clone() as _, )); - let lock = lock.unwrap_or_else(|| Arc::new(MemLock::default())); let selector_ctx = SelectorContext { server_addr: options.server_addr.clone(), datanode_lease_secs: distributed_time_constants::DATANODE_LEASE_SECS, @@ -384,7 +373,6 @@ impl MetasrvBuilder { flow_selector: Arc::new(RoundRobinSelector::new(SelectTarget::Flownode)), handler_group: Arc::new(handler_group), election, - lock, procedure_manager, mailbox, procedure_executor: ddl_manager, diff --git a/src/meta-srv/src/service.rs b/src/meta-srv/src/service.rs index c3e2e781e8..e260b8b980 100644 --- a/src/meta-srv/src/service.rs +++ b/src/meta-srv/src/service.rs @@ -20,7 +20,6 @@ use tonic::{Response, Status}; pub mod admin; pub mod cluster; mod heartbeat; -pub mod lock; pub mod mailbox; pub mod procedure; pub mod store; diff --git a/src/meta-srv/src/service/lock.rs b/src/meta-srv/src/service/lock.rs deleted file mode 100644 index 4334bdfc37..0000000000 --- a/src/meta-srv/src/service/lock.rs +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright 2023 Greptime Team -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use api::v1::meta::{lock_server, LockRequest, LockResponse, UnlockRequest, UnlockResponse}; -use tonic::{Request, Response}; - -use super::GrpcResult; -use crate::lock::Opts; -use crate::metasrv::Metasrv; - -#[async_trait::async_trait] -impl lock_server::Lock for Metasrv { - async fn lock(&self, request: Request) -> GrpcResult { - let LockRequest { - name, expire_secs, .. - } = request.into_inner(); - let expire_secs = Some(expire_secs as u64); - - let key = self.lock().lock(name, Opts { expire_secs }).await?; - - let resp = LockResponse { - key, - ..Default::default() - }; - - Ok(Response::new(resp)) - } - - async fn unlock(&self, request: Request) -> GrpcResult { - let UnlockRequest { key, .. } = request.into_inner(); - - let _ = self.lock().unlock(key).await?; - - let resp = UnlockResponse { - ..Default::default() - }; - - Ok(Response::new(resp)) - } -} From 2f2b4b306cb480c1396f452f2c27dc2206705d38 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Mon, 14 Oct 2024 11:09:03 +0800 Subject: [PATCH 072/128] feat!: implement interval type by multiple structs (#4772) * define structs and methods Signed-off-by: Ruihang Xia * feat: re-implement interval types in time crate * feat: use new * feat: interval value * feat: query crate interval * feat: pg and mysql interval * chore: remove unused imports * chore: remove commented codes * feat: make flow compile but may not work * feat: flow datetime * test: fix some tests * test: fix some flow tests(WIP) * chore: some fix test&docs * fix: change interval order * chore: remove unused codes * chore: fix cilppy * chore: now signature change * chore: remove todo * feat: update error message --------- Signed-off-by: Ruihang Xia Co-authored-by: Ruihang Xia Co-authored-by: discord9 --- src/api/src/helper.rs | 108 ++- .../function/src/scalars/date/date_add.rs | 126 +-- .../function/src/scalars/date/date_sub.rs | 128 +--- src/common/grpc/src/select.rs | 4 +- src/common/time/src/date.rs | 71 +- src/common/time/src/datetime.rs | 86 ++- src/common/time/src/interval.rs | 724 +++++++----------- src/common/time/src/lib.rs | 2 +- src/common/time/src/timestamp.rs | 98 ++- src/datatypes/src/interval.rs | 84 +- src/datatypes/src/types/interval_type.rs | 10 +- src/datatypes/src/types/primitive_type.rs | 14 +- src/datatypes/src/types/string_type.rs | 10 +- src/datatypes/src/value.rs | 199 +++-- src/datatypes/src/vectors/helper.rs | 7 +- src/flow/src/compute/render/reduce.rs | 41 +- src/flow/src/expr/df_func.rs | 19 +- src/flow/src/expr/func.rs | 171 +++-- src/flow/src/repr.rs | 10 +- src/flow/src/transform.rs | 18 +- src/flow/src/transform/aggr.rs | 120 ++- src/flow/src/transform/literal.rs | 21 +- src/mito2/src/row_converter.rs | 55 +- src/query/src/range_select/plan_rewrite.rs | 117 +-- src/servers/src/mysql/writer.rs | 6 +- src/servers/src/postgres/types.rs | 86 ++- src/servers/src/postgres/types/interval.rs | 50 +- src/sql/src/statements.rs | 4 +- .../standalone/common/range/interval.result | 27 +- .../standalone/common/range/interval.sql | 2 +- 30 files changed, 1190 insertions(+), 1228 deletions(-) diff --git a/src/api/src/helper.rs b/src/api/src/helper.rs index 101cae8802..4281201c42 100644 --- a/src/api/src/helper.rs +++ b/src/api/src/helper.rs @@ -17,10 +17,11 @@ use std::sync::Arc; use common_base::BitVec; use common_decimal::decimal128::{DECIMAL128_DEFAULT_SCALE, DECIMAL128_MAX_PRECISION}; use common_decimal::Decimal128; -use common_time::interval::IntervalUnit; use common_time::time::Time; use common_time::timestamp::TimeUnit; -use common_time::{Date, DateTime, Interval, Timestamp}; +use common_time::{ + Date, DateTime, IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth, Timestamp, +}; use datatypes::prelude::{ConcreteDataType, ValueRef}; use datatypes::scalars::ScalarVector; use datatypes::types::{ @@ -456,13 +457,11 @@ pub fn push_vals(column: &mut Column, origin_count: usize, vector: VectorRef) { TimeUnit::Microsecond => values.time_microsecond_values.push(val.value()), TimeUnit::Nanosecond => values.time_nanosecond_values.push(val.value()), }, - Value::Interval(val) => match val.unit() { - IntervalUnit::YearMonth => values.interval_year_month_values.push(val.to_i32()), - IntervalUnit::DayTime => values.interval_day_time_values.push(val.to_i64()), - IntervalUnit::MonthDayNano => values - .interval_month_day_nano_values - .push(convert_i128_to_interval(val.to_i128())), - }, + Value::IntervalYearMonth(val) => values.interval_year_month_values.push(val.to_i32()), + Value::IntervalDayTime(val) => values.interval_day_time_values.push(val.to_i64()), + Value::IntervalMonthDayNano(val) => values + .interval_month_day_nano_values + .push(convert_month_day_nano_to_pb(val)), Value::Decimal128(val) => values.decimal128_values.push(convert_to_pb_decimal128(val)), Value::List(_) | Value::Duration(_) => unreachable!(), }); @@ -507,14 +506,12 @@ fn ddl_request_type(request: &DdlRequest) -> &'static str { } } -/// Converts an i128 value to google protobuf type [IntervalMonthDayNano]. -pub fn convert_i128_to_interval(v: i128) -> v1::IntervalMonthDayNano { - let interval = Interval::from_i128(v); - let (months, days, nanoseconds) = interval.to_month_day_nano(); +/// Converts an interval to google protobuf type [IntervalMonthDayNano]. +pub fn convert_month_day_nano_to_pb(v: IntervalMonthDayNano) -> v1::IntervalMonthDayNano { v1::IntervalMonthDayNano { - months, - days, - nanoseconds, + months: v.months, + days: v.days, + nanoseconds: v.nanoseconds, } } @@ -562,11 +559,15 @@ pub fn pb_value_to_value_ref<'a>( ValueData::TimeMillisecondValue(t) => ValueRef::Time(Time::new_millisecond(*t)), ValueData::TimeMicrosecondValue(t) => ValueRef::Time(Time::new_microsecond(*t)), ValueData::TimeNanosecondValue(t) => ValueRef::Time(Time::new_nanosecond(*t)), - ValueData::IntervalYearMonthValue(v) => ValueRef::Interval(Interval::from_i32(*v)), - ValueData::IntervalDayTimeValue(v) => ValueRef::Interval(Interval::from_i64(*v)), + ValueData::IntervalYearMonthValue(v) => { + ValueRef::IntervalYearMonth(IntervalYearMonth::from_i32(*v)) + } + ValueData::IntervalDayTimeValue(v) => { + ValueRef::IntervalDayTime(IntervalDayTime::from_i64(*v)) + } ValueData::IntervalMonthDayNanoValue(v) => { - let interval = Interval::from_month_day_nano(v.months, v.days, v.nanoseconds); - ValueRef::Interval(interval) + let interval = IntervalMonthDayNano::new(v.months, v.days, v.nanoseconds); + ValueRef::IntervalMonthDayNano(interval) } ValueData::Decimal128Value(v) => { // get precision and scale from datatype_extension @@ -657,7 +658,7 @@ pub fn pb_values_to_vector_ref(data_type: &ConcreteDataType, values: Values) -> IntervalType::MonthDayNano(_) => { Arc::new(IntervalMonthDayNanoVector::from_iter_values( values.interval_month_day_nano_values.iter().map(|x| { - Interval::from_month_day_nano(x.months, x.days, x.nanoseconds).to_i128() + IntervalMonthDayNano::new(x.months, x.days, x.nanoseconds).to_i128() }), )) } @@ -802,18 +803,18 @@ pub fn pb_values_to_values(data_type: &ConcreteDataType, values: Values) -> Vec< ConcreteDataType::Interval(IntervalType::YearMonth(_)) => values .interval_year_month_values .into_iter() - .map(|v| Value::Interval(Interval::from_i32(v))) + .map(|v| Value::IntervalYearMonth(IntervalYearMonth::from_i32(v))) .collect(), ConcreteDataType::Interval(IntervalType::DayTime(_)) => values .interval_day_time_values .into_iter() - .map(|v| Value::Interval(Interval::from_i64(v))) + .map(|v| Value::IntervalDayTime(IntervalDayTime::from_i64(v))) .collect(), ConcreteDataType::Interval(IntervalType::MonthDayNano(_)) => values .interval_month_day_nano_values .into_iter() .map(|v| { - Value::Interval(Interval::from_month_day_nano( + Value::IntervalMonthDayNano(IntervalMonthDayNano::new( v.months, v.days, v.nanoseconds, @@ -941,18 +942,16 @@ pub fn to_proto_value(value: Value) -> Option { value_data: Some(ValueData::TimeNanosecondValue(v.value())), }, }, - Value::Interval(v) => match v.unit() { - IntervalUnit::YearMonth => v1::Value { - value_data: Some(ValueData::IntervalYearMonthValue(v.to_i32())), - }, - IntervalUnit::DayTime => v1::Value { - value_data: Some(ValueData::IntervalDayTimeValue(v.to_i64())), - }, - IntervalUnit::MonthDayNano => v1::Value { - value_data: Some(ValueData::IntervalMonthDayNanoValue( - convert_i128_to_interval(v.to_i128()), - )), - }, + Value::IntervalYearMonth(v) => v1::Value { + value_data: Some(ValueData::IntervalYearMonthValue(v.to_i32())), + }, + Value::IntervalDayTime(v) => v1::Value { + value_data: Some(ValueData::IntervalDayTimeValue(v.to_i64())), + }, + Value::IntervalMonthDayNano(v) => v1::Value { + value_data: Some(ValueData::IntervalMonthDayNanoValue( + convert_month_day_nano_to_pb(v), + )), }, Value::Decimal128(v) => v1::Value { value_data: Some(ValueData::Decimal128Value(convert_to_pb_decimal128(v))), @@ -1044,13 +1043,11 @@ pub fn value_to_grpc_value(value: Value) -> GrpcValue { TimeUnit::Microsecond => ValueData::TimeMicrosecondValue(v.value()), TimeUnit::Nanosecond => ValueData::TimeNanosecondValue(v.value()), }), - Value::Interval(v) => Some(match v.unit() { - IntervalUnit::YearMonth => ValueData::IntervalYearMonthValue(v.to_i32()), - IntervalUnit::DayTime => ValueData::IntervalDayTimeValue(v.to_i64()), - IntervalUnit::MonthDayNano => { - ValueData::IntervalMonthDayNanoValue(convert_i128_to_interval(v.to_i128())) - } - }), + Value::IntervalYearMonth(v) => Some(ValueData::IntervalYearMonthValue(v.to_i32())), + Value::IntervalDayTime(v) => Some(ValueData::IntervalDayTimeValue(v.to_i64())), + Value::IntervalMonthDayNano(v) => Some(ValueData::IntervalMonthDayNanoValue( + convert_month_day_nano_to_pb(v), + )), Value::Decimal128(v) => Some(ValueData::Decimal128Value(convert_to_pb_decimal128(v))), Value::List(_) | Value::Duration(_) => unreachable!(), }, @@ -1061,6 +1058,7 @@ pub fn value_to_grpc_value(value: Value) -> GrpcValue { mod tests { use std::sync::Arc; + use common_time::interval::IntervalUnit; use datatypes::types::{ Int32Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalYearMonthType, TimeMillisecondType, TimeSecondType, TimestampMillisecondType, TimestampSecondType, @@ -1506,11 +1504,11 @@ mod tests { #[test] fn test_convert_i128_to_interval() { - let i128_val = 3000; - let interval = convert_i128_to_interval(i128_val); + let i128_val = 3; + let interval = convert_month_day_nano_to_pb(IntervalMonthDayNano::from_i128(i128_val)); assert_eq!(interval.months, 0); assert_eq!(interval.days, 0); - assert_eq!(interval.nanoseconds, 3000); + assert_eq!(interval.nanoseconds, 3); } #[test] @@ -1590,9 +1588,9 @@ mod tests { }, ); let expect = vec![ - Value::Interval(Interval::from_year_month(1_i32)), - Value::Interval(Interval::from_year_month(2_i32)), - Value::Interval(Interval::from_year_month(3_i32)), + Value::IntervalYearMonth(IntervalYearMonth::new(1_i32)), + Value::IntervalYearMonth(IntervalYearMonth::new(2_i32)), + Value::IntervalYearMonth(IntervalYearMonth::new(3_i32)), ]; assert_eq!(expect, actual); @@ -1605,9 +1603,9 @@ mod tests { }, ); let expect = vec![ - Value::Interval(Interval::from_i64(1_i64)), - Value::Interval(Interval::from_i64(2_i64)), - Value::Interval(Interval::from_i64(3_i64)), + Value::IntervalDayTime(IntervalDayTime::from_i64(1_i64)), + Value::IntervalDayTime(IntervalDayTime::from_i64(2_i64)), + Value::IntervalDayTime(IntervalDayTime::from_i64(3_i64)), ]; assert_eq!(expect, actual); @@ -1636,9 +1634,9 @@ mod tests { }, ); let expect = vec![ - Value::Interval(Interval::from_month_day_nano(1, 2, 3)), - Value::Interval(Interval::from_month_day_nano(5, 6, 7)), - Value::Interval(Interval::from_month_day_nano(9, 10, 11)), + Value::IntervalMonthDayNano(IntervalMonthDayNano::new(1, 2, 3)), + Value::IntervalMonthDayNano(IntervalMonthDayNano::new(5, 6, 7)), + Value::IntervalMonthDayNano(IntervalMonthDayNano::new(9, 10, 11)), ]; assert_eq!(expect, actual); } diff --git a/src/common/function/src/scalars/date/date_add.rs b/src/common/function/src/scalars/date/date_add.rs index 1052acb868..2307d2caab 100644 --- a/src/common/function/src/scalars/date/date_add.rs +++ b/src/common/function/src/scalars/date/date_add.rs @@ -14,18 +14,19 @@ use std::fmt; -use common_query::error::{InvalidFuncArgsSnafu, Result, UnsupportedInputDataTypeSnafu}; +use common_query::error::{ArrowComputeSnafu, IntoVectorSnafu, InvalidFuncArgsSnafu, Result}; use common_query::prelude::Signature; -use datatypes::data_type::DataType; +use datatypes::arrow::compute::kernels::numeric; use datatypes::prelude::ConcreteDataType; -use datatypes::value::ValueRef; -use datatypes::vectors::VectorRef; -use snafu::ensure; +use datatypes::vectors::{Helper, VectorRef}; +use snafu::{ensure, ResultExt}; use crate::function::{Function, FunctionContext}; use crate::helper; -/// A function adds an interval value to Timestamp, Date or DateTime, and return the result. +/// A function adds an interval value to Timestamp, Date, and return the result. +/// The implementation of datetime type is based on Date64 which is incorrect so this function +/// doesn't support the datetime type. #[derive(Clone, Debug, Default)] pub struct DateAddFunction; @@ -44,7 +45,6 @@ impl Function for DateAddFunction { helper::one_of_sigs2( vec![ ConcreteDataType::date_datatype(), - ConcreteDataType::datetime_datatype(), ConcreteDataType::timestamp_second_datatype(), ConcreteDataType::timestamp_millisecond_datatype(), ConcreteDataType::timestamp_microsecond_datatype(), @@ -69,64 +69,14 @@ impl Function for DateAddFunction { } ); - let left = &columns[0]; - let right = &columns[1]; + let left = columns[0].to_arrow_array(); + let right = columns[1].to_arrow_array(); - let size = left.len(); - let left_datatype = columns[0].data_type(); - match left_datatype { - ConcreteDataType::Timestamp(_) => { - let mut result = left_datatype.create_mutable_vector(size); - for i in 0..size { - let ts = left.get(i).as_timestamp(); - let interval = right.get(i).as_interval(); - - let new_ts = match (ts, interval) { - (Some(ts), Some(interval)) => ts.add_interval(interval), - _ => ts, - }; - - result.push_value_ref(ValueRef::from(new_ts)); - } - - Ok(result.to_vector()) - } - ConcreteDataType::Date(_) => { - let mut result = left_datatype.create_mutable_vector(size); - for i in 0..size { - let date = left.get(i).as_date(); - let interval = right.get(i).as_interval(); - let new_date = match (date, interval) { - (Some(date), Some(interval)) => date.add_interval(interval), - _ => date, - }; - - result.push_value_ref(ValueRef::from(new_date)); - } - - Ok(result.to_vector()) - } - ConcreteDataType::DateTime(_) => { - let mut result = left_datatype.create_mutable_vector(size); - for i in 0..size { - let datetime = left.get(i).as_datetime(); - let interval = right.get(i).as_interval(); - let new_datetime = match (datetime, interval) { - (Some(datetime), Some(interval)) => datetime.add_interval(interval), - _ => datetime, - }; - - result.push_value_ref(ValueRef::from(new_datetime)); - } - - Ok(result.to_vector()) - } - _ => UnsupportedInputDataTypeSnafu { - function: NAME, - datatypes: columns.iter().map(|c| c.data_type()).collect::>(), - } - .fail(), - } + let result = numeric::add(&left, &right).context(ArrowComputeSnafu)?; + let arrow_type = result.data_type().clone(); + Helper::try_into_vector(result).context(IntoVectorSnafu { + data_type: arrow_type, + }) } } @@ -144,8 +94,7 @@ mod tests { use datatypes::prelude::ConcreteDataType; use datatypes::value::Value; use datatypes::vectors::{ - DateTimeVector, DateVector, IntervalDayTimeVector, IntervalYearMonthVector, - TimestampSecondVector, + DateVector, IntervalDayTimeVector, IntervalYearMonthVector, TimestampSecondVector, }; use super::{DateAddFunction, *}; @@ -168,16 +117,15 @@ mod tests { ConcreteDataType::date_datatype(), f.return_type(&[ConcreteDataType::date_datatype()]).unwrap() ); - assert_eq!( - ConcreteDataType::datetime_datatype(), - f.return_type(&[ConcreteDataType::datetime_datatype()]) - .unwrap() - ); - assert!(matches!(f.signature(), + assert!( + matches!(f.signature(), Signature { type_signature: TypeSignature::OneOf(sigs), volatility: Volatility::Immutable - } if sigs.len() == 18)); + } if sigs.len() == 15), + "{:?}", + f.signature() + ); } #[test] @@ -243,36 +191,4 @@ mod tests { } } } - - #[test] - fn test_datetime_date_add() { - let f = DateAddFunction; - - let dates = vec![Some(123), None, Some(42), None]; - // Intervals in months - let intervals = vec![1, 2, 3, 1]; - let results = [Some(2678400123), None, Some(7776000042), None]; - - let date_vector = DateTimeVector::from(dates.clone()); - let interval_vector = IntervalYearMonthVector::from_vec(intervals); - let args: Vec = vec![Arc::new(date_vector), Arc::new(interval_vector)]; - let vector = f.eval(FunctionContext::default(), &args).unwrap(); - - assert_eq!(4, vector.len()); - for (i, _t) in dates.iter().enumerate() { - let v = vector.get(i); - let result = results.get(i).unwrap(); - - if result.is_none() { - assert_eq!(Value::Null, v); - continue; - } - match v { - Value::DateTime(date) => { - assert_eq!(date.val(), result.unwrap()); - } - _ => unreachable!(), - } - } - } } diff --git a/src/common/function/src/scalars/date/date_sub.rs b/src/common/function/src/scalars/date/date_sub.rs index b1f87e880a..1e00db5331 100644 --- a/src/common/function/src/scalars/date/date_sub.rs +++ b/src/common/function/src/scalars/date/date_sub.rs @@ -14,18 +14,19 @@ use std::fmt; -use common_query::error::{InvalidFuncArgsSnafu, Result, UnsupportedInputDataTypeSnafu}; +use common_query::error::{ArrowComputeSnafu, IntoVectorSnafu, InvalidFuncArgsSnafu, Result}; use common_query::prelude::Signature; -use datatypes::data_type::DataType; +use datatypes::arrow::compute::kernels::numeric; use datatypes::prelude::ConcreteDataType; -use datatypes::value::ValueRef; -use datatypes::vectors::VectorRef; -use snafu::ensure; +use datatypes::vectors::{Helper, VectorRef}; +use snafu::{ensure, ResultExt}; use crate::function::{Function, FunctionContext}; use crate::helper; -/// A function subtracts an interval value to Timestamp, Date or DateTime, and return the result. +/// A function subtracts an interval value to Timestamp, Date, and return the result. +/// The implementation of datetime type is based on Date64 which is incorrect so this function +/// doesn't support the datetime type. #[derive(Clone, Debug, Default)] pub struct DateSubFunction; @@ -44,7 +45,6 @@ impl Function for DateSubFunction { helper::one_of_sigs2( vec![ ConcreteDataType::date_datatype(), - ConcreteDataType::datetime_datatype(), ConcreteDataType::timestamp_second_datatype(), ConcreteDataType::timestamp_millisecond_datatype(), ConcreteDataType::timestamp_microsecond_datatype(), @@ -69,65 +69,14 @@ impl Function for DateSubFunction { } ); - let left = &columns[0]; - let right = &columns[1]; + let left = columns[0].to_arrow_array(); + let right = columns[1].to_arrow_array(); - let size = left.len(); - let left_datatype = columns[0].data_type(); - - match left_datatype { - ConcreteDataType::Timestamp(_) => { - let mut result = left_datatype.create_mutable_vector(size); - for i in 0..size { - let ts = left.get(i).as_timestamp(); - let interval = right.get(i).as_interval(); - - let new_ts = match (ts, interval) { - (Some(ts), Some(interval)) => ts.sub_interval(interval), - _ => ts, - }; - - result.push_value_ref(ValueRef::from(new_ts)); - } - - Ok(result.to_vector()) - } - ConcreteDataType::Date(_) => { - let mut result = left_datatype.create_mutable_vector(size); - for i in 0..size { - let date = left.get(i).as_date(); - let interval = right.get(i).as_interval(); - let new_date = match (date, interval) { - (Some(date), Some(interval)) => date.sub_interval(interval), - _ => date, - }; - - result.push_value_ref(ValueRef::from(new_date)); - } - - Ok(result.to_vector()) - } - ConcreteDataType::DateTime(_) => { - let mut result = left_datatype.create_mutable_vector(size); - for i in 0..size { - let datetime = left.get(i).as_datetime(); - let interval = right.get(i).as_interval(); - let new_datetime = match (datetime, interval) { - (Some(datetime), Some(interval)) => datetime.sub_interval(interval), - _ => datetime, - }; - - result.push_value_ref(ValueRef::from(new_datetime)); - } - - Ok(result.to_vector()) - } - _ => UnsupportedInputDataTypeSnafu { - function: NAME, - datatypes: columns.iter().map(|c| c.data_type()).collect::>(), - } - .fail(), - } + let result = numeric::sub(&left, &right).context(ArrowComputeSnafu)?; + let arrow_type = result.data_type().clone(); + Helper::try_into_vector(result).context(IntoVectorSnafu { + data_type: arrow_type, + }) } } @@ -145,8 +94,7 @@ mod tests { use datatypes::prelude::ConcreteDataType; use datatypes::value::Value; use datatypes::vectors::{ - DateTimeVector, DateVector, IntervalDayTimeVector, IntervalYearMonthVector, - TimestampSecondVector, + DateVector, IntervalDayTimeVector, IntervalYearMonthVector, TimestampSecondVector, }; use super::{DateSubFunction, *}; @@ -174,11 +122,15 @@ mod tests { f.return_type(&[ConcreteDataType::datetime_datatype()]) .unwrap() ); - assert!(matches!(f.signature(), + assert!( + matches!(f.signature(), Signature { type_signature: TypeSignature::OneOf(sigs), volatility: Volatility::Immutable - } if sigs.len() == 18)); + } if sigs.len() == 15), + "{:?}", + f.signature() + ); } #[test] @@ -250,42 +202,4 @@ mod tests { } } } - - #[test] - fn test_datetime_date_sub() { - let f = DateSubFunction; - let millis_per_month = 3600 * 24 * 30 * 1000; - - let dates = vec![ - Some(123 * millis_per_month), - None, - Some(42 * millis_per_month), - None, - ]; - // Intervals in months - let intervals = vec![1, 2, 3, 1]; - let results = [Some(316137600000), None, Some(100915200000), None]; - - let date_vector = DateTimeVector::from(dates.clone()); - let interval_vector = IntervalYearMonthVector::from_vec(intervals); - let args: Vec = vec![Arc::new(date_vector), Arc::new(interval_vector)]; - let vector = f.eval(FunctionContext::default(), &args).unwrap(); - - assert_eq!(4, vector.len()); - for (i, _t) in dates.iter().enumerate() { - let v = vector.get(i); - let result = results.get(i).unwrap(); - - if result.is_none() { - assert_eq!(Value::Null, v); - continue; - } - match v { - Value::DateTime(date) => { - assert_eq!(date.val(), result.unwrap()); - } - _ => unreachable!(), - } - } - } } diff --git a/src/common/grpc/src/select.rs b/src/common/grpc/src/select.rs index ba13acf3b7..493893f49d 100644 --- a/src/common/grpc/src/select.rs +++ b/src/common/grpc/src/select.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use api::helper::{convert_i128_to_interval, convert_to_pb_decimal128}; +use api::helper::{convert_month_day_nano_to_pb, convert_to_pb_decimal128}; use api::v1::column::Values; use common_base::BitVec; use datatypes::types::{IntervalType, TimeType, TimestampType, WrapperType}; @@ -211,7 +211,7 @@ pub fn values(arrays: &[VectorRef]) -> Result { ConcreteDataType::Interval(IntervalType::MonthDayNano(_)), IntervalMonthDayNanoVector, interval_month_day_nano_values, - |x| { convert_i128_to_interval(x.into_native()) } + |x| { convert_month_day_nano_to_pb(x) } ), ( ConcreteDataType::Decimal128(_), diff --git a/src/common/time/src/date.rs b/src/common/time/src/date.rs index 86759d737d..4274913925 100644 --- a/src/common/time/src/date.rs +++ b/src/common/time/src/date.rs @@ -14,13 +14,13 @@ use std::fmt::{Display, Formatter, Write}; -use chrono::{Datelike, Days, LocalResult, Months, NaiveDate, NaiveTime, TimeZone}; +use chrono::{Datelike, Days, LocalResult, Months, NaiveDate, NaiveTime, TimeDelta, TimeZone}; use serde::{Deserialize, Serialize}; use serde_json::Value; use snafu::ResultExt; use crate::error::{InvalidDateStrSnafu, ParseDateStrSnafu, Result}; -use crate::interval::Interval; +use crate::interval::{IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth}; use crate::timezone::get_timezone; use crate::util::datetime_to_utc; use crate::Timezone; @@ -134,29 +134,64 @@ impl Date { (self.0 as i64) * 24 * 3600 } - /// Adds given Interval to the current date. - /// Returns None if the resulting date would be out of range. - pub fn add_interval(&self, interval: Interval) -> Option { + // FIXME(yingwen): remove add/sub intervals later + /// Adds given [IntervalYearMonth] to the current date. + pub fn add_year_month(&self, interval: IntervalYearMonth) -> Option { let naive_date = self.to_chrono_date()?; - let (months, days, _) = interval.to_month_day_nano(); - naive_date - .checked_add_months(Months::new(months as u32))? - .checked_add_days(Days::new(days as u64)) + .checked_add_months(Months::new(interval.months as u32)) .map(Into::into) } - /// Subtracts given Interval to the current date. - /// Returns None if the resulting date would be out of range. - pub fn sub_interval(&self, interval: Interval) -> Option { + /// Adds given [IntervalDayTime] to the current date. + pub fn add_day_time(&self, interval: IntervalDayTime) -> Option { let naive_date = self.to_chrono_date()?; - let (months, days, _) = interval.to_month_day_nano(); + naive_date + .checked_add_days(Days::new(interval.days as u64))? + .checked_add_signed(TimeDelta::milliseconds(interval.milliseconds as i64)) + .map(Into::into) + } + + /// Adds given [IntervalMonthDayNano] to the current date. + pub fn add_month_day_nano(&self, interval: IntervalMonthDayNano) -> Option { + let naive_date = self.to_chrono_date()?; naive_date - .checked_sub_months(Months::new(months as u32))? - .checked_sub_days(Days::new(days as u64)) + .checked_add_months(Months::new(interval.months as u32))? + .checked_add_days(Days::new(interval.days as u64))? + .checked_add_signed(TimeDelta::nanoseconds(interval.nanoseconds)) + .map(Into::into) + } + + /// Subtracts given [IntervalYearMonth] to the current date. + pub fn sub_year_month(&self, interval: IntervalYearMonth) -> Option { + let naive_date = self.to_chrono_date()?; + + naive_date + .checked_sub_months(Months::new(interval.months as u32)) + .map(Into::into) + } + + /// Subtracts given [IntervalDayTime] to the current date. + pub fn sub_day_time(&self, interval: IntervalDayTime) -> Option { + let naive_date = self.to_chrono_date()?; + + naive_date + .checked_sub_days(Days::new(interval.days as u64))? + .checked_sub_signed(TimeDelta::milliseconds(interval.milliseconds as i64)) + .map(Into::into) + } + + /// Subtracts given [IntervalMonthDayNano] to the current date. + pub fn sub_month_day_nano(&self, interval: IntervalMonthDayNano) -> Option { + let naive_date = self.to_chrono_date()?; + + naive_date + .checked_sub_months(Months::new(interval.months as u32))? + .checked_sub_days(Days::new(interval.days as u64))? + .checked_sub_signed(TimeDelta::nanoseconds(interval.nanoseconds)) .map(Into::into) } @@ -246,12 +281,12 @@ mod tests { fn test_add_sub_interval() { let date = Date::new(1000); - let interval = Interval::from_year_month(3); + let interval = IntervalYearMonth::new(3); - let new_date = date.add_interval(interval).unwrap(); + let new_date = date.add_year_month(interval).unwrap(); assert_eq!(new_date.val(), 1091); - assert_eq!(date, new_date.sub_interval(interval).unwrap()); + assert_eq!(date, new_date.sub_year_month(interval).unwrap()); } #[test] diff --git a/src/common/time/src/datetime.rs b/src/common/time/src/datetime.rs index 4a60470aeb..abeae4908a 100644 --- a/src/common/time/src/datetime.rs +++ b/src/common/time/src/datetime.rs @@ -13,16 +13,18 @@ // limitations under the License. use std::fmt::{Display, Formatter, Write}; -use std::time::Duration; -use chrono::{Days, LocalResult, Months, NaiveDateTime, TimeZone as ChronoTimeZone, Utc}; +use chrono::{ + Days, LocalResult, Months, NaiveDateTime, TimeDelta, TimeZone as ChronoTimeZone, Utc, +}; use serde::{Deserialize, Serialize}; use snafu::ResultExt; use crate::error::{InvalidDateStrSnafu, Result}; +use crate::interval::{IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth}; use crate::timezone::{get_timezone, Timezone}; use crate::util::{datetime_to_utc, format_utc_datetime}; -use crate::{Date, Interval}; +use crate::Date; const DATETIME_FORMAT: &str = "%F %H:%M:%S%.f"; const DATETIME_FORMAT_WITH_TZ: &str = "%F %H:%M:%S%.f%z"; @@ -160,32 +162,66 @@ impl DateTime { None => Utc.from_utc_datetime(&v).naive_local(), }) } - /// Adds given Interval to the current datetime. - /// Returns None if the resulting datetime would be out of range. - pub fn add_interval(&self, interval: Interval) -> Option { + + // FIXME(yingwen): remove add/sub intervals later + /// Adds given [IntervalYearMonth] to the current datetime. + pub fn add_year_month(&self, interval: IntervalYearMonth) -> Option { let naive_datetime = self.to_chrono_datetime()?; - let (months, days, nsecs) = interval.to_month_day_nano(); - let naive_datetime = naive_datetime - .checked_add_months(Months::new(months as u32))? - .checked_add_days(Days::new(days as u64))? - + Duration::from_nanos(nsecs as u64); - - Some(naive_datetime.into()) + naive_datetime + .checked_add_months(Months::new(interval.months as u32)) + .map(Into::into) } - /// Subtracts given Interval to the current datetime. - /// Returns None if the resulting datetime would be out of range. - pub fn sub_interval(&self, interval: Interval) -> Option { + /// Adds given [IntervalDayTime] to the current datetime. + pub fn add_day_time(&self, interval: IntervalDayTime) -> Option { let naive_datetime = self.to_chrono_datetime()?; - let (months, days, nsecs) = interval.to_month_day_nano(); - let naive_datetime = naive_datetime - .checked_sub_months(Months::new(months as u32))? - .checked_sub_days(Days::new(days as u64))? - - Duration::from_nanos(nsecs as u64); + naive_datetime + .checked_add_days(Days::new(interval.days as u64))? + .checked_add_signed(TimeDelta::milliseconds(interval.milliseconds as i64)) + .map(Into::into) + } - Some(naive_datetime.into()) + /// Adds given [IntervalMonthDayNano] to the current datetime. + pub fn add_month_day_nano(&self, interval: IntervalMonthDayNano) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + naive_datetime + .checked_add_months(Months::new(interval.months as u32))? + .checked_add_days(Days::new(interval.days as u64))? + .checked_add_signed(TimeDelta::nanoseconds(interval.nanoseconds)) + .map(Into::into) + } + + /// Subtracts given [IntervalYearMonth] to the current datetime. + pub fn sub_year_month(&self, interval: IntervalYearMonth) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + naive_datetime + .checked_sub_months(Months::new(interval.months as u32)) + .map(Into::into) + } + + /// Subtracts given [IntervalDayTime] to the current datetime. + pub fn sub_day_time(&self, interval: IntervalDayTime) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + naive_datetime + .checked_sub_days(Days::new(interval.days as u64))? + .checked_sub_signed(TimeDelta::milliseconds(interval.milliseconds as i64)) + .map(Into::into) + } + + /// Subtracts given [IntervalMonthDayNano] to the current datetime. + pub fn sub_month_day_nano(&self, interval: IntervalMonthDayNano) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + naive_datetime + .checked_sub_months(Months::new(interval.months as u32))? + .checked_sub_days(Days::new(interval.days as u64))? + .checked_sub_signed(TimeDelta::nanoseconds(interval.nanoseconds)) + .map(Into::into) } /// Convert to [common_time::date]. @@ -231,12 +267,12 @@ mod tests { fn test_add_sub_interval() { let datetime = DateTime::new(1000); - let interval = Interval::from_day_time(1, 200); + let interval = IntervalDayTime::new(1, 200); - let new_datetime = datetime.add_interval(interval).unwrap(); + let new_datetime = datetime.add_day_time(interval).unwrap(); assert_eq!(new_datetime.val(), 1000 + 3600 * 24 * 1000 + 200); - assert_eq!(datetime, new_datetime.sub_interval(interval).unwrap()); + assert_eq!(datetime, new_datetime.sub_day_time(interval).unwrap()); } #[test] diff --git a/src/common/time/src/interval.rs b/src/common/time/src/interval.rs index cd57028d29..0ca40f7d79 100644 --- a/src/common/time/src/interval.rs +++ b/src/common/time/src/interval.rs @@ -12,18 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::Ordering; -use std::default::Default; -use std::fmt::{self, Display, Formatter, Write}; -use std::hash::{Hash, Hasher}; +use std::hash::Hash; use arrow::datatypes::IntervalUnit as ArrowIntervalUnit; use serde::{Deserialize, Serialize}; -use serde_json::Value; -use snafu::ResultExt; - -use crate::duration::Duration; -use crate::error::{Result, TimestampOverflowSnafu}; #[derive( Debug, Default, Copy, Clone, PartialEq, Eq, Hash, PartialOrd, Ord, Serialize, Deserialize, @@ -61,268 +53,269 @@ impl From for IntervalUnit { } } -/// Interval Type represents a period of time. -/// It is composed of months, days and nanoseconds. -/// 3 kinds of interval are supported: year-month, day-time and -/// month-day-nano, which will be stored in the following format. -/// Interval data format: -/// | months | days | nsecs | -/// | 4bytes | 4bytes | 8bytes | -#[derive(Debug, Clone, Default, Copy, Serialize, Deserialize)] -pub struct Interval { - months: i32, - days: i32, - nsecs: i64, - unit: IntervalUnit, +// The `Value` type requires Serialize, Deserialize. +#[derive( + Debug, Default, Copy, Clone, Eq, PartialEq, Hash, Ord, PartialOrd, Serialize, Deserialize, +)] +#[repr(C)] +pub struct IntervalYearMonth { + /// Number of months + pub months: i32, } -// Nanosecond convert to other time unit -pub const NANOS_PER_SEC: i64 = 1_000_000_000; -pub const NANOS_PER_MILLI: i64 = 1_000_000; -pub const NANOS_PER_MICRO: i64 = 1_000; -pub const NANOS_PER_HOUR: i64 = 60 * 60 * NANOS_PER_SEC; -pub const NANOS_PER_DAY: i64 = 24 * NANOS_PER_HOUR; -pub const NANOS_PER_MONTH: i64 = 30 * NANOS_PER_DAY; - -pub const DAYS_PER_MONTH: i64 = 30; - -impl Interval { - /// Creates a new interval from months, days and nanoseconds. - /// Precision is nanosecond. - pub fn from_month_day_nano(months: i32, days: i32, nsecs: i64) -> Self { - Interval { - months, - days, - nsecs, - unit: IntervalUnit::MonthDayNano, - } - } - - /// Creates a new interval from months. - pub fn from_year_month(months: i32) -> Self { - Interval { - months, - days: 0, - nsecs: 0, - unit: IntervalUnit::YearMonth, - } - } - - /// Creates a new interval from days and milliseconds. - pub fn from_day_time(days: i32, millis: i32) -> Self { - Interval { - months: 0, - days, - nsecs: (millis as i64) * NANOS_PER_MILLI, - unit: IntervalUnit::DayTime, - } - } - - pub fn to_duration(&self) -> Result { - Ok(Duration::new_nanosecond( - self.to_nanosecond() - .try_into() - .context(TimestampOverflowSnafu)?, - )) - } - - /// Return a tuple(months, days, nanoseconds) from the interval. - pub fn to_month_day_nano(&self) -> (i32, i32, i64) { - (self.months, self.days, self.nsecs) - } - - /// Converts the interval to nanoseconds. - pub fn to_nanosecond(&self) -> i128 { - let days = (self.days as i64) + DAYS_PER_MONTH * (self.months as i64); - (self.nsecs as i128) + (NANOS_PER_DAY as i128) * (days as i128) - } - - /// Smallest interval value. - pub const MIN: Self = Self { - months: i32::MIN, - days: i32::MIN, - nsecs: i64::MIN, - unit: IntervalUnit::MonthDayNano, - }; - - /// Largest interval value. - pub const MAX: Self = Self { - months: i32::MAX, - days: i32::MAX, - nsecs: i64::MAX, - unit: IntervalUnit::MonthDayNano, - }; - - /// Returns the justified interval. - /// allows you to adjust the interval of 30-day as one month and the interval of 24-hour as one day - pub fn justified_interval(&self) -> Self { - let mut result = *self; - let extra_months_d = self.days as i64 / DAYS_PER_MONTH; - let extra_months_nsecs = self.nsecs / NANOS_PER_MONTH; - result.days -= (extra_months_d * DAYS_PER_MONTH) as i32; - result.nsecs -= extra_months_nsecs * NANOS_PER_MONTH; - - let extra_days = self.nsecs / NANOS_PER_DAY; - result.nsecs -= extra_days * NANOS_PER_DAY; - - result.months += extra_months_d as i32 + extra_months_nsecs as i32; - result.days += extra_days as i32; - - result - } - - /// Convert Interval to nanoseconds, - /// to check whether Interval is positive - pub fn is_positive(&self) -> bool { - self.to_nanosecond() > 0 - } - - /// is_zero - pub fn is_zero(&self) -> bool { - self.months == 0 && self.days == 0 && self.nsecs == 0 - } - - /// get unit - pub fn unit(&self) -> IntervalUnit { - self.unit - } - - /// Multiple Interval by an integer with overflow check. - /// Returns justified Interval, or `None` if overflow occurred. - pub fn checked_mul_int(&self, rhs: I) -> Option - where - I: TryInto, - { - let rhs = rhs.try_into().ok()?; - let months = self.months.checked_mul(rhs)?; - let days = self.days.checked_mul(rhs)?; - let nsecs = self.nsecs.checked_mul(rhs as i64)?; - - Some( - Self { - months, - days, - nsecs, - unit: self.unit, - } - .justified_interval(), - ) - } - - /// Convert Interval to ISO 8601 string - pub fn to_iso8601_string(self) -> String { - IntervalFormat::from(self).to_iso8601_string() - } - - /// Convert Interval to postgres verbose string - pub fn to_postgres_string(self) -> String { - IntervalFormat::from(self).to_postgres_string() - } - - /// Convert Interval to sql_standard string - pub fn to_sql_standard_string(self) -> String { - IntervalFormat::from(self).to_sql_standard_string() - } - - /// Interval Type and i128 [IntervalUnit::MonthDayNano] Convert - /// v consists of months(i32) | days(i32) | nsecs(i64) - pub fn from_i128(v: i128) -> Self { - Interval { - nsecs: v as i64, - days: (v >> 64) as i32, - months: (v >> 96) as i32, - unit: IntervalUnit::MonthDayNano, - } - } - - /// `Interval` Type and i64 [IntervalUnit::DayTime] Convert - /// v consists of days(i32) | milliseconds(i32) - pub fn from_i64(v: i64) -> Self { - Interval { - nsecs: ((v as i32) as i64) * NANOS_PER_MILLI, - days: (v >> 32) as i32, - months: 0, - unit: IntervalUnit::DayTime, - } - } - - /// `Interval` Type and i32 [IntervalUnit::YearMonth] Convert - /// v consists of months(i32) - pub fn from_i32(v: i32) -> Self { - Interval { - nsecs: 0, - days: 0, - months: v, - unit: IntervalUnit::YearMonth, - } - } - - pub fn to_i128(&self) -> i128 { - // 128 96 64 0 - // +-------+-------+-------+-------+-------+-------+-------+-------+ - // | months | days | nanoseconds | - // +-------+-------+-------+-------+-------+-------+-------+-------+ - let months = (self.months as u128 & u32::MAX as u128) << 96; - let days = (self.days as u128 & u32::MAX as u128) << 64; - let nsecs = self.nsecs as u128 & u64::MAX as u128; - (months | days | nsecs) as i128 - } - - pub fn to_i64(&self) -> i64 { - // 64 32 0 - // +-------+-------+-------+-------+-------+-------+-------+-------+ - // | days | milliseconds | - // +-------+-------+-------+-------+-------+-------+-------+-------+ - let days = (self.days as u64 & u32::MAX as u64) << 32; - let milliseconds = (self.nsecs / NANOS_PER_MILLI) as u64 & u32::MAX as u64; - (days | milliseconds) as i64 +impl IntervalYearMonth { + pub fn new(months: i32) -> Self { + Self { months } } pub fn to_i32(&self) -> i32 { self.months } + pub fn from_i32(months: i32) -> Self { + Self { months } + } + pub fn negative(&self) -> Self { - Self { - months: -self.months, - days: -self.days, - nsecs: -self.nsecs, - unit: self.unit, + Self::new(-self.months) + } + + pub fn to_iso8601_string(&self) -> String { + IntervalFormat::from(*self).to_iso8601_string() + } +} + +impl From for IntervalFormat { + fn from(interval: IntervalYearMonth) -> Self { + IntervalFormat { + years: interval.months / 12, + months: interval.months % 12, + ..Default::default() } } } -impl From for Interval { +impl From for IntervalYearMonth { + fn from(v: i32) -> Self { + Self::from_i32(v) + } +} + +impl From for i32 { + fn from(v: IntervalYearMonth) -> Self { + v.to_i32() + } +} + +impl From for serde_json::Value { + fn from(v: IntervalYearMonth) -> Self { + serde_json::Value::from(v.to_i32()) + } +} + +#[derive( + Debug, Default, Copy, Clone, Eq, PartialEq, Hash, Ord, PartialOrd, Serialize, Deserialize, +)] +#[repr(C)] +pub struct IntervalDayTime { + /// Number of days + pub days: i32, + /// Number of milliseconds + pub milliseconds: i32, +} + +impl IntervalDayTime { + /// The additive identity i.e. `0`. + pub const ZERO: Self = Self::new(0, 0); + + /// The multiplicative inverse, i.e. `-1`. + pub const MINUS_ONE: Self = Self::new(-1, -1); + + /// The maximum value that can be represented + pub const MAX: Self = Self::new(i32::MAX, i32::MAX); + + /// The minimum value that can be represented + pub const MIN: Self = Self::new(i32::MIN, i32::MIN); + + pub const fn new(days: i32, milliseconds: i32) -> Self { + Self { days, milliseconds } + } + + pub fn to_i64(&self) -> i64 { + let d = (self.days as u64 & u32::MAX as u64) << 32; + let m = self.milliseconds as u64 & u32::MAX as u64; + (d | m) as i64 + } + + pub fn from_i64(value: i64) -> Self { + let days = (value >> 32) as i32; + let milliseconds = value as i32; + Self { days, milliseconds } + } + + pub fn negative(&self) -> Self { + Self::new(-self.days, -self.milliseconds) + } + + pub fn to_iso8601_string(&self) -> String { + IntervalFormat::from(*self).to_iso8601_string() + } + + pub fn as_millis(&self) -> i64 { + self.days as i64 * MS_PER_DAY + self.milliseconds as i64 + } +} + +impl From for IntervalDayTime { + fn from(v: i64) -> Self { + Self::from_i64(v) + } +} + +impl From for i64 { + fn from(v: IntervalDayTime) -> Self { + v.to_i64() + } +} + +impl From for serde_json::Value { + fn from(v: IntervalDayTime) -> Self { + serde_json::Value::from(v.to_i64()) + } +} + +// Millisecond convert to other time unit +pub const MS_PER_SEC: i64 = 1_000; +pub const MS_PER_MINUTE: i64 = 60 * MS_PER_SEC; +pub const MS_PER_HOUR: i64 = 60 * MS_PER_MINUTE; +pub const MS_PER_DAY: i64 = 24 * MS_PER_HOUR; +pub const NANOS_PER_MILLI: i64 = 1_000_000; + +impl From for IntervalFormat { + fn from(interval: IntervalDayTime) -> Self { + IntervalFormat { + days: interval.days, + hours: interval.milliseconds as i64 / MS_PER_HOUR, + minutes: (interval.milliseconds as i64 % MS_PER_HOUR) / MS_PER_MINUTE, + seconds: (interval.milliseconds as i64 % MS_PER_MINUTE) / MS_PER_SEC, + microseconds: (interval.milliseconds as i64 % MS_PER_SEC) * MS_PER_SEC, + ..Default::default() + } + } +} + +#[derive( + Debug, Default, Copy, Clone, Eq, PartialEq, Hash, Ord, PartialOrd, Serialize, Deserialize, +)] +#[repr(C)] +pub struct IntervalMonthDayNano { + /// Number of months + pub months: i32, + /// Number of days + pub days: i32, + /// Number of nanoseconds + pub nanoseconds: i64, +} + +impl IntervalMonthDayNano { + /// The additive identity i.e. `0`. + pub const ZERO: Self = Self::new(0, 0, 0); + + /// The multiplicative inverse, i.e. `-1`. + pub const MINUS_ONE: Self = Self::new(-1, -1, -1); + + /// The maximum value that can be represented + pub const MAX: Self = Self::new(i32::MAX, i32::MAX, i64::MAX); + + /// The minimum value that can be represented + pub const MIN: Self = Self::new(i32::MIN, i32::MIN, i64::MIN); + + pub const fn new(months: i32, days: i32, nanoseconds: i64) -> Self { + Self { + months, + days, + nanoseconds, + } + } + + pub fn to_i128(&self) -> i128 { + let m = (self.months as u128 & u32::MAX as u128) << 96; + let d = (self.days as u128 & u32::MAX as u128) << 64; + let n = self.nanoseconds as u128 & u64::MAX as u128; + (m | d | n) as i128 + } + + pub fn from_i128(value: i128) -> Self { + let months = (value >> 96) as i32; + let days = (value >> 64) as i32; + let nanoseconds = value as i64; + Self { + months, + days, + nanoseconds, + } + } + + pub fn negative(&self) -> Self { + Self::new(-self.months, -self.days, -self.nanoseconds) + } + + pub fn to_iso8601_string(&self) -> String { + IntervalFormat::from(*self).to_iso8601_string() + } +} + +impl From for IntervalMonthDayNano { fn from(v: i128) -> Self { Self::from_i128(v) } } -impl From for i128 { - fn from(v: Interval) -> Self { +impl From for i128 { + fn from(v: IntervalMonthDayNano) -> Self { v.to_i128() } } -impl From for serde_json::Value { - fn from(v: Interval) -> Self { - Value::String(v.to_string()) +impl From for serde_json::Value { + fn from(v: IntervalMonthDayNano) -> Self { + serde_json::Value::from(v.to_i128().to_string()) } } -impl Display for Interval { - fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - let mut s = String::new(); - if self.months != 0 { - write!(s, "{} months ", self.months)?; +// Nanosecond convert to other time unit +pub const NS_PER_SEC: i64 = 1_000_000_000; +pub const NS_PER_MINUTE: i64 = 60 * NS_PER_SEC; +pub const NS_PER_HOUR: i64 = 60 * NS_PER_MINUTE; +pub const NS_PER_DAY: i64 = 24 * NS_PER_HOUR; + +impl From for IntervalFormat { + fn from(interval: IntervalMonthDayNano) -> Self { + IntervalFormat { + years: interval.months / 12, + months: interval.months % 12, + days: interval.days, + hours: interval.nanoseconds / NS_PER_HOUR, + minutes: (interval.nanoseconds % NS_PER_HOUR) / NS_PER_MINUTE, + seconds: (interval.nanoseconds % NS_PER_MINUTE) / NS_PER_SEC, + microseconds: (interval.nanoseconds % NS_PER_SEC) / 1_000, } - if self.days != 0 { - write!(s, "{} days ", self.days)?; - } - if self.nsecs != 0 { - write!(s, "{} nsecs", self.nsecs)?; - } - write!(f, "{}", s.trim()) + } +} + +pub fn interval_year_month_to_month_day_nano(interval: IntervalYearMonth) -> IntervalMonthDayNano { + IntervalMonthDayNano { + months: interval.months, + days: 0, + nanoseconds: 0, + } +} + +pub fn interval_day_time_to_month_day_nano(interval: IntervalDayTime) -> IntervalMonthDayNano { + IntervalMonthDayNano { + months: 0, + days: interval.days, + nanoseconds: interval.milliseconds as i64 * NANOS_PER_MILLI, } } @@ -339,31 +332,6 @@ pub struct IntervalFormat { pub microseconds: i64, } -impl From for IntervalFormat { - fn from(val: Interval) -> IntervalFormat { - let months = val.months; - let days = val.days; - let microseconds = val.nsecs / NANOS_PER_MICRO; - let years = (months - (months % 12)) / 12; - let months = months - years * 12; - let hours = (microseconds - (microseconds % 3_600_000_000)) / 3_600_000_000; - let microseconds = microseconds - hours * 3_600_000_000; - let minutes = (microseconds - (microseconds % 60_000_000)) / 60_000_000; - let microseconds = microseconds - minutes * 60_000_000; - let seconds = (microseconds - (microseconds % 1_000_000)) / 1_000_000; - let microseconds = microseconds - seconds * 1_000_000; - IntervalFormat { - years, - months, - days, - hours, - minutes, - seconds, - microseconds, - } - } -} - impl IntervalFormat { /// All the field in the interval is 0 pub fn is_zero(&self) -> bool { @@ -540,117 +508,37 @@ fn get_time_part( interval } -/// IntervalCompare is used to compare two intervals -/// It makes interval into nanoseconds style. -#[derive(PartialEq, Eq, Hash, PartialOrd, Ord)] -struct IntervalCompare(i128); - -impl From for IntervalCompare { - fn from(interval: Interval) -> Self { - Self(interval.to_nanosecond()) - } -} - -impl Ord for Interval { - fn cmp(&self, other: &Self) -> Ordering { - IntervalCompare::from(*self).cmp(&IntervalCompare::from(*other)) - } -} - -impl PartialOrd for Interval { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Eq for Interval {} - -impl PartialEq for Interval { - fn eq(&self, other: &Self) -> bool { - self.cmp(other).is_eq() - } -} - -impl Hash for Interval { - fn hash(&self, state: &mut H) { - IntervalCompare::from(*self).hash(state) - } -} - #[cfg(test)] mod tests { - use std::collections::HashMap; - use super::*; - use crate::timestamp::TimeUnit; #[test] fn test_from_year_month() { - let interval = Interval::from_year_month(1); + let interval = IntervalYearMonth::new(1); assert_eq!(interval.months, 1); } #[test] fn test_from_date_time() { - let interval = Interval::from_day_time(1, 2); + let interval = IntervalDayTime::new(1, 2); assert_eq!(interval.days, 1); - assert_eq!(interval.nsecs, 2_000_000); + assert_eq!(interval.milliseconds, 2); } #[test] - fn test_to_duration() { - let interval = Interval::from_day_time(1, 2); - - let duration = interval.to_duration().unwrap(); - assert_eq!(86400002000000, duration.value()); - assert_eq!(TimeUnit::Nanosecond, duration.unit()); - - let interval = Interval::from_year_month(12); - - let duration = interval.to_duration().unwrap(); - assert_eq!(31104000000000000, duration.value()); - assert_eq!(TimeUnit::Nanosecond, duration.unit()); - } - - #[test] - fn test_interval_is_positive() { - let interval = Interval::from_year_month(1); - assert!(interval.is_positive()); - let interval = Interval::from_year_month(-1); - assert!(!interval.is_positive()); - - let interval = Interval::from_day_time(1, i32::MIN); - assert!(!interval.is_positive()); - } - - #[test] - fn test_to_nanosecond() { - let interval = Interval::from_year_month(1); - assert_eq!(interval.to_nanosecond(), 2592000000000000); - let interval = Interval::from_day_time(1, 2); - assert_eq!(interval.to_nanosecond(), 86400002000000); - - let max_interval = Interval::from_month_day_nano(i32::MAX, i32::MAX, i64::MAX); - assert_eq!(max_interval.to_nanosecond(), 5751829423496836854775807); - - let min_interval = Interval::from_month_day_nano(i32::MIN, i32::MIN, i64::MIN); - assert_eq!(min_interval.to_nanosecond(), -5751829426175236854775808); - } - - #[test] - fn test_interval_is_zero() { - let interval = Interval::from_month_day_nano(1, 1, 1); - assert!(!interval.is_zero()); - let interval = Interval::from_month_day_nano(0, 0, 0); - assert!(interval.is_zero()); + fn test_from_month_day_nano() { + let interval = IntervalMonthDayNano::new(1, 2, 3); + assert_eq!(interval.months, 1); + assert_eq!(interval.days, 2); + assert_eq!(interval.nanoseconds, 3); } #[test] fn test_interval_i128_convert() { let test_interval_eq = |month, day, nano| { - let interval = Interval::from_month_day_nano(month, day, nano); + let interval = IntervalMonthDayNano::new(month, day, nano); let interval_i128 = interval.to_i128(); - let interval2 = Interval::from_i128(interval_i128); + let interval2 = IntervalMonthDayNano::from_i128(interval_i128); assert_eq!(interval, interval2); }; @@ -666,11 +554,26 @@ mod tests { test_interval_eq(i32::MAX, i32::MIN, i64::MIN); test_interval_eq(i32::MIN, i32::MAX, i64::MIN); test_interval_eq(i32::MIN, i32::MIN, i64::MIN); + + let interval = IntervalMonthDayNano::from_i128(1); + assert_eq!(interval, IntervalMonthDayNano::new(0, 0, 1)); + assert_eq!(1, IntervalMonthDayNano::new(0, 0, 1).to_i128()); + } + + #[test] + fn test_interval_i64_convert() { + let interval = IntervalDayTime::from_i64(1); + assert_eq!(interval, IntervalDayTime::new(0, 1)); + assert_eq!(1, IntervalDayTime::new(0, 1).to_i64()); } #[test] fn test_convert_interval_format() { - let interval = Interval::from_month_day_nano(14, 160, 1000000); + let interval = IntervalMonthDayNano { + months: 14, + days: 160, + nanoseconds: 1000000, + }; let interval_format = IntervalFormat::from(interval); assert_eq!(interval_format.years, 1); assert_eq!(interval_format.months, 2); @@ -681,94 +584,34 @@ mod tests { assert_eq!(interval_format.microseconds, 1000); } - #[test] - fn test_interval_hash() { - let interval = Interval::from_month_day_nano(1, 31, 1); - let interval2 = Interval::from_month_day_nano(2, 1, 1); - let mut map = HashMap::new(); - map.insert(interval, 1); - assert_eq!(map.get(&interval2), Some(&1)); - } - - #[test] - fn test_interval_mul_int() { - let interval = Interval::from_month_day_nano(1, 1, 1); - let interval2 = interval.checked_mul_int(2).unwrap(); - assert_eq!(interval2.months, 2); - assert_eq!(interval2.days, 2); - assert_eq!(interval2.nsecs, 2); - - // test justified interval - let interval = Interval::from_month_day_nano(1, 31, 1); - let interval2 = interval.checked_mul_int(2).unwrap(); - assert_eq!(interval2.months, 4); - assert_eq!(interval2.days, 2); - assert_eq!(interval2.nsecs, 2); - - // test overflow situation - let interval = Interval::from_month_day_nano(i32::MAX, 1, 1); - let interval2 = interval.checked_mul_int(2); - assert!(interval2.is_none()); - } - - #[test] - fn test_display() { - let interval = Interval::from_month_day_nano(1, 1, 1); - assert_eq!(interval.to_string(), "1 months 1 days 1 nsecs"); - - let interval = Interval::from_month_day_nano(14, 31, 10000000000); - assert_eq!(interval.to_string(), "14 months 31 days 10000000000 nsecs"); - } - - #[test] - fn test_interval_justified() { - let interval = Interval::from_month_day_nano(1, 131, 1).justified_interval(); - let interval2 = Interval::from_month_day_nano(5, 11, 1); - assert_eq!(interval, interval2); - - let interval = Interval::from_month_day_nano(1, 1, NANOS_PER_MONTH + 2 * NANOS_PER_DAY) - .justified_interval(); - let interval2 = Interval::from_month_day_nano(2, 3, 0); - assert_eq!(interval, interval2); - } - - #[test] - fn test_serde_json() { - let interval = Interval::from_month_day_nano(1, 1, 1); - let json = serde_json::to_string(&interval).unwrap(); - assert_eq!( - json, - "{\"months\":1,\"days\":1,\"nsecs\":1,\"unit\":\"MonthDayNano\"}" - ); - let interval2: Interval = serde_json::from_str(&json).unwrap(); - assert_eq!(interval, interval2); - } - #[test] fn test_to_iso8601_string() { // Test interval zero - let interval = Interval::from_month_day_nano(0, 0, 0); + let interval = IntervalMonthDayNano::new(0, 0, 0); assert_eq!(interval.to_iso8601_string(), "PT0S"); - let interval = Interval::from_month_day_nano(1, 1, 1); + let interval = IntervalMonthDayNano::new(1, 1, 1); assert_eq!(interval.to_iso8601_string(), "P0Y1M1DT0H0M0S"); - let interval = Interval::from_month_day_nano(14, 31, 10000000000); + let interval = IntervalMonthDayNano::new(14, 31, 10000000000); assert_eq!(interval.to_iso8601_string(), "P1Y2M31DT0H0M10S"); - let interval = Interval::from_month_day_nano(14, 31, 23210200000000); + let interval = IntervalMonthDayNano::new(14, 31, 23210200000000); assert_eq!(interval.to_iso8601_string(), "P1Y2M31DT6H26M50.2S"); } #[test] fn test_to_postgres_string() { // Test interval zero - let interval = Interval::from_month_day_nano(0, 0, 0); - assert_eq!(interval.to_postgres_string(), "00:00:00"); - - let interval = Interval::from_month_day_nano(23, 100, 23210200000000); + let interval = IntervalMonthDayNano::new(0, 0, 0); assert_eq!( - interval.to_postgres_string(), + IntervalFormat::from(interval).to_postgres_string(), + "00:00:00" + ); + + let interval = IntervalMonthDayNano::new(23, 100, 23210200000000); + assert_eq!( + IntervalFormat::from(interval).to_postgres_string(), "1 year 11 mons 100 days 06:26:50.200000" ); } @@ -776,18 +619,21 @@ mod tests { #[test] fn test_to_sql_standard_string() { // Test zero interval - let interval = Interval::from_month_day_nano(0, 0, 0); - assert_eq!(interval.to_sql_standard_string(), "0"); + let interval = IntervalMonthDayNano::new(0, 0, 0); + assert_eq!(IntervalFormat::from(interval).to_sql_standard_string(), "0"); - let interval = Interval::from_month_day_nano(23, 100, 23210200000000); + let interval = IntervalMonthDayNano::new(23, 100, 23210200000000); assert_eq!( - interval.to_sql_standard_string(), + IntervalFormat::from(interval).to_sql_standard_string(), "+1-11 +100 +6:26:50.200000" ); // Test interval without year, month, day - let interval = Interval::from_month_day_nano(0, 0, 23210200000000); - assert_eq!(interval.to_sql_standard_string(), "6:26:50.200000"); + let interval = IntervalMonthDayNano::new(0, 0, 23210200000000); + assert_eq!( + IntervalFormat::from(interval).to_sql_standard_string(), + "6:26:50.200000" + ); } #[test] diff --git a/src/common/time/src/lib.rs b/src/common/time/src/lib.rs index 770057394c..fa025bf661 100644 --- a/src/common/time/src/lib.rs +++ b/src/common/time/src/lib.rs @@ -27,7 +27,7 @@ pub mod util; pub use date::Date; pub use datetime::DateTime; pub use duration::Duration; -pub use interval::Interval; +pub use interval::{IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth}; pub use range::RangeMillis; pub use timestamp::Timestamp; pub use timestamp_millis::TimestampMillis; diff --git a/src/common/time/src/timestamp.rs b/src/common/time/src/timestamp.rs index 503c44cf99..258e9c1a96 100644 --- a/src/common/time/src/timestamp.rs +++ b/src/common/time/src/timestamp.rs @@ -20,16 +20,17 @@ use std::time::Duration; use arrow::datatypes::TimeUnit as ArrowTimeUnit; use chrono::{ - DateTime, Days, LocalResult, Months, NaiveDate, NaiveDateTime, NaiveTime, + DateTime, Days, LocalResult, Months, NaiveDate, NaiveDateTime, NaiveTime, TimeDelta, TimeZone as ChronoTimeZone, Utc, }; use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; +use crate::error; use crate::error::{ArithmeticOverflowSnafu, ParseTimestampSnafu, Result, TimestampOverflowSnafu}; +use crate::interval::{IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth}; use crate::timezone::{get_timezone, Timezone}; use crate::util::{datetime_to_utc, div_ceil}; -use crate::{error, Interval}; /// Timestamp represents the value of units(seconds/milliseconds/microseconds/nanoseconds) elapsed /// since UNIX epoch. The valid value range of [Timestamp] depends on it's unit (all in UTC timezone): @@ -140,40 +141,77 @@ impl Timestamp { }) } - /// Adds given Interval to the current timestamp. - /// Returns None if the resulting timestamp would be out of range. - pub fn add_interval(&self, interval: Interval) -> Option { + // FIXME(yingwen): remove add/sub intervals later + /// Adds given [IntervalYearMonth] to the current timestamp. + pub fn add_year_month(&self, interval: IntervalYearMonth) -> Option { let naive_datetime = self.to_chrono_datetime()?; - let (months, days, nsecs) = interval.to_month_day_nano(); - let naive_datetime = naive_datetime - .checked_add_months(Months::new(months as u32))? - .checked_add_days(Days::new(days as u64))? - + Duration::from_nanos(nsecs as u64); + let naive_datetime = + naive_datetime.checked_add_months(Months::new(interval.months as u32))?; - match Timestamp::from_chrono_datetime(naive_datetime) { - // Have to convert the new timestamp by the current unit. - Some(ts) => ts.convert_to(self.unit), - None => None, - } + // Have to convert the new timestamp by the current unit. + Timestamp::from_chrono_datetime(naive_datetime).and_then(|ts| ts.convert_to(self.unit)) } - /// Subtracts given Interval to the current timestamp. - /// Returns None if the resulting timestamp would be out of range. - pub fn sub_interval(&self, interval: Interval) -> Option { + /// Adds given [IntervalDayTime] to the current timestamp. + pub fn add_day_time(&self, interval: IntervalDayTime) -> Option { let naive_datetime = self.to_chrono_datetime()?; - let (months, days, nsecs) = interval.to_month_day_nano(); let naive_datetime = naive_datetime - .checked_sub_months(Months::new(months as u32))? - .checked_sub_days(Days::new(days as u64))? - - Duration::from_nanos(nsecs as u64); + .checked_add_days(Days::new(interval.days as u64))? + .checked_add_signed(TimeDelta::milliseconds(interval.milliseconds as i64))?; - match Timestamp::from_chrono_datetime(naive_datetime) { - // Have to convert the new timestamp by the current unit. - Some(ts) => ts.convert_to(self.unit), - None => None, - } + // Have to convert the new timestamp by the current unit. + Timestamp::from_chrono_datetime(naive_datetime).and_then(|ts| ts.convert_to(self.unit)) + } + + /// Adds given [IntervalMonthDayNano] to the current timestamp. + pub fn add_month_day_nano(&self, interval: IntervalMonthDayNano) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + let naive_datetime = naive_datetime + .checked_add_months(Months::new(interval.months as u32))? + .checked_add_days(Days::new(interval.days as u64))? + .checked_add_signed(TimeDelta::nanoseconds(interval.nanoseconds))?; + + // Have to convert the new timestamp by the current unit. + Timestamp::from_chrono_datetime(naive_datetime).and_then(|ts| ts.convert_to(self.unit)) + } + + /// Subtracts given [IntervalYearMonth] to the current timestamp. + pub fn sub_year_month(&self, interval: IntervalYearMonth) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + let naive_datetime = + naive_datetime.checked_sub_months(Months::new(interval.months as u32))?; + + // Have to convert the new timestamp by the current unit. + Timestamp::from_chrono_datetime(naive_datetime).and_then(|ts| ts.convert_to(self.unit)) + } + + /// Subtracts given [IntervalDayTime] to the current timestamp. + pub fn sub_day_time(&self, interval: IntervalDayTime) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + let naive_datetime = naive_datetime + .checked_sub_days(Days::new(interval.days as u64))? + .checked_sub_signed(TimeDelta::milliseconds(interval.milliseconds as i64))?; + + // Have to convert the new timestamp by the current unit. + Timestamp::from_chrono_datetime(naive_datetime).and_then(|ts| ts.convert_to(self.unit)) + } + + /// Subtracts given [IntervalMonthDayNano] to the current timestamp. + pub fn sub_month_day_nano(&self, interval: IntervalMonthDayNano) -> Option { + let naive_datetime = self.to_chrono_datetime()?; + + let naive_datetime = naive_datetime + .checked_sub_months(Months::new(interval.months as u32))? + .checked_sub_days(Days::new(interval.days as u64))? + .checked_sub_signed(TimeDelta::nanoseconds(interval.nanoseconds))?; + + // Have to convert the new timestamp by the current unit. + Timestamp::from_chrono_datetime(naive_datetime).and_then(|ts| ts.convert_to(self.unit)) } /// Subtracts current timestamp with another timestamp, yielding a duration. @@ -688,13 +726,13 @@ mod tests { fn test_add_sub_interval() { let ts = Timestamp::new(1000, TimeUnit::Millisecond); - let interval = Interval::from_day_time(1, 200); + let interval = IntervalDayTime::new(1, 200); - let new_ts = ts.add_interval(interval).unwrap(); + let new_ts = ts.add_day_time(interval).unwrap(); assert_eq!(new_ts.unit(), TimeUnit::Millisecond); assert_eq!(new_ts.value(), 1000 + 3600 * 24 * 1000 + 200); - assert_eq!(ts, new_ts.sub_interval(interval).unwrap()); + assert_eq!(ts, new_ts.sub_day_time(interval).unwrap()); } #[test] diff --git a/src/datatypes/src/interval.rs b/src/datatypes/src/interval.rs index c0969abc44..b724de8022 100644 --- a/src/datatypes/src/interval.rs +++ b/src/datatypes/src/interval.rs @@ -12,11 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_time::interval::Interval; +use common_time::{IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth}; use paste::paste; -use serde::{Deserialize, Serialize}; -use crate::prelude::{Scalar, Value, ValueRef}; +use crate::prelude::Scalar; use crate::scalars::ScalarRef; use crate::types::{ IntervalDayTimeType, IntervalMonthDayNanoType, IntervalYearMonthType, WrapperType, @@ -26,39 +25,6 @@ use crate::vectors::{IntervalDayTimeVector, IntervalMonthDayNanoVector, Interval macro_rules! define_interval_with_unit { ($unit: ident, $native_ty: ty) => { paste! { - #[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] - pub struct [](pub Interval); - - impl [] { - pub fn new(val: $native_ty) -> Self { - Self(Interval:: [](val)) - } - } - - impl Default for [] { - fn default() -> Self { - Self::new(0) - } - } - - impl From<[]> for Value { - fn from(t: []) -> Value { - Value::Interval(t.0) - } - } - - impl From<[]> for serde_json::Value { - fn from(t: []) -> Self { - t.0.into() - } - } - - impl From<[]> for ValueRef<'static> { - fn from(t: []) -> Self { - ValueRef::Interval(t.0) - } - } - impl Scalar for [] { type VectorType = []; type RefType<'a> = []; @@ -87,41 +53,11 @@ macro_rules! define_interval_with_unit { type Native = $native_ty; fn from_native(value: Self::Native) -> Self { - Self::new(value) + Self::[](value) } fn into_native(self) -> Self::Native { - self.0.[]() - } - } - - impl From<$native_ty> for [] { - fn from(val: $native_ty) -> Self { - []::from_native(val as $native_ty) - } - } - - impl From<[]> for $native_ty { - fn from(val: []) -> Self { - val.0.[]() - } - } - - impl TryFrom for Option<[]> { - type Error = $crate::error::Error; - - #[inline] - fn try_from(from: Value) -> std::result::Result { - match from { - Value::Interval(v) if v.unit() == common_time::interval::IntervalUnit::$unit => { - Ok(Some([](v))) - }, - Value::Null => Ok(None), - _ => $crate::error::TryFromValueSnafu { - reason: format!("{:?} is not a {}", from, stringify!([])), - } - .fail(), - } + self.[]() } } } @@ -138,17 +74,17 @@ mod tests { #[test] fn test_interval_scalar() { - let interval = IntervalYearMonth::new(1000); + let interval = IntervalYearMonth::from(1000); assert_eq!(interval, interval.as_scalar_ref()); assert_eq!(interval, interval.to_owned_scalar()); assert_eq!(1000, interval.into_native()); - let interval = IntervalDayTime::new(1000); + let interval = IntervalDayTime::from(1000); assert_eq!(interval, interval.as_scalar_ref()); assert_eq!(interval, interval.to_owned_scalar()); assert_eq!(1000, interval.into_native()); - let interval = IntervalMonthDayNano::new(1000); + let interval = IntervalMonthDayNano::from(1000); assert_eq!(interval, interval.as_scalar_ref()); assert_eq!(interval, interval.to_owned_scalar()); assert_eq!(1000, interval.into_native()); @@ -156,15 +92,15 @@ mod tests { #[test] fn test_interval_convert_to_native_type() { - let interval = IntervalMonthDayNano::new(1000); + let interval = IntervalMonthDayNano::from(1000); let native_value: i128 = interval.into(); assert_eq!(native_value, 1000); - let interval = IntervalDayTime::new(1000); + let interval = IntervalDayTime::from(1000); let native_interval: i64 = interval.into(); assert_eq!(native_interval, 1000); - let interval = IntervalYearMonth::new(1000); + let interval = IntervalYearMonth::from(1000); let native_interval: i32 = interval.into(); assert_eq!(native_interval, 1000); } diff --git a/src/datatypes/src/types/interval_type.rs b/src/datatypes/src/types/interval_type.rs index 7ee7964982..77f1b47d6d 100644 --- a/src/datatypes/src/types/interval_type.rs +++ b/src/datatypes/src/types/interval_type.rs @@ -17,8 +17,9 @@ use arrow::datatypes::{ IntervalMonthDayNanoType as ArrowIntervalMonthDayNanoType, IntervalUnit as ArrowIntervalUnit, IntervalYearMonthType as ArrowIntervalYearMonthType, }; -use common_time::interval::IntervalUnit; -use common_time::Interval; +use common_time::interval::{ + IntervalDayTime, IntervalMonthDayNano, IntervalUnit, IntervalYearMonth, +}; use enum_dispatch::enum_dispatch; use paste::paste; use serde::{Deserialize, Serialize}; @@ -26,7 +27,6 @@ use snafu::OptionExt; use crate::data_type::ConcreteDataType; use crate::error; -use crate::interval::{IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth}; use crate::prelude::{ DataType, LogicalTypeId, MutableVector, ScalarVectorBuilder, Value, ValueRef, Vector, }; @@ -75,7 +75,7 @@ macro_rules! impl_data_type_for_interval { } fn default_value(&self) -> Value { - Value::Interval(Interval::from_i128(0)) + Value::[]([]::default()) } fn as_arrow_type(&self) -> ArrowDataType { @@ -124,7 +124,7 @@ macro_rules! impl_data_type_for_interval { fn cast_value_ref(value: ValueRef) -> crate::Result> { match value { ValueRef::Null => Ok(None), - ValueRef::Interval(t) => Ok(Some([](t))), + ValueRef::[](t) => Ok(Some(t)), other => error::CastTypeSnafu { msg: format!("Failed to cast value {:?} to {}", other, stringify!([])), } diff --git a/src/datatypes/src/types/primitive_type.rs b/src/datatypes/src/types/primitive_type.rs index cae71976fd..86972bd5ee 100644 --- a/src/datatypes/src/types/primitive_type.rs +++ b/src/datatypes/src/types/primitive_type.rs @@ -16,7 +16,6 @@ use std::cmp::Ordering; use std::fmt; use arrow::datatypes::{ArrowNativeType, ArrowPrimitiveType, DataType as ArrowDataType}; -use common_time::interval::IntervalUnit; use common_time::{Date, DateTime}; use serde::{Deserialize, Serialize}; use snafu::OptionExt; @@ -30,6 +29,7 @@ use crate::types::{DateTimeType, DateType}; use crate::value::{Value, ValueRef}; use crate::vectors::{MutableVector, PrimitiveVector, PrimitiveVectorBuilder, Vector}; +// TODO(yingwen): Can we remove `Into`? /// Represents the wrapper type that wraps a native type using the `newtype pattern`, /// such as [Date](`common_time::Date`) is a wrapper type for the underlying native /// type `i32`. @@ -364,11 +364,7 @@ impl DataType for Int64Type { Value::DateTime(v) => Some(Value::Int64(v.val())), Value::Timestamp(v) => Some(Value::Int64(v.value())), Value::Time(v) => Some(Value::Int64(v.value())), - Value::Interval(v) => match v.unit() { - IntervalUnit::DayTime => Some(Value::Int64(v.to_i64())), - IntervalUnit::YearMonth => None, - IntervalUnit::MonthDayNano => None, - }, + // We don't allow casting interval type to int. _ => None, } } @@ -410,11 +406,7 @@ impl DataType for Int32Type { Value::Float64(v) => num::cast::cast(v).map(Value::Int32), Value::String(v) => v.as_utf8().parse::().map(Value::Int32).ok(), Value::Date(v) => Some(Value::Int32(v.val())), - Value::Interval(v) => match v.unit() { - IntervalUnit::YearMonth => Some(Value::Int32(v.to_i32())), - IntervalUnit::DayTime => None, - IntervalUnit::MonthDayNano => None, - }, + // We don't allow casting interval type to int. _ => None, } } diff --git a/src/datatypes/src/types/string_type.rs b/src/datatypes/src/types/string_type.rs index 38045a600a..06a5e7c7f6 100644 --- a/src/datatypes/src/types/string_type.rs +++ b/src/datatypes/src/types/string_type.rs @@ -78,7 +78,15 @@ impl DataType for StringType { Value::DateTime(v) => Some(Value::String(StringBytes::from(v.to_string()))), Value::Timestamp(v) => Some(Value::String(StringBytes::from(v.to_iso8601_string()))), Value::Time(v) => Some(Value::String(StringBytes::from(v.to_iso8601_string()))), - Value::Interval(v) => Some(Value::String(StringBytes::from(v.to_iso8601_string()))), + Value::IntervalYearMonth(v) => { + Some(Value::String(StringBytes::from(v.to_iso8601_string()))) + } + Value::IntervalDayTime(v) => { + Some(Value::String(StringBytes::from(v.to_iso8601_string()))) + } + Value::IntervalMonthDayNano(v) => { + Some(Value::String(StringBytes::from(v.to_iso8601_string()))) + } Value::Duration(v) => Some(Value::String(StringBytes::from(v.to_string()))), Value::Decimal128(v) => Some(Value::String(StringBytes::from(v.to_string()))), diff --git a/src/datatypes/src/value.rs b/src/datatypes/src/value.rs index a8e59da513..b973a3156b 100644 --- a/src/datatypes/src/value.rs +++ b/src/datatypes/src/value.rs @@ -28,7 +28,7 @@ use common_time::datetime::DateTime; use common_time::interval::IntervalUnit; use common_time::time::Time; use common_time::timestamp::{TimeUnit, Timestamp}; -use common_time::{Duration, Interval, Timezone}; +use common_time::{Duration, IntervalDayTime, IntervalMonthDayNano, IntervalYearMonth, Timezone}; use datafusion_common::ScalarValue; use greptime_proto::v1::value::ValueData; pub use ordered_float::OrderedFloat; @@ -78,7 +78,10 @@ pub enum Value { Timestamp(Timestamp), Time(Time), Duration(Duration), - Interval(Interval), + // Interval types: + IntervalYearMonth(IntervalYearMonth), + IntervalDayTime(IntervalDayTime), + IntervalMonthDayNano(IntervalMonthDayNano), List(ListValue), } @@ -111,7 +114,15 @@ impl Display for Value { Value::DateTime(v) => write!(f, "{v}"), Value::Timestamp(v) => write!(f, "{}", v.to_iso8601_string()), Value::Time(t) => write!(f, "{}", t.to_iso8601_string()), - Value::Interval(v) => write!(f, "{}", v.to_iso8601_string()), + Value::IntervalYearMonth(v) => { + write!(f, "{}", v.to_iso8601_string()) + } + Value::IntervalDayTime(v) => { + write!(f, "{}", v.to_iso8601_string()) + } + Value::IntervalMonthDayNano(v) => { + write!(f, "{}", v.to_iso8601_string()) + } Value::Duration(d) => write!(f, "{d}"), Value::List(v) => { let items = v @@ -153,7 +164,15 @@ macro_rules! define_data_type_func { $struct::DateTime(_) => ConcreteDataType::datetime_datatype(), $struct::Time(t) => ConcreteDataType::time_datatype(*t.unit()), $struct::Timestamp(v) => ConcreteDataType::timestamp_datatype(v.unit()), - $struct::Interval(v) => ConcreteDataType::interval_datatype(v.unit()), + $struct::IntervalYearMonth(_) => { + ConcreteDataType::interval_datatype(IntervalUnit::YearMonth) + } + $struct::IntervalDayTime(_) => { + ConcreteDataType::interval_datatype(IntervalUnit::DayTime) + } + $struct::IntervalMonthDayNano(_) => { + ConcreteDataType::interval_datatype(IntervalUnit::MonthDayNano) + } $struct::List(list) => ConcreteDataType::list_datatype(list.datatype().clone()), $struct::Duration(d) => ConcreteDataType::duration_datatype(d.unit()), $struct::Decimal128(d) => { @@ -206,7 +225,9 @@ impl Value { Value::List(v) => ValueRef::List(ListValueRef::Ref { val: v }), Value::Timestamp(v) => ValueRef::Timestamp(*v), Value::Time(v) => ValueRef::Time(*v), - Value::Interval(v) => ValueRef::Interval(*v), + Value::IntervalYearMonth(v) => ValueRef::IntervalYearMonth(*v), + Value::IntervalDayTime(v) => ValueRef::IntervalDayTime(*v), + Value::IntervalMonthDayNano(v) => ValueRef::IntervalMonthDayNano(*v), Value::Duration(v) => ValueRef::Duration(*v), Value::Decimal128(v) => ValueRef::Decimal128(*v), } @@ -220,14 +241,6 @@ impl Value { } } - /// Cast Value to Interval. Return None if value is not a valid interval data type. - pub fn as_interval(&self) -> Option { - match self { - Value::Interval(i) => Some(*i), - _ => None, - } - } - /// Cast Value to utf8 String. Return None if value is not a valid string data type. pub fn as_string(&self) -> Option { match self { @@ -255,12 +268,35 @@ impl Value { /// Cast Value to [Time]. Return None if value is not a valid time data type. pub fn as_time(&self) -> Option