From 4b4afc879c607b188e23596bbe31684ea664d6a0 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 18 Oct 2023 13:32:20 +0800 Subject: [PATCH 01/58] fix(connector): offset miscounting in nd_streaming (#12866) --- e2e_test/s3/fs_source_v2.py | 5 +- .../src/source/filesystem/nd_streaming.rs | 215 +++++++++++------- 2 files changed, 133 insertions(+), 87 deletions(-) diff --git a/e2e_test/s3/fs_source_v2.py b/e2e_test/s3/fs_source_v2.py index 7ac5d91ad4bd7..a687c9be19c9d 100644 --- a/e2e_test/s3/fs_source_v2.py +++ b/e2e_test/s3/fs_source_v2.py @@ -39,10 +39,7 @@ def format_csv(data, with_header): writer.writeheader() for item_data in file_data: writer.writerow(item_data) - # For now paser can only handle \n line seperator, - # and tailing white spaces are not allowed. - # TODO: remove replace and rstrip later - csv_files.append(ostream.getvalue().replace('\r', '').rstrip()) + csv_files.append(ostream.getvalue()) return csv_files def do_test(config, file_num, item_num_per_file, prefix, fmt): diff --git a/src/connector/src/source/filesystem/nd_streaming.rs b/src/connector/src/source/filesystem/nd_streaming.rs index e1f5e88eda01e..5574063ffc855 100644 --- a/src/connector/src/source/filesystem/nd_streaming.rs +++ b/src/connector/src/source/filesystem/nd_streaming.rs @@ -12,25 +12,24 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::io::BufRead; - +use anyhow::anyhow; use bytes::BytesMut; +use futures::io::Cursor; +use futures::AsyncBufReadExt; use futures_async_stream::try_stream; use crate::source::{BoxSourceStream, SourceMessage}; #[try_stream(boxed, ok = Vec, error = anyhow::Error)] -/// This function splits a byte stream by the newline character '\n' into a message stream. +/// This function splits a byte stream by the newline separator "(\r)\n" into a message stream. /// It can be difficult to split and compute offsets correctly when the bytes are received in /// chunks. There are two cases to consider: -/// - When a bytes chunk does not end with '\n', we should not treat the last segment as a new line -/// message, but keep it for the next chunk, and insert it before next chunk's first line -/// beginning. -/// - When a bytes chunk ends with '\n', there is no additional action required. +/// - When a bytes chunk does not end with "(\r)\n", we should not treat the last segment as a new line +/// message, but keep it for the next chunk, and prepend it to the first line of the next chunk. +/// - When a bytes chunk ends with "(\r)\n", there is no additional action required. pub async fn split_stream(data_stream: BoxSourceStream) { - let mut buf = BytesMut::new(); - let mut last_message = None; + #[for_await] for batch in data_stream { let batch = batch?; @@ -46,54 +45,70 @@ pub async fn split_stream(data_stream: BoxSourceStream) { .unwrap(); let mut offset: usize = offset.parse()?; - - // Never panic because we check batch is not empty - let last_item = batch.last().unwrap(); - let end_offset: usize = last_item.offset.parse::().unwrap() - + last_item - .payload - .as_ref() - .map(|p| p.len()) - .unwrap_or_default(); + let mut buf = BytesMut::new(); for msg in batch { let payload = msg.payload.unwrap_or_default(); buf.extend(payload); } let mut msgs = Vec::new(); - for (i, line) in buf.lines().enumerate() { - let mut line = line?; - - // Insert the trailing of the last chunk in front of the first line, do not count - // the length here. - if i == 0 && last_message.is_some() { - let msg: SourceMessage = std::mem::take(&mut last_message).unwrap(); - let last_payload = msg.payload.unwrap(); - offset -= last_payload.len(); - line = String::from_utf8(last_payload).unwrap() + &line; - } - let len = line.as_bytes().len(); - - msgs.push(SourceMessage { - key: None, - payload: Some(line.into()), - offset: (offset + len).to_string(), - split_id: split_id.clone(), - meta: meta.clone(), - }); - offset += len; - offset += 1; - } - if offset > end_offset { - last_message = msgs.pop(); - } + let mut cursor = Cursor::new(buf.freeze()); + let mut line_cnt: usize = 0; + loop { + let mut line = String::new(); + match cursor.read_line(&mut line).await { + Ok(0) => { + if !msgs.is_empty() { + yield msgs; + } + break; + } + Ok(_n) => { + if line_cnt == 0 && last_message.is_some() { + let msg: SourceMessage = std::mem::take(&mut last_message).unwrap(); + let last_payload = msg.payload.unwrap(); + offset -= last_payload.len(); + line.insert_str(0, &String::from_utf8(last_payload).unwrap()); + } + + let mut separator = String::with_capacity(2); + for delim in ['\n', '\r'] { + if line.ends_with(delim) { + separator.insert(0, line.pop().unwrap()); + } else { + // If the data is batched as "XXXX\r" and "\nXXXX", + // the line will be "XXXX\r" here because the cursor reaches EOF. + // Hence we should break the delim loop here, + // otherwise the \r would be treated as separator even without \n. + break; + } + } + + let len = line.len(); + + offset += len + separator.len(); + let msg = SourceMessage { + key: None, + payload: Some(line.into()), + offset: offset.to_string(), + split_id: split_id.clone(), + meta: meta.clone(), + }; - if !msgs.is_empty() { - yield msgs; - } + msgs.push(msg); - buf.clear(); + if separator.is_empty() { + // Not ending with \n, prepend to the first line of the next batch + last_message = msgs.pop(); + } + } + Err(e) => return Err(anyhow!(e)), + } + + line_cnt += 1; + } } + if let Some(msg) = last_message { yield vec![msg]; } @@ -109,40 +124,74 @@ mod tests { #[tokio::test] async fn test_split_stream() { - const N1: usize = 10000; - const N2: usize = 500; - const N3: usize = 50; - let lines = (0..N1) - .map(|x| (0..x % N2).map(|_| 'A').collect::()) - .collect::>(); - let total_chars = lines.iter().map(|e| e.len()).sum::(); - let text = lines.join("\n").into_bytes(); - let split_id: Arc = "1".to_string().into_boxed_str().into(); - let s = text - .chunks(N2) - .enumerate() - .map(move |(i, e)| { - Ok(e.chunks(N3) - .enumerate() - .map(|(j, buf)| SourceMessage { - key: None, - payload: Some(buf.to_owned()), - offset: (i * N2 + j * N3).to_string(), - split_id: split_id.clone(), - meta: crate::source::SourceMeta::Empty, - }) - .collect::>()) - }) - .collect::>(); - let stream = futures::stream::iter(s).boxed(); - let msg_stream = split_stream(stream).try_collect::>().await.unwrap(); - let items = msg_stream - .into_iter() - .flatten() - .map(|e| String::from_utf8(e.payload.unwrap()).unwrap()) - .collect::>(); - assert_eq!(items.len(), N1); - let text = items.join(""); - assert_eq!(text.len(), total_chars); + // Test with tail separators. + for tail_separator in ["", "\n", "\r\n"] { + const N1: usize = 10000; + const N2: usize = 500; + const N3: usize = 50; + let lines = (0..N1) + .map(|x| (0..x % N2).map(|_| 'A').collect::()) + .collect::>(); + let total_chars = lines.iter().map(|e| e.len()).sum::(); + // Join lines with \n & \r\n alternately + let delims = ["\n", "\r\n"]; + let text = lines + .iter() + .enumerate() + .skip(1) + .fold(lines[0].clone(), |acc, (idx, now)| { + format!("{}{}{}", acc, delims[idx % 2], now) + }) + + tail_separator; + let text = text.into_bytes(); + let split_id: Arc = "1".to_string().into_boxed_str().into(); + let s = text + .chunks(N2) + .enumerate() + .map(move |(i, e)| { + Ok(e.chunks(N3) + .enumerate() + .map(|(j, buf)| SourceMessage { + key: None, + payload: Some(buf.to_owned()), + offset: (i * N2 + j * N3).to_string(), + split_id: split_id.clone(), + meta: crate::source::SourceMeta::Empty, + }) + .collect::>()) + }) + .collect::>(); + let stream = futures::stream::iter(s).boxed(); + let msg_stream = split_stream(stream).try_collect::>().await.unwrap(); + // Check the correctness of each line's offset + let mut expected_offset: usize = 0; + msg_stream + .iter() + .flatten() + .enumerate() + .for_each(|(idx, msg)| { + expected_offset += lines[idx].len() + + if idx < lines.len() - 1 { + delims[1 - idx % 2].len() + } else { + tail_separator.len() + }; + assert_eq!( + msg.offset.parse::().unwrap(), + expected_offset, + "idx = {}, tail_separator = {:?}", + idx, + tail_separator + ); + }); + let items = msg_stream + .into_iter() + .flatten() + .map(|e| String::from_utf8(e.payload.unwrap()).unwrap()) + .collect::>(); + assert_eq!(items.len(), N1); + let text = items.join(""); + assert_eq!(text.len(), total_chars); + } } } From 01ed55b0a9219d7c3543b7aec29d1484ebfcf0c1 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Wed, 18 Oct 2023 14:37:17 +0800 Subject: [PATCH 02/58] feat(s3): ban new cluster startup if s3 bucket has retention configured (#12806) --- src/meta/src/hummock/manager/mod.rs | 7 ++++++- src/object_store/src/object/s3.rs | 14 ++++++++++++-- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index f644db74714f7..24f1f29775353 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -353,7 +353,12 @@ impl HummockManager { if let risingwave_object_store::object::ObjectStoreImpl::S3(s3) = object_store.as_ref() && !env.opts.do_not_config_object_storage_lifecycle { - s3.inner().configure_bucket_lifecycle().await; + let is_bucket_expiration_configured = s3.inner().configure_bucket_lifecycle().await; + if is_bucket_expiration_configured{ + return Err(ObjectError::internal("Cluster cannot start with object expiration configured for bucket because RisingWave data will be lost when object expiration kicks in. + Please disable object expiration and restart the cluster.") + .into()); + } } } let checkpoint_path = version_checkpoint_path(state_store_dir); diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index a53f1a6825281..69e7f3687fdeb 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -671,7 +671,6 @@ impl S3ObjectStore { range: impl ObjectRangeBounds, ) -> GetObjectFluentBuilder { let req = self.client.get_object().bucket(&self.bucket).key(path); - if range.is_full() { return req; } @@ -697,7 +696,7 @@ impl S3ObjectStore { /// - /// - MinIO /// - - pub async fn configure_bucket_lifecycle(&self) { + pub async fn configure_bucket_lifecycle(&self) -> bool { // Check if lifecycle is already configured to avoid overriding existing configuration. let bucket = self.bucket.as_str(); let mut configured_rules = vec![]; @@ -707,8 +706,12 @@ impl S3ObjectStore { .bucket(bucket) .send() .await; + let mut is_expiration_configured = false; if let Ok(config) = &get_config_result { for rule in config.rules().unwrap_or_default() { + if rule.expiration().is_some() { + is_expiration_configured = true; + } if matches!(rule.status().unwrap(), ExpirationStatus::Enabled) && rule.abort_incomplete_multipart_upload().is_some() { @@ -755,6 +758,13 @@ impl S3ObjectStore { tracing::warn!("Failed to configure life cycle rule for S3 bucket: {:?}. It is recommended to configure it manually to avoid unnecessary storage cost.", bucket); } } + if is_expiration_configured { + tracing::info!( + "S3 bucket {} has already configured the expiration for the lifecycle.", + bucket, + ); + } + is_expiration_configured } #[inline(always)] From b782185a30b2ff6d7ac7a45045d85a4c60cf5dad Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Wed, 18 Oct 2023 15:19:39 +0800 Subject: [PATCH 03/58] refactor(meta): adapt ValTransaction to metadata model v2 (#12918) Co-authored-by: William Wen --- .../manager/compaction_group_manager.rs | 18 +- src/meta/src/hummock/manager/mod.rs | 4 +- src/meta/src/lib.rs | 1 + src/meta/src/manager/catalog/mod.rs | 2 +- src/meta/src/manager/cluster.rs | 4 +- src/meta/src/manager/system_param/mod.rs | 2 +- src/meta/src/manager/system_param/model.rs | 9 +- src/meta/src/model/mod.rs | 132 +++++---- src/meta/src/model_v2/ext/hummock.rs | 61 ++++ src/meta/src/model_v2/ext/mod.rs | 16 + src/meta/src/model_v2/mod.rs | 2 + src/meta/src/model_v2/trx.rs | 276 ++++++++++++++++++ 12 files changed, 453 insertions(+), 74 deletions(-) create mode 100644 src/meta/src/model_v2/ext/hummock.rs create mode 100644 src/meta/src/model_v2/ext/mod.rs create mode 100644 src/meta/src/model_v2/trx.rs diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index fbdfb66f4b0b8..f3853c8d08df5 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -255,7 +255,7 @@ impl HummockManager { assert!(sst_split_info.is_empty()); let mut trx = Transaction::default(); - new_version_delta.apply_to_txn(&mut trx)?; + new_version_delta.apply_to_txn(&mut trx).await?; self.env.meta_store().txn(trx).await?; versioning.current_version = current_version; new_version_delta.commit(); @@ -350,7 +350,7 @@ impl HummockManager { assert!(sst_split_info.is_empty()); let mut trx = Transaction::default(); - new_version_delta.apply_to_txn(&mut trx)?; + new_version_delta.apply_to_txn(&mut trx).await?; self.env.meta_store().txn(trx).await?; for group_id in &groups_to_remove { let max_level = versioning @@ -599,9 +599,9 @@ impl HummockManager { let mut current_version = versioning.current_version.clone(); let sst_split_info = current_version.apply_version_delta(&new_version_delta); - let mut branched_ssts = BTreeMapTransaction::new(&mut versioning.branched_ssts); + let mut branched_ssts = BTreeMapTransaction::<'_, _, _>::new(&mut versioning.branched_ssts); let mut trx = Transaction::default(); - new_version_delta.apply_to_txn(&mut trx)?; + new_version_delta.apply_to_txn(&mut trx).await?; if let Some((new_compaction_group_id, config)) = new_group { let mut compaction_group_manager = self.compaction_group_manager.write().await; let insert = BTreeMapEntryTransaction::new_insert( @@ -612,7 +612,7 @@ impl HummockManager { compaction_config: Arc::new(config), }, ); - insert.apply_to_txn(&mut trx)?; + insert.apply_to_txn(&mut trx).await?; self.env.meta_store().txn(trx).await?; insert.commit(); } else { @@ -780,7 +780,7 @@ impl CompactionGroupManager { compaction_groups.insert(*id, new_entry); } let mut trx = Transaction::default(); - compaction_groups.apply_to_txn(&mut trx)?; + compaction_groups.apply_to_txn(&mut trx).await?; meta_store.txn(trx).await?; compaction_groups.commit(); let r = compaction_group_ids @@ -826,7 +826,7 @@ impl CompactionGroupManager { } let mut trx = Transaction::default(); - compaction_groups.apply_to_txn(&mut trx)?; + compaction_groups.apply_to_txn(&mut trx).await?; meta_store.txn(trx).await?; compaction_groups.commit(); Ok(result) @@ -849,7 +849,7 @@ impl CompactionGroupManager { }, ); let mut trx = Transaction::default(); - insert.apply_to_txn(&mut trx)?; + insert.apply_to_txn(&mut trx).await?; meta_store.txn(trx).await?; insert.commit(); Ok(()) @@ -875,7 +875,7 @@ impl CompactionGroupManager { compaction_groups.remove(group); } let mut trx = Transaction::default(); - compaction_groups.apply_to_txn(&mut trx)?; + compaction_groups.apply_to_txn(&mut trx).await?; meta_store.txn(trx).await?; compaction_groups.commit(); Ok(()) diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 24f1f29775353..3a99dd898d1c9 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -159,7 +159,7 @@ macro_rules! commit_multi_var { let mut trx = $trx_extern_part; // Apply the change in `ValTransaction` to trx $( - $val_txn.apply_to_txn(&mut trx)?; + $val_txn.apply_to_txn(&mut trx).await?; )* // Commit to state store $hummock_mgr.commit_trx($hummock_mgr.env.meta_store(), trx, $context_id) @@ -1495,7 +1495,7 @@ impl HummockManager { .id_gen_manager() .generate_interval::<{ IdCategory::HummockSstableId }>(new_sst_id_number as u64) .await?; - let mut branched_ssts = BTreeMapTransaction::new(&mut versioning.branched_ssts); + let mut branched_ssts = BTreeMapTransaction::<'_, _, _>::new(&mut versioning.branched_ssts); let original_sstables = std::mem::take(&mut sstables); sstables.reserve_exact(original_sstables.len() - incorrect_ssts.len() + new_sst_id_number); let mut incorrect_ssts = incorrect_ssts.into_iter(); diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index 7a147274a8d2f..a5c52d5e3e0f3 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -33,6 +33,7 @@ #![feature(is_sorted)] #![feature(impl_trait_in_assoc_type)] #![feature(type_name_of_val)] +#![feature(async_fn_in_trait)] pub mod backup_restore; mod barrier; diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 1c8f0c2f397c0..e1c13c315a377 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -79,7 +79,7 @@ macro_rules! commit_meta_with_trx { async { // Apply the change in `ValTransaction` to trx $( - $val_txn.apply_to_txn(&mut $trx)?; + $val_txn.apply_to_txn(&mut $trx).await?; )* // Commit to meta store $manager.env.meta_store().txn($trx).await?; diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 925eeb45a9f0c..a31979c8871b0 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -261,7 +261,7 @@ impl ClusterManager { .unwrap() .is_unschedulable = target; - var_txn.apply_to_txn(&mut txn)?; + var_txn.apply_to_txn(&mut txn).await?; var_txns.push(var_txn); } } @@ -557,7 +557,7 @@ impl ClusterManagerCore { worker_id ); - var_txn.apply_to_txn(&mut txn)?; + var_txn.apply_to_txn(&mut txn).await?; var_txns.push(var_txn); } } diff --git a/src/meta/src/manager/system_param/mod.rs b/src/meta/src/manager/system_param/mod.rs index cdedad61d8d71..eb24e0db0f340 100644 --- a/src/meta/src/manager/system_param/mod.rs +++ b/src/meta/src/manager/system_param/mod.rs @@ -89,7 +89,7 @@ impl SystemParamsManager { set_system_param(mem_txn.deref_mut(), name, value).map_err(MetaError::system_param)?; let mut store_txn = Transaction::default(); - mem_txn.apply_to_txn(&mut store_txn)?; + mem_txn.apply_to_txn(&mut store_txn).await?; self.meta_store.txn(store_txn).await?; mem_txn.commit(); diff --git a/src/meta/src/manager/system_param/model.rs b/src/meta/src/manager/system_param/model.rs index bed4f3d86e8a4..d486d6a5d74c6 100644 --- a/src/meta/src/manager/system_param/model.rs +++ b/src/meta/src/manager/system_param/model.rs @@ -67,20 +67,21 @@ impl SystemParamsModel for SystemParams { S: MetaStore, { let mut txn = Transaction::default(); - self.upsert_in_transaction(&mut txn)?; + self.upsert_in_transaction(&mut txn).await?; Ok(store.txn(txn).await?) } } -impl Transactional for SystemParams { - fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { +#[async_trait] +impl Transactional for SystemParams { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { for (k, v) in system_params_to_kv(self).map_err(MetadataModelError::internal)? { trx.put(Self::cf_name(), k.into_bytes(), v.into_bytes()); } Ok(()) } - fn delete_in_transaction(&self, _trx: &mut Transaction) -> MetadataModelResult<()> { + async fn delete_in_transaction(&self, _trx: &mut Transaction) -> MetadataModelResult<()> { unreachable!() } } diff --git a/src/meta/src/model/mod.rs b/src/meta/src/model/mod.rs index bb07e7e7b6cf1..f1fe0285d9ae8 100644 --- a/src/meta/src/model/mod.rs +++ b/src/meta/src/model/mod.rs @@ -24,6 +24,7 @@ mod user; use std::collections::btree_map::{Entry, VacantEntry}; use std::collections::BTreeMap; use std::fmt::Debug; +use std::marker::PhantomData; use std::ops::{Deref, DerefMut}; use async_trait::async_trait; @@ -48,9 +49,10 @@ pub type DispatcherId = u64; /// A global, unique identifier of a fragment pub type FragmentId = u32; -pub trait Transactional { - fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()>; - fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()>; +#[async_trait] +pub trait Transactional { + async fn upsert_in_transaction(&self, trx: &mut TXN) -> MetadataModelResult<()>; + async fn delete_in_transaction(&self, trx: &mut TXN) -> MetadataModelResult<()>; } mod private { @@ -203,11 +205,12 @@ for_all_metadata_models!(impl_metadata_model_marker); /// `Transactional` defines operations supported in a transaction. /// Read operations can be supported if necessary. -impl Transactional for T +#[async_trait] +impl Transactional for T where - T: MetadataModel, + T: MetadataModel + Sync, { - fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { trx.put( Self::cf_name(), self.key()?.encode_to_vec(), @@ -216,7 +219,7 @@ where Ok(()) } - fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { trx.delete(Self::cf_name(), self.key()?.encode_to_vec()); Ok(()) } @@ -225,11 +228,12 @@ where /// Trait that wraps a local memory value and applies the change to the local memory value on /// `commit` or leaves the local memory value untouched on `abort`. pub trait ValTransaction: Sized { + type TXN; /// Commit the change to local memory value fn commit(self); /// Apply the change (upsert or delete) to `txn` - fn apply_to_txn(&self, txn: &mut Transaction) -> MetadataModelResult<()>; + async fn apply_to_txn(&self, txn: &mut Self::TXN) -> MetadataModelResult<()>; /// Abort the `VarTransaction` and leave the local memory value untouched fn abort(self) { @@ -243,26 +247,28 @@ pub trait ValTransaction: Sized { /// When `commit` is called, the change to `new_value` will be applied to the `orig_value_ref` /// When `abort` is called, the `VarTransaction` is dropped and the local memory value is /// untouched. -pub struct VarTransaction<'a, T: Transactional> { +pub struct VarTransaction<'a, TXN, T: Transactional> { orig_value_ref: &'a mut T, new_value: Option, + _phantom: PhantomData, } -impl<'a, T> VarTransaction<'a, T> +impl<'a, TXN, T> VarTransaction<'a, TXN, T> where - T: Transactional, + T: Transactional, { /// Create a `VarTransaction` that wraps a raw variable - pub fn new(val_ref: &'a mut T) -> VarTransaction<'a, T> { + pub fn new(val_ref: &'a mut T) -> VarTransaction<'a, TXN, T> { VarTransaction { // lazy initialization new_value: None, orig_value_ref: val_ref, + _phantom: PhantomData, } } } -impl<'a, T: Transactional> Deref for VarTransaction<'a, T> { +impl<'a, TXN, T: Transactional> Deref for VarTransaction<'a, TXN, T> { type Target = T; fn deref(&self) -> &Self::Target { @@ -273,9 +279,9 @@ impl<'a, T: Transactional> Deref for VarTransaction<'a, T> { } } -impl<'a, T> DerefMut for VarTransaction<'a, T> +impl<'a, TXN, T> DerefMut for VarTransaction<'a, TXN, T> where - T: Clone + Transactional, + T: Clone + Transactional, { fn deref_mut(&mut self) -> &mut Self::Target { if self.new_value.is_none() { @@ -285,21 +291,23 @@ where } } -impl<'a, T> ValTransaction for VarTransaction<'a, T> +impl<'a, TXN, T> ValTransaction for VarTransaction<'a, TXN, T> where - T: Transactional + PartialEq, + T: Transactional + PartialEq, { + type TXN = TXN; + fn commit(self) { if let Some(new_value) = self.new_value { *self.orig_value_ref = new_value; } } - fn apply_to_txn(&self, txn: &mut Transaction) -> MetadataModelResult<()> { + async fn apply_to_txn(&self, txn: &mut Self::TXN) -> MetadataModelResult<()> { if let Some(new_value) = &self.new_value { // Apply the change to `txn` only when the value is modified if *self.orig_value_ref != *new_value { - new_value.upsert_in_transaction(txn) + new_value.upsert_in_transaction(txn).await } else { Ok(()) } @@ -418,25 +426,27 @@ enum BTreeMapOp { /// are stored in `staging`. On `commit`, it will apply the changes stored in `staging` to the in /// memory btree map. When serve `get` and `get_mut`, it merges the value stored in `staging` and /// `tree_ref`. -pub struct BTreeMapTransaction<'a, K: Ord, V> { +pub struct BTreeMapTransaction<'a, K: Ord, V, TXN = Transaction> { /// A reference to the original `BTreeMap`. All access to this field should be immutable, /// except when we commit the staging changes to the original map. tree_ref: &'a mut BTreeMap, /// Store all the staging changes that will be applied to the original map on commit staging: BTreeMap>, + _phantom: PhantomData, } -impl<'a, K: Ord + Debug, V: Clone> BTreeMapTransaction<'a, K, V> { - pub fn new(tree_ref: &'a mut BTreeMap) -> BTreeMapTransaction<'a, K, V> { +impl<'a, K: Ord + Debug, V: Clone, TXN> BTreeMapTransaction<'a, K, V, TXN> { + pub fn new(tree_ref: &'a mut BTreeMap) -> BTreeMapTransaction<'a, K, V, TXN> { Self { tree_ref, staging: BTreeMap::default(), + _phantom: PhantomData, } } /// Start a `BTreeMapEntryTransaction` when the `key` exists #[allow(dead_code)] - pub fn new_entry_txn(&mut self, key: K) -> Option> { + pub fn new_entry_txn(&mut self, key: K) -> Option> { BTreeMapEntryTransaction::new(self.tree_ref, key, None) } @@ -447,13 +457,17 @@ impl<'a, K: Ord + Debug, V: Clone> BTreeMapTransaction<'a, K, V> { &mut self, key: K, default_val: V, - ) -> BTreeMapEntryTransaction<'_, K, V> { + ) -> BTreeMapEntryTransaction<'_, K, V, TXN> { BTreeMapEntryTransaction::new(self.tree_ref, key, Some(default_val)) .expect("default value is provided and should return `Some`") } /// Start a `BTreeMapEntryTransaction` that inserts the `val` into `key`. - pub fn new_entry_insert_txn(&mut self, key: K, val: V) -> BTreeMapEntryTransaction<'_, K, V> { + pub fn new_entry_insert_txn( + &mut self, + key: K, + val: V, + ) -> BTreeMapEntryTransaction<'_, K, V, TXN> { BTreeMapEntryTransaction::new_insert(self.tree_ref, key, val) } @@ -549,21 +563,23 @@ impl<'a, K: Ord + Debug, V: Clone> BTreeMapTransaction<'a, K, V> { } } -impl<'a, K: Ord + Debug, V: Transactional + Clone> ValTransaction - for BTreeMapTransaction<'a, K, V> +impl<'a, K: Ord + Debug, V: Transactional + Clone, TXN> ValTransaction + for BTreeMapTransaction<'a, K, V, TXN> { + type TXN = TXN; + fn commit(self) { self.commit_memory(); } - fn apply_to_txn(&self, txn: &mut Transaction) -> MetadataModelResult<()> { + async fn apply_to_txn(&self, txn: &mut Self::TXN) -> MetadataModelResult<()> { // Add the staging operation to txn for (k, op) in &self.staging { match op { - BTreeMapOp::Insert(v) => v.upsert_in_transaction(txn)?, + BTreeMapOp::Insert(v) => v.upsert_in_transaction(txn).await?, BTreeMapOp::Delete => { if let Some(v) = self.tree_ref.get(k) { - v.delete_in_transaction(txn)?; + v.delete_in_transaction(txn).await?; } } } @@ -573,24 +589,26 @@ impl<'a, K: Ord + Debug, V: Transactional + Clone> ValTransaction } /// Transaction wrapper for a `BTreeMap` entry value of given `key` -pub struct BTreeMapEntryTransaction<'a, K, V> { +pub struct BTreeMapEntryTransaction<'a, K, V, TXN> { tree_ref: &'a mut BTreeMap, pub key: K, pub new_value: V, + _phantom: PhantomData, } -impl<'a, K: Ord + Debug, V: Clone> BTreeMapEntryTransaction<'a, K, V> { +impl<'a, K: Ord + Debug, V: Clone, TXN> BTreeMapEntryTransaction<'a, K, V, TXN> { /// Create a `ValTransaction` that wraps a `BTreeMap` entry of the given `key`. /// If the tree does not contain `key`, the `default_val` will be used as the initial value pub fn new_insert( tree_ref: &'a mut BTreeMap, key: K, value: V, - ) -> BTreeMapEntryTransaction<'a, K, V> { + ) -> BTreeMapEntryTransaction<'a, K, V, TXN> { BTreeMapEntryTransaction { new_value: value, tree_ref, key, + _phantom: PhantomData, } } @@ -604,7 +622,7 @@ impl<'a, K: Ord + Debug, V: Clone> BTreeMapEntryTransaction<'a, K, V> { tree_ref: &'a mut BTreeMap, key: K, default_val: Option, - ) -> Option> { + ) -> Option> { tree_ref .get(&key) .cloned() @@ -613,11 +631,12 @@ impl<'a, K: Ord + Debug, V: Clone> BTreeMapEntryTransaction<'a, K, V> { new_value: orig_value, tree_ref, key, + _phantom: PhantomData, }) } } -impl<'a, K, V> Deref for BTreeMapEntryTransaction<'a, K, V> { +impl<'a, K, V, TXN> Deref for BTreeMapEntryTransaction<'a, K, V, TXN> { type Target = V; fn deref(&self) -> &Self::Target { @@ -625,24 +644,26 @@ impl<'a, K, V> Deref for BTreeMapEntryTransaction<'a, K, V> { } } -impl<'a, K, V> DerefMut for BTreeMapEntryTransaction<'a, K, V> { +impl<'a, K, V, TXN> DerefMut for BTreeMapEntryTransaction<'a, K, V, TXN> { fn deref_mut(&mut self) -> &mut Self::Target { &mut self.new_value } } -impl<'a, K: Ord, V: PartialEq + Transactional> ValTransaction - for BTreeMapEntryTransaction<'a, K, V> +impl<'a, K: Ord, V: PartialEq + Transactional, TXN> ValTransaction + for BTreeMapEntryTransaction<'a, K, V, TXN> { + type TXN = TXN; + fn commit(self) { self.tree_ref.insert(self.key, self.new_value); } - fn apply_to_txn(&self, txn: &mut Transaction) -> MetadataModelResult<()> { + async fn apply_to_txn(&self, txn: &mut Self::TXN) -> MetadataModelResult<()> { if !self.tree_ref.contains_key(&self.key) || *self.tree_ref.get(&self.key).unwrap() != self.new_value { - self.new_value.upsert_in_transaction(txn)? + self.new_value.upsert_in_transaction(txn).await? } Ok(()) } @@ -661,8 +682,9 @@ mod tests { const TEST_CF: &str = "test-cf"; - impl Transactional for TestTransactional { - fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + #[async_trait] + impl Transactional for TestTransactional { + async fn upsert_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { trx.put( TEST_CF.to_string(), self.key.as_bytes().into(), @@ -671,14 +693,14 @@ mod tests { Ok(()) } - fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { + async fn delete_in_transaction(&self, trx: &mut Transaction) -> MetadataModelResult<()> { trx.delete(TEST_CF.to_string(), self.key.as_bytes().into()); Ok(()) } } - #[test] - fn test_simple_var_transaction_commit() { + #[tokio::test] + async fn test_simple_var_transaction_commit() { let mut kv = TestTransactional { key: "key", value: "original", @@ -687,7 +709,7 @@ mod tests { num_txn.value = "modified"; assert_eq!(num_txn.value, "modified"); let mut txn = Transaction::default(); - num_txn.apply_to_txn(&mut txn).unwrap(); + num_txn.apply_to_txn(&mut txn).await.unwrap(); let txn_op = txn.get_operations(); assert_eq!(1, txn_op.len()); assert!(matches!( @@ -717,8 +739,8 @@ mod tests { assert_eq!("original", kv.value); } - #[test] - fn test_tree_map_transaction_commit() { + #[tokio::test] + async fn test_tree_map_transaction_commit() { let mut map: BTreeMap = BTreeMap::new(); map.insert( "to-remove".to_string(), @@ -800,7 +822,7 @@ mod tests { ); let mut txn = Transaction::default(); - map_txn.apply_to_txn(&mut txn).unwrap(); + map_txn.apply_to_txn(&mut txn).await.unwrap(); let txn_ops = txn.get_operations(); assert_eq!(5, txn_ops.len()); for op in txn_ops { @@ -860,8 +882,8 @@ mod tests { assert_eq!(map_copy, map); } - #[test] - fn test_tree_map_entry_update_transaction_commit() { + #[tokio::test] + async fn test_tree_map_entry_update_transaction_commit() { let mut map: BTreeMap = BTreeMap::new(); map.insert( "first".to_string(), @@ -875,7 +897,7 @@ mod tests { let mut first_entry_txn = map_txn.new_entry_txn("first".to_string()).unwrap(); first_entry_txn.value = "first-value"; let mut txn = Transaction::default(); - first_entry_txn.apply_to_txn(&mut txn).unwrap(); + first_entry_txn.apply_to_txn(&mut txn).await.unwrap(); let txn_ops = txn.get_operations(); assert_eq!(1, txn_ops.len()); assert!( @@ -885,8 +907,8 @@ mod tests { assert_eq!("first-value", map.get("first").unwrap().value); } - #[test] - fn test_tree_map_entry_insert_transaction_commit() { + #[tokio::test] + async fn test_tree_map_entry_insert_transaction_commit() { let mut map: BTreeMap = BTreeMap::new(); let mut map_txn = BTreeMapTransaction::new(&mut map); @@ -898,7 +920,7 @@ mod tests { }, ); let mut txn = Transaction::default(); - first_entry_txn.apply_to_txn(&mut txn).unwrap(); + first_entry_txn.apply_to_txn(&mut txn).await.unwrap(); let txn_ops = txn.get_operations(); assert_eq!(1, txn_ops.len()); assert!( diff --git a/src/meta/src/model_v2/ext/hummock.rs b/src/meta/src/model_v2/ext/hummock.rs new file mode 100644 index 0000000000000..77111e2e7d202 --- /dev/null +++ b/src/meta/src/model_v2/ext/hummock.rs @@ -0,0 +1,61 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_pb::hummock::HummockPinnedVersion; +use sea_orm::sea_query::OnConflict; +use sea_orm::ActiveValue::{Set, Unchanged}; +use sea_orm::EntityTrait; + +use crate::model::{MetadataModelResult, Transactional}; +use crate::model_v2::hummock_pinned_version; +use crate::model_v2::trx::Transaction; + +#[async_trait::async_trait] +impl Transactional for HummockPinnedVersion { + async fn upsert_in_transaction( + &self, + trx: &mut crate::model_v2::trx::Transaction, + ) -> MetadataModelResult<()> { + // TODO: error type conversion + // TODO: integer type conversion + let m = hummock_pinned_version::ActiveModel { + context_id: Unchanged(self.context_id.try_into().unwrap()), + min_pinned_id: Set(self.min_pinned_id.try_into().unwrap()), + }; + hummock_pinned_version::Entity::insert(m) + .on_conflict( + OnConflict::column(hummock_pinned_version::Column::ContextId) + .update_columns([hummock_pinned_version::Column::MinPinnedId]) + .to_owned(), + ) + .exec(trx) + .await + .unwrap(); + Ok(()) + } + + async fn delete_in_transaction( + &self, + trx: &mut crate::model_v2::trx::Transaction, + ) -> MetadataModelResult<()> { + // TODO: error type conversion + // TODO: integer type conversion + let id: i32 = self.context_id.try_into().unwrap(); + hummock_pinned_version::Entity::delete_by_id(id) + .exec(trx) + .await + .unwrap(); + Ok(()) + } +} diff --git a/src/meta/src/model_v2/ext/mod.rs b/src/meta/src/model_v2/ext/mod.rs new file mode 100644 index 0000000000000..47a5ce8623dc4 --- /dev/null +++ b/src/meta/src/model_v2/ext/mod.rs @@ -0,0 +1,16 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod hummock; +pub use hummock::*; diff --git a/src/meta/src/model_v2/mod.rs b/src/meta/src/model_v2/mod.rs index dfc4dc43fc3f1..6370271551e88 100644 --- a/src/meta/src/model_v2/mod.rs +++ b/src/meta/src/model_v2/mod.rs @@ -26,6 +26,7 @@ pub mod connection; pub mod database; pub mod election_leader; pub mod election_member; +pub mod ext; pub mod fragment; pub mod function; pub mod hummock_pinned_snapshot; @@ -40,6 +41,7 @@ pub mod sink; pub mod source; pub mod system_parameter; pub mod table; +pub mod trx; pub mod user; pub mod user_privilege; pub mod view; diff --git a/src/meta/src/model_v2/trx.rs b/src/meta/src/model_v2/trx.rs new file mode 100644 index 0000000000000..4bfe6d0261de4 --- /dev/null +++ b/src/meta/src/model_v2/trx.rs @@ -0,0 +1,276 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub type Transaction = sea_orm::DatabaseTransaction; + +#[cfg(not(madsim))] +#[cfg(test)] +mod tests { + use std::collections::BTreeMap; + + use risingwave_pb::hummock::HummockPinnedVersion; + use sea_orm::{EntityTrait, TransactionTrait}; + + use crate::controller::SqlMetaStore; + use crate::model::{BTreeMapTransaction, ValTransaction, VarTransaction}; + use crate::model_v2::hummock_pinned_version::Model as HummockPinnedVersionModel; + use crate::model_v2::prelude::HummockPinnedVersion as HummockPinnedVersionEntity; + use crate::model_v2::trx::Transaction; + + #[tokio::test] + async fn test_simple_var_transaction_commit() { + let store = SqlMetaStore::for_test().await; + let db = &store.conn; + let mut kv = HummockPinnedVersion { + context_id: 1, + min_pinned_id: 2, + }; + let mut num_txn = VarTransaction::<'_, Transaction, _>::new(&mut kv); + num_txn.min_pinned_id = 3; + assert_eq!(num_txn.min_pinned_id, 3); + let mut txn = db.begin().await.unwrap(); + num_txn.apply_to_txn(&mut txn).await.unwrap(); + txn.commit().await.unwrap(); + let db_val = HummockPinnedVersionEntity::find_by_id(1) + .one(db) + .await + .unwrap() + .unwrap(); + assert_eq!(db_val.min_pinned_id, 3); + num_txn.commit(); + assert_eq!(kv.min_pinned_id, 3); + } + + #[test] + fn test_simple_var_transaction_abort() { + let mut kv = HummockPinnedVersion { + context_id: 1, + min_pinned_id: 11, + }; + let mut num_txn = VarTransaction::<'_, Transaction, _>::new(&mut kv); + num_txn.min_pinned_id = 2; + num_txn.abort(); + assert_eq!(11, kv.min_pinned_id); + } + + #[tokio::test] + async fn test_tree_map_transaction_commit() { + let mut map: BTreeMap = BTreeMap::new(); + // to remove + map.insert( + 1, + HummockPinnedVersion { + context_id: 1, + min_pinned_id: 11, + }, + ); + // to-remove-after-modify + map.insert( + 2, + HummockPinnedVersion { + context_id: 2, + min_pinned_id: 22, + }, + ); + // first + map.insert( + 3, + HummockPinnedVersion { + context_id: 3, + min_pinned_id: 33, + }, + ); + + let mut map_copy = map.clone(); + let mut map_txn = BTreeMapTransaction::new(&mut map); + map_txn.remove(1); + map_txn.insert( + 2, + HummockPinnedVersion { + context_id: 2, + min_pinned_id: 0, + }, + ); + map_txn.remove(2); + // first + map_txn.insert( + 3, + HummockPinnedVersion { + context_id: 3, + min_pinned_id: 333, + }, + ); + // second + map_txn.insert( + 4, + HummockPinnedVersion { + context_id: 4, + min_pinned_id: 44, + }, + ); + assert_eq!( + &HummockPinnedVersion { + context_id: 4, + min_pinned_id: 44 + }, + map_txn.get(&4).unwrap() + ); + // third + map_txn.insert( + 5, + HummockPinnedVersion { + context_id: 5, + min_pinned_id: 55, + }, + ); + assert_eq!( + &HummockPinnedVersion { + context_id: 5, + min_pinned_id: 55 + }, + map_txn.get(&5).unwrap() + ); + + let mut third_entry = map_txn.get_mut(5).unwrap(); + third_entry.min_pinned_id = 555; + assert_eq!( + &HummockPinnedVersion { + context_id: 5, + min_pinned_id: 555 + }, + map_txn.get(&5).unwrap() + ); + + let store = SqlMetaStore::for_test().await; + let db = &store.conn; + let mut txn = db.begin().await.unwrap(); + map_txn.apply_to_txn(&mut txn).await.unwrap(); + txn.commit().await.unwrap(); + + let db_rows: Vec = + HummockPinnedVersionEntity::find().all(db).await.unwrap(); + assert_eq!(db_rows.len(), 3); + assert_eq!( + 1, + db_rows + .iter() + .filter(|m| m.context_id == 3 && m.min_pinned_id == 333) + .count() + ); + assert_eq!( + 1, + db_rows + .iter() + .filter(|m| m.context_id == 4 && m.min_pinned_id == 44) + .count() + ); + assert_eq!( + 1, + db_rows + .iter() + .filter(|m| m.context_id == 5 && m.min_pinned_id == 555) + .count() + ); + map_txn.commit(); + + // replay the change to local copy and compare + map_copy.remove(&1).unwrap(); + map_copy.insert( + 2, + HummockPinnedVersion { + context_id: 2, + min_pinned_id: 22, + }, + ); + map_copy.remove(&2).unwrap(); + map_copy.insert( + 3, + HummockPinnedVersion { + context_id: 3, + min_pinned_id: 333, + }, + ); + map_copy.insert( + 4, + HummockPinnedVersion { + context_id: 4, + min_pinned_id: 44, + }, + ); + map_copy.insert( + 5, + HummockPinnedVersion { + context_id: 5, + min_pinned_id: 555, + }, + ); + assert_eq!(map_copy, map); + } + + #[tokio::test] + async fn test_tree_map_entry_update_transaction_commit() { + let mut map: BTreeMap = BTreeMap::new(); + map.insert( + 1, + HummockPinnedVersion { + context_id: 1, + min_pinned_id: 11, + }, + ); + + let mut map_txn = BTreeMapTransaction::new(&mut map); + let mut first_entry_txn = map_txn.new_entry_txn(1).unwrap(); + first_entry_txn.min_pinned_id = 111; + + let store = SqlMetaStore::for_test().await; + let db = &store.conn; + let mut txn = db.begin().await.unwrap(); + first_entry_txn.apply_to_txn(&mut txn).await.unwrap(); + txn.commit().await.unwrap(); + first_entry_txn.commit(); + + let db_rows: Vec = + HummockPinnedVersionEntity::find().all(db).await.unwrap(); + assert_eq!(db_rows.len(), 1); + assert_eq!( + 1, + db_rows + .iter() + .filter(|m| m.context_id == 1 && m.min_pinned_id == 111) + .count() + ); + assert_eq!(111, map.get(&1).unwrap().min_pinned_id); + } + + #[tokio::test] + async fn test_tree_map_entry_insert_transaction_commit() { + let mut map: BTreeMap = BTreeMap::new(); + + let mut map_txn = BTreeMapTransaction::new(&mut map); + let first_entry_txn = map_txn.new_entry_insert_txn( + 1, + HummockPinnedVersion { + context_id: 1, + min_pinned_id: 11, + }, + ); + let store = SqlMetaStore::for_test().await; + let db = &store.conn; + let mut txn = db.begin().await.unwrap(); + first_entry_txn.apply_to_txn(&mut txn).await.unwrap(); + txn.commit().await.unwrap(); + first_entry_txn.commit(); + assert_eq!(11, map.get(&1).unwrap().min_pinned_id); + } +} From f1021932f27711dd07487b95089c6b968e1faf79 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 18 Oct 2023 16:07:12 +0800 Subject: [PATCH 04/58] refactor(meta): split meta into smaller crates (step 1) (#12924) --- Cargo.lock | 87 +++-- Cargo.toml | 5 + src/batch/Cargo.toml | 2 - src/batch/src/lib.rs | 1 - src/cmd/Cargo.toml | 2 +- src/cmd/src/lib.rs | 4 +- src/cmd_all/Cargo.toml | 2 +- src/cmd_all/src/README.md | 2 +- src/cmd_all/src/bin/risingwave.rs | 2 +- src/cmd_all/src/playground.rs | 4 +- src/cmd_all/src/standalone.rs | 4 +- src/common/heap_profiling/Cargo.toml | 10 +- src/common/src/lib.rs | 1 - src/compute/src/lib.rs | 1 - src/connector/Cargo.toml | 6 +- src/connector/src/lib.rs | 1 - src/frontend/Cargo.toml | 1 - src/meta/Cargo.toml | 5 - src/meta/README.md | 9 + src/meta/node/Cargo.toml | 57 +++ src/meta/node/src/lib.rs | 344 ++++++++++++++++++ src/meta/{src/rpc => node/src}/server.rs | 103 ++---- src/meta/service/Cargo.toml | 53 +++ .../service => service/src}/backup_service.rs | 0 .../service => service/src}/cloud_service.rs | 0 .../src}/cluster_service.rs | 0 .../service => service/src}/ddl_service.rs | 0 .../service => service/src}/health_service.rs | 6 + .../src}/heartbeat_service.rs | 0 .../src}/hummock_service.rs | 2 +- .../rpc/service/mod.rs => service/src/lib.rs} | 30 ++ .../src}/meta_member_service.rs | 4 +- .../src}/notification_service.rs | 0 .../service => service/src}/scale_service.rs | 0 .../src}/serving_service.rs | 0 .../src}/sink_coordination_service.rs | 4 +- .../service => service/src}/stream_service.rs | 0 .../src}/system_params_service.rs | 0 .../src}/telemetry_service.rs | 0 .../service => service/src}/user_service.rs | 0 src/meta/src/barrier/mod.rs | 4 +- src/meta/src/barrier/recovery.rs | 2 +- src/meta/src/barrier/schedule.rs | 2 +- src/meta/src/controller/mod.rs | 2 +- src/meta/src/hummock/compaction/mod.rs | 4 +- .../src/hummock/compaction/selector/mod.rs | 5 +- src/meta/src/hummock/manager/checkpoint.rs | 10 +- src/meta/src/hummock/manager/context.rs | 2 +- src/meta/src/hummock/manager/mod.rs | 2 +- src/meta/src/hummock/manager/worker.rs | 2 +- .../hummock/model/compaction_group_config.rs | 4 +- src/meta/src/lib.rs | 343 +---------------- src/meta/src/manager/mod.rs | 21 +- .../sink_coordination/coordinator_worker.rs | 6 +- .../src/manager/sink_coordination/manager.rs | 8 +- src/meta/src/manager/sink_coordination/mod.rs | 16 +- src/meta/src/manager/streaming_job.rs | 4 +- src/meta/src/model/cluster.rs | 19 +- src/meta/src/model/stream.rs | 10 +- src/meta/src/rpc/ddl_controller.rs | 6 +- src/meta/src/rpc/election/sql.rs | 6 +- src/meta/src/rpc/metrics.rs | 6 +- src/meta/src/rpc/mod.rs | 14 +- src/meta/src/serving/mod.rs | 4 +- src/meta/src/stream/scale.rs | 4 +- src/meta/src/stream/source_manager.rs | 2 +- src/meta/src/stream/stream_manager.rs | 10 +- src/meta/src/telemetry.rs | 11 +- src/rpc_client/src/lib.rs | 1 - src/source/Cargo.toml | 1 - src/source/src/lib.rs | 1 - src/storage/Cargo.toml | 1 - src/storage/backup/src/lib.rs | 1 - src/storage/compactor/Cargo.toml | 3 - src/storage/src/lib.rs | 1 - src/stream/Cargo.toml | 5 - src/stream/src/lib.rs | 1 - src/tests/compaction_test/Cargo.toml | 1 + .../src/compaction_test_runner.rs | 4 +- src/tests/simulation/Cargo.toml | 2 +- src/tests/simulation/src/cluster.rs | 4 +- src/utils/runtime/Cargo.toml | 5 - 82 files changed, 722 insertions(+), 590 deletions(-) create mode 100644 src/meta/README.md create mode 100644 src/meta/node/Cargo.toml create mode 100644 src/meta/node/src/lib.rs rename src/meta/{src/rpc => node/src}/server.rs (92%) create mode 100644 src/meta/service/Cargo.toml rename src/meta/{src/rpc/service => service/src}/backup_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/cloud_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/cluster_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/ddl_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/health_service.rs (93%) rename src/meta/{src/rpc/service => service/src}/heartbeat_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/hummock_service.rs (99%) rename src/meta/{src/rpc/service/mod.rs => service/src/lib.rs} (71%) rename src/meta/{src/rpc/service => service/src}/meta_member_service.rs (97%) rename src/meta/{src/rpc/service => service/src}/notification_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/scale_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/serving_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/sink_coordination_service.rs (92%) rename src/meta/{src/rpc/service => service/src}/stream_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/system_params_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/telemetry_service.rs (100%) rename src/meta/{src/rpc/service => service/src}/user_service.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index 19854624d08ba..247852213c422 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4036,12 +4036,6 @@ dependencies = [ "windows-sys 0.48.0", ] -[[package]] -name = "iter-chunks" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b7abddfc4e19bc38f3922e41b341fedb4e1470e922f024c4e5ae5922f56c7593" - [[package]] name = "itertools" version = "0.10.5" @@ -6893,7 +6887,6 @@ dependencies = [ "anyhow", "assert_matches", "async-recursion", - "async-stream", "async-trait", "criterion", "either", @@ -6927,7 +6920,6 @@ dependencies = [ "tokio-metrics", "tokio-stream", "tracing", - "uuid", "workspace-hack", ] @@ -6977,7 +6969,7 @@ dependencies = [ "risingwave_ctl", "risingwave_expr_impl", "risingwave_frontend", - "risingwave_meta", + "risingwave_meta_node", "risingwave_rt", "task_stats_alloc", "tikv-jemallocator", @@ -7003,7 +6995,7 @@ dependencies = [ "risingwave_ctl", "risingwave_expr_impl", "risingwave_frontend", - "risingwave_meta", + "risingwave_meta_node", "risingwave_rt", "shell-words", "strum 0.25.0", @@ -7172,6 +7164,7 @@ dependencies = [ "risingwave_hummock_sdk", "risingwave_hummock_test", "risingwave_meta", + "risingwave_meta_node", "risingwave_object_store", "risingwave_pb", "risingwave_rpc_client", @@ -7185,14 +7178,12 @@ dependencies = [ name = "risingwave_compactor" version = "1.3.0-alpha" dependencies = [ - "anyhow", "async-trait", "await-tree", "clap", "madsim-tokio", "madsim-tonic", "parking_lot 0.12.1", - "prometheus", "risingwave_common", "risingwave_common_heap_profiling", "risingwave_common_service", @@ -7201,7 +7192,6 @@ dependencies = [ "risingwave_rpc_client", "risingwave_storage", "serde", - "serde_json", "tracing", "workspace-hack", ] @@ -7266,7 +7256,6 @@ dependencies = [ "aws-smithy-http", "aws-types", "base64 0.21.4", - "bincode 1.3.3", "byteorder", "bytes", "chrono", @@ -7290,15 +7279,12 @@ dependencies = [ "jsonschema-transpiler", "madsim-rdkafka", "madsim-tokio", - "madsim-tonic", "maplit", "moka", "mysql_async", "mysql_common", "nexmark", - "nkeys", "num-bigint", - "opendal 0.39.0", "parking_lot 0.12.1", "paste", "prometheus", @@ -7493,7 +7479,6 @@ dependencies = [ "madsim-tonic", "maplit", "md5", - "more-asserts", "num-integer", "parking_lot 0.12.1", "parse-display", @@ -7672,17 +7657,14 @@ dependencies = [ "num-integer", "num-traits", "parking_lot 0.12.1", - "parse-display", "prometheus", "prometheus-http-query", "prost 0.12.1", "rand", - "regex", "reqwest", "risingwave_backup", "risingwave_common", "risingwave_common_heap_profiling", - "risingwave_common_service", "risingwave_connector", "risingwave_hummock_sdk", "risingwave_object_store", @@ -7695,9 +7677,7 @@ dependencies = [ "serde", "serde_json", "sqlx", - "static_assertions", "sync-point", - "tempfile", "thiserror", "tokio-retry", "tokio-stream", @@ -7709,6 +7689,56 @@ dependencies = [ "workspace-hack", ] +[[package]] +name = "risingwave_meta_node" +version = "1.3.0-alpha" +dependencies = [ + "anyhow", + "clap", + "either", + "futures", + "itertools 0.11.0", + "madsim-etcd-client", + "madsim-tokio", + "madsim-tonic", + "model_migration", + "prometheus-http-query", + "regex", + "risingwave_common", + "risingwave_common_heap_profiling", + "risingwave_common_service", + "risingwave_meta", + "risingwave_meta_service", + "risingwave_pb", + "risingwave_rpc_client", + "sea-orm", + "tracing", + "workspace-hack", +] + +[[package]] +name = "risingwave_meta_service" +version = "1.3.0-alpha" +dependencies = [ + "anyhow", + "async-trait", + "either", + "futures", + "itertools 0.11.0", + "madsim-tokio", + "madsim-tonic", + "regex", + "risingwave_common", + "risingwave_connector", + "risingwave_meta", + "risingwave_pb", + "sea-orm", + "sync-point", + "tokio-stream", + "tracing", + "workspace-hack", +] + [[package]] name = "risingwave_object_store" version = "1.3.0-alpha" @@ -7824,7 +7854,6 @@ name = "risingwave_rt" version = "1.3.0-alpha" dependencies = [ "await-tree", - "chrono", "console", "console-subscriber", "either", @@ -7836,7 +7865,6 @@ dependencies = [ "opentelemetry-semantic-conventions", "parking_lot 0.12.1", "pprof", - "prometheus", "risingwave_common", "risingwave_variables", "rlimit", @@ -7878,7 +7906,7 @@ dependencies = [ "risingwave_e2e_extended_mode_test", "risingwave_expr_impl", "risingwave_frontend", - "risingwave_meta", + "risingwave_meta_node", "risingwave_pb", "risingwave_rpc_client", "risingwave_sqlparser", @@ -7902,7 +7930,6 @@ dependencies = [ "anyhow", "assert_matches", "criterion", - "easy-ext", "futures", "futures-async-stream", "itertools 0.11.0", @@ -7996,7 +8023,6 @@ dependencies = [ name = "risingwave_storage" version = "1.3.0-alpha" dependencies = [ - "anyhow", "arc-swap", "async-trait", "auto_enums", @@ -8069,7 +8095,6 @@ dependencies = [ "await-tree", "bytes", "criterion", - "dyn-clone", "educe", "either", "enum-as-inner", @@ -8078,7 +8103,6 @@ dependencies = [ "futures-async-stream", "governor", "hytra", - "iter-chunks", "itertools 0.11.0", "local_stats_alloc", "lru 0.7.6", @@ -8087,9 +8111,7 @@ dependencies = [ "maplit", "memcomparable", "multimap 0.9.0", - "num-traits", "parking_lot 0.12.1", - "parse-display", "pin-project", "prometheus", "prost 0.12.1", @@ -8108,7 +8130,6 @@ dependencies = [ "serde_json", "serde_yaml", "smallvec", - "spin 0.9.8", "static_assertions", "task_stats_alloc", "thiserror", diff --git a/Cargo.toml b/Cargo.toml index a975a0186e712..74a097d4eb9d7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -7,6 +7,7 @@ members = [ "src/cmd_all", "src/common", "src/common/common_service", + "src/common/heap_profiling", "src/compute", "src/connector", "src/ctl", @@ -18,6 +19,8 @@ members = [ "src/java_binding", "src/jni_core", "src/meta", + "src/meta/node", + "src/meta/service", "src/meta/src/model_v2/migration", "src/object_store", "src/prost", @@ -139,6 +142,8 @@ risingwave_hummock_sdk = { path = "./src/storage/hummock_sdk" } risingwave_hummock_test = { path = "./src/storage/hummock_test" } risingwave_hummock_trace = { path = "./src/storage/hummock_trace" } risingwave_meta = { path = "./src/meta" } +risingwave_meta_service = { path = "./src/meta/service" } +risingwave_meta_node = { path = "./src/meta/node" } risingwave_object_store = { path = "./src/object_store" } risingwave_pb = { path = "./src/prost" } risingwave_rpc_client = { path = "./src/rpc_client" } diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 17a19d4771f60..fef154450a563 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -17,7 +17,6 @@ normal = ["workspace-hack"] anyhow = "1" assert_matches = "1" async-recursion = "1" -async-stream = "0.3.5" async-trait = "0.1" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } @@ -53,7 +52,6 @@ tokio-metrics = "0.3.0" tokio-stream = "0.1" tonic = { workspace = true } tracing = "0.1" -uuid = "1" [target.'cfg(enable_task_local_alloc)'.dependencies] task_stats_alloc = { path = "../utils/task_stats_alloc" } diff --git a/src/batch/src/lib.rs b/src/batch/src/lib.rs index 17fc6cfab2bc1..9104c96c951f5 100644 --- a/src/batch/src/lib.rs +++ b/src/batch/src/lib.rs @@ -15,7 +15,6 @@ #![expect(dead_code)] #![allow(clippy::derive_partial_eq_without_eq)] #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(exact_size_is_empty)] #![feature(type_alias_impl_trait)] #![cfg_attr(coverage, feature(no_coverage))] diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 78dc7040f2ada..8ba72d6a24af4 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -27,7 +27,7 @@ risingwave_compute = { workspace = true } risingwave_ctl = { workspace = true } risingwave_expr_impl = { workspace = true } risingwave_frontend = { workspace = true } -risingwave_meta = { workspace = true } +risingwave_meta_node = { workspace = true } risingwave_rt = { workspace = true } tikv-jemallocator = { workspace = true, features = [ "unprefixed_malloc_on_supported_platforms", diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index 7dd34a8364c9a..8d3629eaf657f 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -16,7 +16,7 @@ use risingwave_compactor::CompactorOpts; use risingwave_compute::ComputeNodeOpts; use risingwave_ctl::CliOpts as CtlOpts; use risingwave_frontend::FrontendOpts; -use risingwave_meta::MetaNodeOpts; +use risingwave_meta_node::MetaNodeOpts; use risingwave_rt::{init_risingwave_logger, main_okk, LoggerSettings}; /// Define the `main` function for a component. @@ -48,7 +48,7 @@ pub fn compute(opts: ComputeNodeOpts) { pub fn meta(opts: MetaNodeOpts) { init_risingwave_logger(LoggerSettings::new("meta")); - main_okk(risingwave_meta::start(opts)); + main_okk(risingwave_meta_node::start(opts)); } pub fn frontend(opts: FrontendOpts) { diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index 922fd3b5812bc..9a4b34c094196 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -31,7 +31,7 @@ risingwave_compute = { workspace = true } risingwave_ctl = { workspace = true } risingwave_expr_impl = { workspace = true } risingwave_frontend = { workspace = true } -risingwave_meta = { workspace = true } +risingwave_meta_node = { workspace = true } risingwave_rt = { workspace = true } shell-words = "1.1.0" strum = "0.25" diff --git a/src/cmd_all/src/README.md b/src/cmd_all/src/README.md index fbbae4439f97f..0284817b99a92 100644 --- a/src/cmd_all/src/README.md +++ b/src/cmd_all/src/README.md @@ -26,5 +26,5 @@ You may run and reference the [demo script](../scripts/e2e-full-standalone-demo. Standalone mode simply passes the options to the corresponding node, and starts them in the same process. -For example `--meta-opts` is parsed, and then Meta Node's entrypoint, `risingwave_meta::start`, is called with the parsed options. +For example `--meta-opts` is parsed, and then Meta Node's entrypoint, `risingwave_meta_node::start`, is called with the parsed options. If any option is missing, the corresponding node will not be started. \ No newline at end of file diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index 1d92ca768f88d..3e9088e16b9e2 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -25,7 +25,7 @@ use risingwave_compactor::CompactorOpts; use risingwave_compute::ComputeNodeOpts; use risingwave_ctl::CliOpts as CtlOpts; use risingwave_frontend::FrontendOpts; -use risingwave_meta::MetaNodeOpts; +use risingwave_meta_node::MetaNodeOpts; use strum::IntoEnumIterator; use strum_macros::{Display, EnumIter, EnumString, IntoStaticStr}; use tracing::Level; diff --git a/src/cmd_all/src/playground.rs b/src/cmd_all/src/playground.rs index b1fd1a30ef461..76ca89be17c76 100644 --- a/src/cmd_all/src/playground.rs +++ b/src/cmd_all/src/playground.rs @@ -159,9 +159,9 @@ pub async fn playground(opts: PlaygroundOpts) -> Result<()> { RisingWaveService::Meta(mut opts) => { opts.insert(0, "meta-node".into()); tracing::info!("starting meta-node thread with cli args: {:?}", opts); - let opts = risingwave_meta::MetaNodeOpts::parse_from(opts); + let opts = risingwave_meta_node::MetaNodeOpts::parse_from(opts); let _meta_handle = tokio::spawn(async move { - risingwave_meta::start(opts).await; + risingwave_meta_node::start(opts).await; tracing::warn!("meta is stopped, shutdown all nodes"); // As a playground, it's fine to just kill everything. if idle_exit { diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index f7c8068cf33b9..8ebe2c7112c49 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -17,7 +17,7 @@ use clap::Parser; use risingwave_compactor::CompactorOpts; use risingwave_compute::ComputeNodeOpts; use risingwave_frontend::FrontendOpts; -use risingwave_meta::MetaNodeOpts; +use risingwave_meta_node::MetaNodeOpts; use shell_words::split; use tokio::signal; @@ -142,7 +142,7 @@ pub async fn standalone(opts: StandaloneOpts) -> Result<()> { tracing::info!("starting meta-node thread with cli args: {:?}", opts); let _meta_handle = tokio::spawn(async move { - risingwave_meta::start(opts).await; + risingwave_meta_node::start(opts).await; tracing::warn!("meta is stopped, shutdown all nodes"); }); // wait for the service to be ready diff --git a/src/common/heap_profiling/Cargo.toml b/src/common/heap_profiling/Cargo.toml index 6c1b9957555bd..c7123eaac5817 100644 --- a/src/common/heap_profiling/Cargo.toml +++ b/src/common/heap_profiling/Cargo.toml @@ -15,16 +15,16 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -tikv-jemalloc-ctl = { workspace = true } -risingwave_common = {workspace =true} -tokio = { version = "0.2", package = "madsim-tokio" } -tracing = "0.1" +anyhow = "1" chrono = { version = "0.4", default-features = false, features = [ "clock", "std", ] } -anyhow = "1" parking_lot = "0.12" +risingwave_common = { workspace = true } +tikv-jemalloc-ctl = { workspace = true } +tokio = { version = "0.2", package = "madsim-tokio" } +tracing = "0.1" [lints] workspace = true diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index 228fb6200b667..2a3575d8dae78 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -18,7 +18,6 @@ )] #![feature(extract_if)] #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(is_sorted)] #![feature(type_alias_impl_trait)] #![feature(test)] diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index bdd84ae402746..65bf59eedf19e 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -13,7 +13,6 @@ // limitations under the License. #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(generators)] #![feature(type_alias_impl_trait)] #![feature(let_chains)] diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index f0890080c8c5d..abb7486de3091 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -34,7 +34,6 @@ aws-sdk-s3 = { workspace = true } aws-smithy-http = { workspace = true } aws-types = { workspace = true } base64 = "0.21" -bincode = "1" byteorder = "1" bytes = { version = "1", features = ["serde"] } chrono = { version = "0.4", default-features = false, features = [ @@ -65,7 +64,7 @@ icelake = { workspace = true } indexmap = { version = "1.9.3", features = ["serde"] } itertools = "0.11" jni = { version = "0.21.1", features = ["invocation"] } -jsonschema-transpiler = { git = "https://github.com/mozilla/jsonschema-transpiler", rev = "c1a89d720d118843d8bcca51084deb0ed223e4b4" } +jst = { package = 'jsonschema-transpiler', git = "https://github.com/mozilla/jsonschema-transpiler", rev = "c1a89d720d118843d8bcca51084deb0ed223e4b4" } maplit = "1.0.2" moka = { version = "0.12", features = ["future"] } mysql_async = { version = "0.32", default-features = false, features = [ @@ -75,9 +74,7 @@ mysql_common = { version = "0.30", default-features = false, features = [ "chrono", ] } nexmark = { version = "0.2", features = ["serde"] } -nkeys = "0.3.2" num-bigint = "0.4" -opendal = "0.39" parking_lot = "0.12" paste = "1" prometheus = { version = "0.13", features = ["process"] } @@ -127,7 +124,6 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ tokio-retry = "0.3" tokio-stream = "0.1" tokio-util = { version = "0.7", features = ["codec", "io"] } -tonic = { workspace = true } tonic_0_9 = { package = "tonic", version = "0.9" } tracing = "0.1" tracing-futures = { version = "0.2", features = ["futures-03"] } diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 4dd1691b00f89..8201e10731a28 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -19,7 +19,6 @@ #![feature(stmt_expr_attributes)] #![feature(box_patterns)] #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(lint_reasons)] #![feature(lazy_cell)] #![feature(result_option_inspect)] diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 7ff97bdd375de..37f9f6326faea 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -36,7 +36,6 @@ iana-time-zone = "0.1" itertools = "0.11" maplit = "1" md5 = "0.7.0" -more-asserts = "0.3" num-integer = "0.1" parking_lot = "0.12" parse-display = "0.8" diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index 5bdcdfd633129..67e9a95026cc7 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -40,17 +40,14 @@ model_migration = { path = "src/model_v2/migration" } num-integer = "0.1" num-traits = "0.2" parking_lot = { version = "0.12", features = ["arc_lock"] } -parse-display = "0.8" prometheus = "0.13" prometheus-http-query = "0.7" prost = { workspace = true } rand = "0.8" -regex = "1" reqwest = "0.11" risingwave_backup = { workspace = true } risingwave_common = { workspace = true } risingwave_common_heap_profiling = { workspace = true } -risingwave_common_service = { workspace = true } risingwave_connector = { workspace = true } risingwave_hummock_sdk = { workspace = true } risingwave_object_store = { workspace = true } @@ -102,8 +99,6 @@ assert_matches = "1" maplit = "1.0.2" rand = "0.8" risingwave_test_runner = { workspace = true } -static_assertions = "1" -tempfile = "3" [features] test = [] diff --git a/src/meta/README.md b/src/meta/README.md new file mode 100644 index 0000000000000..3782d765532a4 --- /dev/null +++ b/src/meta/README.md @@ -0,0 +1,9 @@ +## Organization of the meta crates + +We split the meta module into smaller crates in order to speed up compilation. + +- `meta/node` is the final meta node server +- `meta/service` is tonic grpc service implementations. We may further split this into parallel sub-crates. +- The remaining part `meta/src` is the implementation details imported by services. In the future, we can also try to re-organize this into smaller units. + +Refer to [#12924](https://github.com/risingwavelabs/risingwave/pull/12924) for more details. diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml new file mode 100644 index 0000000000000..8c2a5aeadbe41 --- /dev/null +++ b/src/meta/node/Cargo.toml @@ -0,0 +1,57 @@ +[package] +name = "risingwave_meta_node" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +keywords = { workspace = true } +license = { workspace = true } +repository = { workspace = true } + +[package.metadata.cargo-machete] +ignored = ["workspace-hack"] + +[package.metadata.cargo-udeps.ignore] +normal = ["workspace-hack"] + +[dependencies] +anyhow = "1" +clap = { version = "4", features = ["derive", "env"] } +either = "1" +etcd-client = { workspace = true } +futures = { version = "0.3", default-features = false, features = ["alloc"] } +itertools = "0.11" +model_migration = { path = "../src/model_v2/migration" } +prometheus-http-query = "0.7" +regex = "1" +risingwave_common = { workspace = true } +risingwave_common_heap_profiling = { workspace = true } +risingwave_common_service = { workspace = true } +risingwave_meta = { workspace = true } +risingwave_meta_service = { workspace = true } +risingwave_pb = { workspace = true } +risingwave_rpc_client = { workspace = true } +sea-orm = { version = "0.12.0", features = [ + "sqlx-mysql", + "sqlx-postgres", + "sqlx-sqlite", + "runtime-tokio-native-tls", + "macros", +] } +tokio = { version = "0.2", package = "madsim-tokio", features = [ + "rt", + "rt-multi-thread", + "sync", + "macros", + "time", + "signal", +] } +tonic = { workspace = true } +tracing = "0.1" + +[target.'cfg(not(madsim))'.dependencies] +workspace-hack = { path = "../../workspace-hack" } + +[dev-dependencies] + +[lints] +workspace = true diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs new file mode 100644 index 0000000000000..55c7b27b0c80a --- /dev/null +++ b/src/meta/node/src/lib.rs @@ -0,0 +1,344 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![feature(lint_reasons)] +#![feature(let_chains)] +#![cfg_attr(coverage, feature(no_coverage))] + +mod server; +use std::time::Duration; + +use clap::Parser; +pub use error::{MetaError, MetaResult}; +use risingwave_common::config::OverrideConfig; +use risingwave_common::util::resource_util; +use risingwave_common::{GIT_SHA, RW_VERSION}; +use risingwave_common_heap_profiling::HeapProfiler; +use risingwave_meta::*; +use risingwave_meta_service::*; +pub use rpc::{ElectionClient, ElectionMember, EtcdElectionClient}; +use server::{rpc_serve, MetaStoreSqlBackend}; + +use crate::manager::MetaOpts; + +#[derive(Debug, Clone, Parser, OverrideConfig)] +#[command(version, about = "The central metadata management service")] +pub struct MetaNodeOpts { + #[clap(long, env = "RW_VPC_ID")] + vpc_id: Option, + + #[clap(long, env = "RW_VPC_SECURITY_GROUP_ID")] + security_group_id: Option, + + #[clap(long, env = "RW_LISTEN_ADDR", default_value = "127.0.0.1:5690")] + listen_addr: String, + + /// The address for contacting this instance of the service. + /// This would be synonymous with the service's "public address" + /// or "identifying address". + /// It will serve as a unique identifier in cluster + /// membership and leader election. Must be specified for etcd backend. + #[clap(long, env = "RW_ADVERTISE_ADDR")] + advertise_addr: String, + + #[clap(long, env = "RW_DASHBOARD_HOST")] + dashboard_host: Option, + + #[clap(long, env = "RW_PROMETHEUS_HOST")] + prometheus_host: Option, + + #[clap(long, env = "RW_ETCD_ENDPOINTS", default_value_t = String::from(""))] + etcd_endpoints: String, + + /// Enable authentication with etcd. By default disabled. + #[clap(long, env = "RW_ETCD_AUTH")] + etcd_auth: bool, + + /// Username of etcd, required when --etcd-auth is enabled. + #[clap(long, env = "RW_ETCD_USERNAME", default_value = "")] + etcd_username: String, + + /// Password of etcd, required when --etcd-auth is enabled. + #[clap(long, env = "RW_ETCD_PASSWORD", default_value = "")] + etcd_password: String, + + /// Endpoint of the SQL service, make it non-option when SQL service is required. + #[clap(long, env = "RW_SQL_ENDPOINT")] + sql_endpoint: Option, + + #[clap(long, env = "RW_DASHBOARD_UI_PATH")] + dashboard_ui_path: Option, + + /// For dashboard service to fetch cluster info. + #[clap(long, env = "RW_PROMETHEUS_ENDPOINT")] + prometheus_endpoint: Option, + + /// Endpoint of the connector node, there will be a sidecar connector node + /// colocated with Meta node in the cloud environment + #[clap(long, env = "RW_CONNECTOR_RPC_ENDPOINT")] + pub connector_rpc_endpoint: Option, + + /// Default tag for the endpoint created when creating a privatelink connection. + /// Will be appended to the tags specified in the `tags` field in with clause in `create + /// connection`. + #[clap(long, env = "RW_PRIVATELINK_ENDPOINT_DEFAULT_TAGS")] + pub privatelink_endpoint_default_tags: Option, + + /// The path of `risingwave.toml` configuration file. + /// + /// If empty, default configuration values will be used. + #[clap(long, env = "RW_CONFIG_PATH", default_value = "")] + pub config_path: String, + + #[clap(long, env = "RW_BACKEND", value_enum)] + #[override_opts(path = meta.backend)] + backend: Option, + + /// The interval of periodic barrier. + #[clap(long, env = "RW_BARRIER_INTERVAL_MS")] + #[override_opts(path = system.barrier_interval_ms)] + barrier_interval_ms: Option, + + /// Target size of the Sstable. + #[clap(long, env = "RW_SSTABLE_SIZE_MB")] + #[override_opts(path = system.sstable_size_mb)] + sstable_size_mb: Option, + + /// Size of each block in bytes in SST. + #[clap(long, env = "RW_BLOCK_SIZE_KB")] + #[override_opts(path = system.block_size_kb)] + block_size_kb: Option, + + /// False positive probability of bloom filter. + #[clap(long, env = "RW_BLOOM_FALSE_POSITIVE")] + #[override_opts(path = system.bloom_false_positive)] + bloom_false_positive: Option, + + /// State store url + #[clap(long, env = "RW_STATE_STORE")] + #[override_opts(path = system.state_store)] + state_store: Option, + + /// Remote directory for storing data and metadata objects. + #[clap(long, env = "RW_DATA_DIRECTORY")] + #[override_opts(path = system.data_directory)] + data_directory: Option, + + /// Whether config object storage bucket lifecycle to purge stale data. + #[clap(long, env = "RW_DO_NOT_CONFIG_BUCKET_LIFECYCLE")] + #[override_opts(path = meta.do_not_config_object_storage_lifecycle)] + do_not_config_object_storage_lifecycle: Option, + + /// Remote storage url for storing snapshots. + #[clap(long, env = "RW_BACKUP_STORAGE_URL")] + #[override_opts(path = system.backup_storage_url)] + backup_storage_url: Option, + + /// Remote directory for storing snapshots. + #[clap(long, env = "RW_BACKUP_STORAGE_DIRECTORY")] + #[override_opts(path = system.backup_storage_directory)] + backup_storage_directory: Option, + + #[clap(long, env = "RW_OBJECT_STORE_STREAMING_READ_TIMEOUT_MS", value_enum)] + #[override_opts(path = storage.object_store_streaming_read_timeout_ms)] + pub object_store_streaming_read_timeout_ms: Option, + #[clap(long, env = "RW_OBJECT_STORE_STREAMING_UPLOAD_TIMEOUT_MS", value_enum)] + #[override_opts(path = storage.object_store_streaming_upload_timeout_ms)] + pub object_store_streaming_upload_timeout_ms: Option, + #[clap(long, env = "RW_OBJECT_STORE_UPLOAD_TIMEOUT_MS", value_enum)] + #[override_opts(path = storage.object_store_upload_timeout_ms)] + pub object_store_upload_timeout_ms: Option, + #[clap(long, env = "RW_OBJECT_STORE_READ_TIMEOUT_MS", value_enum)] + #[override_opts(path = storage.object_store_read_timeout_ms)] + pub object_store_read_timeout_ms: Option, + + /// Enable heap profile dump when memory usage is high. + #[clap(long, env = "RW_HEAP_PROFILING_DIR")] + #[override_opts(path = server.heap_profiling.dir)] + pub heap_profiling_dir: Option, +} + +use std::future::Future; +use std::pin::Pin; + +use risingwave_common::config::{load_config, MetaBackend, RwConfig}; +use tracing::info; + +/// Start meta node +pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { + // WARNING: don't change the function signature. Making it `async fn` will cause + // slow compile in release mode. + Box::pin(async move { + info!("Starting meta node"); + info!("> options: {:?}", opts); + let config = load_config(&opts.config_path, &opts); + info!("> config: {:?}", config); + info!("> version: {} ({})", RW_VERSION, GIT_SHA); + let listen_addr = opts.listen_addr.parse().unwrap(); + let dashboard_addr = opts.dashboard_host.map(|x| x.parse().unwrap()); + let prometheus_addr = opts.prometheus_host.map(|x| x.parse().unwrap()); + let backend = match config.meta.backend { + MetaBackend::Etcd => MetaStoreBackend::Etcd { + endpoints: opts + .etcd_endpoints + .split(',') + .map(|x| x.to_string()) + .collect(), + credentials: match opts.etcd_auth { + true => Some((opts.etcd_username, opts.etcd_password)), + false => None, + }, + }, + MetaBackend::Mem => MetaStoreBackend::Mem, + }; + let sql_backend = opts + .sql_endpoint + .map(|endpoint| MetaStoreSqlBackend { endpoint }); + + validate_config(&config); + + let total_memory_bytes = resource_util::memory::system_memory_available_bytes(); + let heap_profiler = + HeapProfiler::new(total_memory_bytes, config.server.heap_profiling.clone()); + // Run a background heap profiler + heap_profiler.start(); + + let max_heartbeat_interval = + Duration::from_secs(config.meta.max_heartbeat_interval_secs as u64); + let max_idle_ms = config.meta.dangerous_max_idle_secs.unwrap_or(0) * 1000; + let in_flight_barrier_nums = config.streaming.in_flight_barrier_nums; + let privatelink_endpoint_default_tags = + opts.privatelink_endpoint_default_tags.map(|tags| { + tags.split(',') + .map(|s| { + let key_val = s.split_once('=').unwrap(); + (key_val.0.to_string(), key_val.1.to_string()) + }) + .collect() + }); + + let add_info = AddressInfo { + advertise_addr: opts.advertise_addr, + listen_addr, + prometheus_addr, + dashboard_addr, + ui_path: opts.dashboard_ui_path, + }; + + let (mut join_handle, leader_lost_handle, shutdown_send) = rpc_serve( + add_info, + backend, + sql_backend, + max_heartbeat_interval, + config.meta.meta_leader_lease_secs, + MetaOpts { + enable_recovery: !config.meta.disable_recovery, + in_flight_barrier_nums, + max_idle_ms, + compaction_deterministic_test: config.meta.enable_compaction_deterministic, + default_parallelism: config.meta.default_parallelism, + vacuum_interval_sec: config.meta.vacuum_interval_sec, + vacuum_spin_interval_ms: config.meta.vacuum_spin_interval_ms, + hummock_version_checkpoint_interval_sec: config + .meta + .hummock_version_checkpoint_interval_sec, + min_delta_log_num_for_hummock_version_checkpoint: config + .meta + .min_delta_log_num_for_hummock_version_checkpoint, + min_sst_retention_time_sec: config.meta.min_sst_retention_time_sec, + full_gc_interval_sec: config.meta.full_gc_interval_sec, + collect_gc_watermark_spin_interval_sec: config + .meta + .collect_gc_watermark_spin_interval_sec, + enable_committed_sst_sanity_check: config.meta.enable_committed_sst_sanity_check, + periodic_compaction_interval_sec: config.meta.periodic_compaction_interval_sec, + node_num_monitor_interval_sec: config.meta.node_num_monitor_interval_sec, + prometheus_endpoint: opts.prometheus_endpoint, + vpc_id: opts.vpc_id, + security_group_id: opts.security_group_id, + connector_rpc_endpoint: opts.connector_rpc_endpoint, + privatelink_endpoint_default_tags, + periodic_space_reclaim_compaction_interval_sec: config + .meta + .periodic_space_reclaim_compaction_interval_sec, + telemetry_enabled: config.server.telemetry_enabled, + periodic_ttl_reclaim_compaction_interval_sec: config + .meta + .periodic_ttl_reclaim_compaction_interval_sec, + periodic_tombstone_reclaim_compaction_interval_sec: config + .meta + .periodic_tombstone_reclaim_compaction_interval_sec, + periodic_split_compact_group_interval_sec: config + .meta + .periodic_split_compact_group_interval_sec, + split_group_size_limit: config.meta.split_group_size_limit, + min_table_split_size: config.meta.move_table_size_limit, + table_write_throughput_threshold: config.meta.table_write_throughput_threshold, + min_table_split_write_throughput: config.meta.min_table_split_write_throughput, + partition_vnode_count: config.meta.partition_vnode_count, + do_not_config_object_storage_lifecycle: config + .meta + .do_not_config_object_storage_lifecycle, + compaction_task_max_heartbeat_interval_secs: config + .meta + .compaction_task_max_heartbeat_interval_secs, + compaction_config: Some(config.meta.compaction_config), + }, + config.system.into_init_system_params(), + ) + .await + .unwrap(); + + tracing::info!("Meta server listening at {}", listen_addr); + + match leader_lost_handle { + None => { + tokio::select! { + _ = tokio::signal::ctrl_c() => { + tracing::info!("receive ctrl+c"); + shutdown_send.send(()).unwrap(); + join_handle.await.unwrap() + } + res = &mut join_handle => res.unwrap(), + }; + } + Some(mut handle) => { + tokio::select! { + _ = &mut handle => { + tracing::info!("receive leader lost signal"); + // When we lose leadership, we will exit as soon as possible. + } + _ = tokio::signal::ctrl_c() => { + tracing::info!("receive ctrl+c"); + shutdown_send.send(()).unwrap(); + join_handle.await.unwrap(); + handle.abort(); + } + res = &mut join_handle => { + res.unwrap(); + handle.abort(); + }, + }; + } + }; + }) +} + +fn validate_config(config: &RwConfig) { + if config.meta.meta_leader_lease_secs <= 2 { + let error_msg = "meta leader lease secs should be larger than 2"; + tracing::error!(error_msg); + panic!("{}", error_msg); + } +} diff --git a/src/meta/src/rpc/server.rs b/src/meta/node/src/server.rs similarity index 92% rename from src/meta/src/rpc/server.rs rename to src/meta/node/src/server.rs index 1a15defe43bf5..d5cbfa3e3b26a 100644 --- a/src/meta/src/rpc/server.rs +++ b/src/meta/node/src/server.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::net::SocketAddr; use std::sync::Arc; use std::time::Duration; @@ -27,6 +26,24 @@ use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common_service::metrics_manager::MetricsManager; use risingwave_common_service::tracing::TracingExtractLayer; +use risingwave_meta::rpc::intercept::MetricsMiddlewareLayer; +use risingwave_meta::rpc::ElectionClientRef; +use risingwave_meta_service::backup_service::BackupServiceImpl; +use risingwave_meta_service::cloud_service::CloudServiceImpl; +use risingwave_meta_service::cluster_service::ClusterServiceImpl; +use risingwave_meta_service::ddl_service::DdlServiceImpl; +use risingwave_meta_service::health_service::HealthServiceImpl; +use risingwave_meta_service::heartbeat_service::HeartbeatServiceImpl; +use risingwave_meta_service::hummock_service::HummockServiceImpl; +use risingwave_meta_service::meta_member_service::MetaMemberServiceImpl; +use risingwave_meta_service::notification_service::NotificationServiceImpl; +use risingwave_meta_service::scale_service::ScaleServiceImpl; +use risingwave_meta_service::serving_service::ServingServiceImpl; +use risingwave_meta_service::sink_coordination_service::SinkCoordinationServiceImpl; +use risingwave_meta_service::stream_service::StreamServiceImpl; +use risingwave_meta_service::system_params_service::SystemParamsServiceImpl; +use risingwave_meta_service::telemetry_service::TelemetryInfoServiceImpl; +use risingwave_meta_service::user_service::UserServiceImpl; use risingwave_pb::backup_service::backup_service_server::BackupServiceServer; use risingwave_pb::cloud_service::cloud_service_server::CloudServiceServer; use risingwave_pb::connector_service::sink_coordination_service_server::SinkCoordinationServiceServer; @@ -50,12 +67,6 @@ use tokio::sync::watch; use tokio::sync::watch::{Receiver as WatchReceiver, Sender as WatchSender}; use tokio::task::JoinHandle; -use super::intercept::MetricsMiddlewareLayer; -use super::service::health_service::HealthServiceImpl; -use super::service::notification_service::NotificationServiceImpl; -use super::service::scale_service::ScaleServiceImpl; -use super::service::serving_service::ServingServiceImpl; -use super::DdlServiceImpl; use crate::backup_restore::BackupManager; use crate::barrier::{BarrierScheduler, GlobalBarrierManager}; use crate::controller::system_param::SystemParamsController; @@ -68,21 +79,9 @@ use crate::manager::{ }; use crate::rpc::cloud_provider::AwsEc2Client; use crate::rpc::election::etcd::EtcdElectionClient; -use crate::rpc::election::ElectionClient; use crate::rpc::metrics::{ start_fragment_info_monitor, start_worker_info_monitor, GLOBAL_META_METRICS, }; -use crate::rpc::service::backup_service::BackupServiceImpl; -use crate::rpc::service::cloud_service::CloudServiceImpl; -use crate::rpc::service::cluster_service::ClusterServiceImpl; -use crate::rpc::service::heartbeat_service::HeartbeatServiceImpl; -use crate::rpc::service::hummock_service::HummockServiceImpl; -use crate::rpc::service::meta_member_service::MetaMemberServiceImpl; -use crate::rpc::service::sink_coordination_service::SinkCoordinationServiceImpl; -use crate::rpc::service::stream_service::StreamServiceImpl; -use crate::rpc::service::system_params_service::SystemParamsServiceImpl; -use crate::rpc::service::telemetry_service::TelemetryInfoServiceImpl; -use crate::rpc::service::user_service::UserServiceImpl; use crate::serving::ServingVnodeMapping; use crate::storage::{ EtcdMetaStore, MemStore, MetaStore, MetaStoreBoxExt, MetaStoreRef, @@ -91,43 +90,13 @@ use crate::storage::{ use crate::stream::{GlobalStreamManager, SourceManager}; use crate::telemetry::{MetaReportCreator, MetaTelemetryInfoFetcher}; use crate::{hummock, serving, MetaError, MetaResult}; - -#[derive(Debug)] -pub enum MetaStoreBackend { - Etcd { - endpoints: Vec, - credentials: Option<(String, String)>, - }, - Mem, -} - #[derive(Debug)] pub struct MetaStoreSqlBackend { pub(crate) endpoint: String, } -#[derive(Clone)] -pub struct AddressInfo { - pub advertise_addr: String, - pub listen_addr: SocketAddr, - pub prometheus_addr: Option, - pub dashboard_addr: Option, - pub ui_path: Option, -} - -impl Default for AddressInfo { - fn default() -> Self { - Self { - advertise_addr: "".to_string(), - listen_addr: SocketAddr::V4("127.0.0.1:0000".parse().unwrap()), - prometheus_addr: None, - dashboard_addr: None, - ui_path: None, - } - } -} - -pub type ElectionClientRef = Arc; +use risingwave_meta::MetaStoreBackend; +use risingwave_meta_service::AddressInfo; pub async fn rpc_serve( address_info: AddressInfo, @@ -636,25 +605,19 @@ pub async fn start_service_as_election_leader( // compaction_scheduler, &env.opts, )); - sub_tasks.push( - start_worker_info_monitor( - cluster_manager.clone(), - election_client.clone(), - Duration::from_secs(env.opts.node_num_monitor_interval_sec), - meta_metrics.clone(), - ) - .await, - ); - sub_tasks.push( - start_fragment_info_monitor( - cluster_manager.clone(), - catalog_manager, - fragment_manager.clone(), - hummock_manager.clone(), - meta_metrics.clone(), - ) - .await, - ); + sub_tasks.push(start_worker_info_monitor( + cluster_manager.clone(), + election_client.clone(), + Duration::from_secs(env.opts.node_num_monitor_interval_sec), + meta_metrics.clone(), + )); + sub_tasks.push(start_fragment_info_monitor( + cluster_manager.clone(), + catalog_manager, + fragment_manager.clone(), + hummock_manager.clone(), + meta_metrics.clone(), + )); if let Some(system_params_ctl) = system_params_controller { sub_tasks.push(SystemParamsController::start_params_notifier( system_params_ctl, diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml new file mode 100644 index 0000000000000..1760ccd56a85a --- /dev/null +++ b/src/meta/service/Cargo.toml @@ -0,0 +1,53 @@ +[package] +name = "risingwave_meta_service" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +keywords = { workspace = true } +license = { workspace = true } +repository = { workspace = true } + +[package.metadata.cargo-machete] +ignored = ["workspace-hack"] + +[package.metadata.cargo-udeps.ignore] +normal = ["workspace-hack"] + +[dependencies] +anyhow = "1" +async-trait = "0.1" +either = "1" +futures = { version = "0.3", default-features = false, features = ["alloc"] } +itertools = "0.11" +regex = "1" +risingwave_common = { workspace = true } +risingwave_connector = { workspace = true } +risingwave_meta = { workspace = true } +risingwave_pb = { workspace = true } +sea-orm = { version = "0.12.0", features = [ + "sqlx-mysql", + "sqlx-postgres", + "sqlx-sqlite", + "runtime-tokio-native-tls", + "macros", +] } +sync-point = { path = "../../utils/sync-point" } +tokio = { version = "0.2", package = "madsim-tokio", features = [ + "rt", + "rt-multi-thread", + "sync", + "macros", + "time", + "signal", +] } +tokio-stream = { version = "0.1", features = ["net"] } +tonic = { workspace = true } +tracing = "0.1" + +[target.'cfg(not(madsim))'.dependencies] +workspace-hack = { path = "../../workspace-hack" } + +[dev-dependencies] + +[lints] +workspace = true diff --git a/src/meta/src/rpc/service/backup_service.rs b/src/meta/service/src/backup_service.rs similarity index 100% rename from src/meta/src/rpc/service/backup_service.rs rename to src/meta/service/src/backup_service.rs diff --git a/src/meta/src/rpc/service/cloud_service.rs b/src/meta/service/src/cloud_service.rs similarity index 100% rename from src/meta/src/rpc/service/cloud_service.rs rename to src/meta/service/src/cloud_service.rs diff --git a/src/meta/src/rpc/service/cluster_service.rs b/src/meta/service/src/cluster_service.rs similarity index 100% rename from src/meta/src/rpc/service/cluster_service.rs rename to src/meta/service/src/cluster_service.rs diff --git a/src/meta/src/rpc/service/ddl_service.rs b/src/meta/service/src/ddl_service.rs similarity index 100% rename from src/meta/src/rpc/service/ddl_service.rs rename to src/meta/service/src/ddl_service.rs diff --git a/src/meta/src/rpc/service/health_service.rs b/src/meta/service/src/health_service.rs similarity index 93% rename from src/meta/src/rpc/service/health_service.rs rename to src/meta/service/src/health_service.rs index bdb01c1ef0760..338091a72de38 100644 --- a/src/meta/src/rpc/service/health_service.rs +++ b/src/meta/service/src/health_service.rs @@ -19,6 +19,12 @@ use tonic::{Request, Response, Status}; pub struct HealthServiceImpl {} +impl Default for HealthServiceImpl { + fn default() -> Self { + Self::new() + } +} + impl HealthServiceImpl { pub fn new() -> Self { Self {} diff --git a/src/meta/src/rpc/service/heartbeat_service.rs b/src/meta/service/src/heartbeat_service.rs similarity index 100% rename from src/meta/src/rpc/service/heartbeat_service.rs rename to src/meta/service/src/heartbeat_service.rs diff --git a/src/meta/src/rpc/service/hummock_service.rs b/src/meta/service/src/hummock_service.rs similarity index 99% rename from src/meta/src/rpc/service/hummock_service.rs rename to src/meta/service/src/hummock_service.rs index 3ae90421d2d87..74dc37b82d21e 100644 --- a/src/meta/src/rpc/service/hummock_service.rs +++ b/src/meta/service/src/hummock_service.rs @@ -27,7 +27,7 @@ use tonic::{Request, Response, Status, Streaming}; use crate::hummock::compaction::selector::ManualCompactionOption; use crate::hummock::{HummockManagerRef, VacuumManagerRef}; use crate::manager::FragmentManagerRef; -use crate::rpc::service::RwReceiverStream; +use crate::RwReceiverStream; pub struct HummockServiceImpl { hummock_manager: HummockManagerRef, vacuum_manager: VacuumManagerRef, diff --git a/src/meta/src/rpc/service/mod.rs b/src/meta/service/src/lib.rs similarity index 71% rename from src/meta/src/rpc/service/mod.rs rename to src/meta/service/src/lib.rs index 4484a8ca68a88..0d473a6ed031f 100644 --- a/src/meta/src/rpc/service/mod.rs +++ b/src/meta/service/src/lib.rs @@ -12,6 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![feature(lint_reasons)] +#![feature(let_chains)] +#![feature(lazy_cell)] +#![feature(impl_trait_in_assoc_type)] +#![cfg_attr(coverage, feature(no_coverage))] + +use risingwave_meta::*; + pub mod backup_service; pub mod cloud_service; pub mod cluster_service; @@ -59,3 +67,25 @@ impl Stream for RwReceiverStream { .map(|opt| opt.map(|res| res.map_err(Into::into))) } } + +use std::net::SocketAddr; + +#[derive(Clone)] +pub struct AddressInfo { + pub advertise_addr: String, + pub listen_addr: SocketAddr, + pub prometheus_addr: Option, + pub dashboard_addr: Option, + pub ui_path: Option, +} +impl Default for AddressInfo { + fn default() -> Self { + Self { + advertise_addr: "".to_string(), + listen_addr: SocketAddr::V4("127.0.0.1:0000".parse().unwrap()), + prometheus_addr: None, + dashboard_addr: None, + ui_path: None, + } + } +} diff --git a/src/meta/src/rpc/service/meta_member_service.rs b/src/meta/service/src/meta_member_service.rs similarity index 97% rename from src/meta/src/rpc/service/meta_member_service.rs rename to src/meta/service/src/meta_member_service.rs index 6fb138b535410..25c4c7ad4cc84 100644 --- a/src/meta/src/rpc/service/meta_member_service.rs +++ b/src/meta/service/src/meta_member_service.rs @@ -14,13 +14,13 @@ use either::Either; use risingwave_common::util::addr::HostAddr; +use risingwave_meta::rpc::ElectionClientRef; use risingwave_pb::common::HostAddress; use risingwave_pb::meta::meta_member_service_server::MetaMemberService; use risingwave_pb::meta::{MembersRequest, MembersResponse, MetaMember}; use tonic::{Request, Response, Status}; -use crate::rpc::server::{AddressInfo, ElectionClientRef}; - +use crate::AddressInfo; #[derive(Clone)] pub struct MetaMemberServiceImpl { election_client_or_self: Either, diff --git a/src/meta/src/rpc/service/notification_service.rs b/src/meta/service/src/notification_service.rs similarity index 100% rename from src/meta/src/rpc/service/notification_service.rs rename to src/meta/service/src/notification_service.rs diff --git a/src/meta/src/rpc/service/scale_service.rs b/src/meta/service/src/scale_service.rs similarity index 100% rename from src/meta/src/rpc/service/scale_service.rs rename to src/meta/service/src/scale_service.rs diff --git a/src/meta/src/rpc/service/serving_service.rs b/src/meta/service/src/serving_service.rs similarity index 100% rename from src/meta/src/rpc/service/serving_service.rs rename to src/meta/service/src/serving_service.rs diff --git a/src/meta/src/rpc/service/sink_coordination_service.rs b/src/meta/service/src/sink_coordination_service.rs similarity index 92% rename from src/meta/src/rpc/service/sink_coordination_service.rs rename to src/meta/service/src/sink_coordination_service.rs index f7d56af9c063f..72c4cb2ff9af4 100644 --- a/src/meta/src/rpc/service/sink_coordination_service.rs +++ b/src/meta/service/src/sink_coordination_service.rs @@ -20,12 +20,12 @@ use tonic::{Request, Response, Status, Streaming}; use crate::manager::sink_coordination::SinkCoordinatorManager; #[derive(Clone)] -pub(crate) struct SinkCoordinationServiceImpl { +pub struct SinkCoordinationServiceImpl { sink_manager: SinkCoordinatorManager, } impl SinkCoordinationServiceImpl { - pub(crate) fn new(sink_manager: SinkCoordinatorManager) -> Self { + pub fn new(sink_manager: SinkCoordinatorManager) -> Self { Self { sink_manager } } } diff --git a/src/meta/src/rpc/service/stream_service.rs b/src/meta/service/src/stream_service.rs similarity index 100% rename from src/meta/src/rpc/service/stream_service.rs rename to src/meta/service/src/stream_service.rs diff --git a/src/meta/src/rpc/service/system_params_service.rs b/src/meta/service/src/system_params_service.rs similarity index 100% rename from src/meta/src/rpc/service/system_params_service.rs rename to src/meta/service/src/system_params_service.rs diff --git a/src/meta/src/rpc/service/telemetry_service.rs b/src/meta/service/src/telemetry_service.rs similarity index 100% rename from src/meta/src/rpc/service/telemetry_service.rs rename to src/meta/service/src/telemetry_service.rs diff --git a/src/meta/src/rpc/service/user_service.rs b/src/meta/service/src/user_service.rs similarity index 100% rename from src/meta/src/rpc/service/user_service.rs rename to src/meta/service/src/user_service.rs diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 262da65f19b0c..8924992c2e18e 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -139,7 +139,7 @@ pub struct GlobalBarrierManager { cluster_manager: ClusterManagerRef, - pub(crate) catalog_manager: CatalogManagerRef, + pub catalog_manager: CatalogManagerRef, fragment_manager: FragmentManagerRef, @@ -151,7 +151,7 @@ pub struct GlobalBarrierManager { metrics: Arc, - pub(crate) env: MetaSrvEnv, + pub env: MetaSrvEnv, tracker: Mutex, } diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index bce901cd6f459..b3b57f2be58f1 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -107,7 +107,7 @@ impl GlobalBarrierManager { /// the cluster or `risectl` command. Used for debugging purpose. /// /// Returns the new state of the barrier manager after recovery. - pub(crate) async fn recovery( + pub async fn recovery( &self, prev_epoch: TracedEpoch, paused_reason: Option, diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 7c9fefd15606b..c4718d97d40f6 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -393,7 +393,7 @@ impl ScheduledBarriers { } /// Make the `checkpoint` of the next barrier must be true - pub(crate) fn force_checkpoint_in_next_barrier(&self) { + pub fn force_checkpoint_in_next_barrier(&self) { self.inner.force_checkpoint.store(true, Ordering::Relaxed) } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 74f01497cc048..3fcb6a7d004ae 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -37,7 +37,7 @@ impl From for MetaError { #[derive(Clone)] pub struct SqlMetaStore { - pub(crate) conn: DatabaseConnection, + pub conn: DatabaseConnection, } impl SqlMetaStore { diff --git a/src/meta/src/hummock/compaction/mod.rs b/src/meta/src/hummock/compaction/mod.rs index 246b00af771af..a056414034243 100644 --- a/src/meta/src/hummock/compaction/mod.rs +++ b/src/meta/src/hummock/compaction/mod.rs @@ -43,8 +43,8 @@ use crate::hummock::level_handler::LevelHandler; use crate::hummock::model::CompactionGroup; pub struct CompactStatus { - pub(crate) compaction_group_id: CompactionGroupId, - pub(crate) level_handlers: Vec, + pub compaction_group_id: CompactionGroupId, + pub level_handlers: Vec, } impl Debug for CompactStatus { diff --git a/src/meta/src/hummock/compaction/selector/mod.rs b/src/meta/src/hummock/compaction/selector/mod.rs index 1bc49afc55653..48d237eb5cf2f 100644 --- a/src/meta/src/hummock/compaction/selector/mod.rs +++ b/src/meta/src/hummock/compaction/selector/mod.rs @@ -335,10 +335,7 @@ pub mod tests { l0 } - pub(crate) fn assert_compaction_task( - compact_task: &CompactionTask, - level_handlers: &[LevelHandler], - ) { + pub fn assert_compaction_task(compact_task: &CompactionTask, level_handlers: &[LevelHandler]) { for i in &compact_task.input.input_levels { for t in &i.table_infos { assert!(level_handlers[i.level_idx as usize].is_pending_compact(&t.sst_id)); diff --git a/src/meta/src/hummock/manager/checkpoint.rs b/src/meta/src/hummock/manager/checkpoint.rs index 4e6bb094d5a59..6aa64292b9db1 100644 --- a/src/meta/src/hummock/manager/checkpoint.rs +++ b/src/meta/src/hummock/manager/checkpoint.rs @@ -36,7 +36,7 @@ const HUMMOCK_INIT_FLAG_KEY: &[u8] = b"hummock_init_flag"; impl HummockManager { /// # Panics /// if checkpoint is not found. - pub(crate) async fn read_checkpoint(&self) -> Result { + pub async fn read_checkpoint(&self) -> Result { use prost::Message; let data = match self .object_store @@ -173,23 +173,23 @@ impl HummockManager { .map_err(Into::into) } - pub(crate) fn pause_version_checkpoint(&self) { + pub fn pause_version_checkpoint(&self) { self.pause_version_checkpoint.store(true, Ordering::Relaxed); tracing::info!("hummock version checkpoint is paused."); } - pub(crate) fn resume_version_checkpoint(&self) { + pub fn resume_version_checkpoint(&self) { self.pause_version_checkpoint .store(false, Ordering::Relaxed); tracing::info!("hummock version checkpoint is resumed."); } - pub(crate) fn is_version_checkpoint_paused(&self) -> bool { + pub fn is_version_checkpoint_paused(&self) -> bool { self.pause_version_checkpoint.load(Ordering::Relaxed) } #[named] - pub(crate) async fn get_checkpoint_version(&self) -> HummockVersion { + pub async fn get_checkpoint_version(&self) -> HummockVersion { let versioning_guard = read_lock!(self, versioning).await; versioning_guard .checkpoint diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index 21751bb968421..b069a31ce5bd3 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -112,7 +112,7 @@ impl HummockManager { Ok(invalid_context_ids) } - pub(crate) async fn commit_epoch_sanity_check( + pub async fn commit_epoch_sanity_check( &self, epoch: HummockEpoch, sstables: &Vec, diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 3a99dd898d1c9..d07337ca47e2b 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -253,7 +253,7 @@ pub enum CompactionResumeTrigger { } impl HummockManager { - pub(crate) async fn new( + pub async fn new( env: MetaSrvEnv, cluster_manager: ClusterManagerRef, fragment_manager: FragmentManagerRef, diff --git a/src/meta/src/hummock/manager/worker.rs b/src/meta/src/hummock/manager/worker.rs index 9f9b0fd911bd4..bc2103635b59f 100644 --- a/src/meta/src/hummock/manager/worker.rs +++ b/src/meta/src/hummock/manager/worker.rs @@ -34,7 +34,7 @@ pub enum HummockManagerEvent { } impl HummockManager { - pub(crate) async fn start_worker( + pub async fn start_worker( self: &HummockManagerRef, mut receiver: HummockManagerEventReceiver, ) -> JoinHandle<()> { diff --git a/src/meta/src/hummock/model/compaction_group_config.rs b/src/meta/src/hummock/model/compaction_group_config.rs index 8331abac62017..fa1bd1f88b3bd 100644 --- a/src/meta/src/hummock/model/compaction_group_config.rs +++ b/src/meta/src/hummock/model/compaction_group_config.rs @@ -23,8 +23,8 @@ use crate::model::{MetadataModel, MetadataModelResult}; #[derive(Debug, Clone, PartialEq)] pub struct CompactionGroup { - pub(crate) group_id: CompactionGroupId, - pub(crate) compaction_config: Arc, + pub group_id: CompactionGroupId, + pub compaction_config: Arc, } impl CompactionGroup { diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index a5c52d5e3e0f3..afe66d27ad8e8 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -14,12 +14,10 @@ #![allow(clippy::derive_partial_eq_without_eq)] #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(type_alias_impl_trait)] -#![feature(extract_if)] -#![feature(custom_test_frameworks)] #![feature(lint_reasons)] #![feature(map_try_insert)] +#![feature(extract_if)] #![feature(hash_extract_if)] #![feature(btree_extract_if)] #![feature(result_option_inspect)] @@ -29,6 +27,7 @@ #![feature(assert_matches)] #![feature(try_blocks)] #![cfg_attr(coverage, feature(no_coverage))] +#![feature(custom_test_frameworks)] #![test_runner(risingwave_test_runner::test_runner::run_failpont_tests)] #![feature(is_sorted)] #![feature(impl_trait_in_assoc_type)] @@ -36,341 +35,31 @@ #![feature(async_fn_in_trait)] pub mod backup_restore; -mod barrier; +pub mod barrier; pub mod controller; #[cfg(not(madsim))] // no need in simulation test -mod dashboard; -mod error; +pub mod dashboard; +pub mod error; pub mod hummock; pub mod manager; pub mod model; pub mod model_v2; -mod rpc; -pub(crate) mod serving; +pub mod rpc; +pub mod serving; pub mod storage; -mod stream; -pub(crate) mod telemetry; - -use std::time::Duration; +pub mod stream; +pub mod telemetry; -use clap::Parser; pub use error::{MetaError, MetaResult}; -use risingwave_common::config::OverrideConfig; -use risingwave_common::util::resource_util; -use risingwave_common::{GIT_SHA, RW_VERSION}; -use risingwave_common_heap_profiling::HeapProfiler; pub use rpc::{ElectionClient, ElectionMember, EtcdElectionClient}; use crate::manager::MetaOpts; -use crate::rpc::server::{rpc_serve, AddressInfo, MetaStoreBackend, MetaStoreSqlBackend}; - -#[derive(Debug, Clone, Parser, OverrideConfig)] -#[command(version, about = "The central metadata management service")] -pub struct MetaNodeOpts { - #[clap(long, env = "RW_VPC_ID")] - vpc_id: Option, - - #[clap(long, env = "RW_VPC_SECURITY_GROUP_ID")] - security_group_id: Option, - - #[clap(long, env = "RW_LISTEN_ADDR", default_value = "127.0.0.1:5690")] - listen_addr: String, - - /// The address for contacting this instance of the service. - /// This would be synonymous with the service's "public address" - /// or "identifying address". - /// It will serve as a unique identifier in cluster - /// membership and leader election. Must be specified for etcd backend. - #[clap(long, env = "RW_ADVERTISE_ADDR")] - advertise_addr: String, - - #[clap(long, env = "RW_DASHBOARD_HOST")] - dashboard_host: Option, - - #[clap(long, env = "RW_PROMETHEUS_HOST")] - prometheus_host: Option, - - #[clap(long, env = "RW_ETCD_ENDPOINTS", default_value_t = String::from(""))] - etcd_endpoints: String, - - /// Enable authentication with etcd. By default disabled. - #[clap(long, env = "RW_ETCD_AUTH")] - etcd_auth: bool, - - /// Username of etcd, required when --etcd-auth is enabled. - #[clap(long, env = "RW_ETCD_USERNAME", default_value = "")] - etcd_username: String, - - /// Password of etcd, required when --etcd-auth is enabled. - #[clap(long, env = "RW_ETCD_PASSWORD", default_value = "")] - etcd_password: String, - - /// Endpoint of the SQL service, make it non-option when SQL service is required. - #[clap(long, env = "RW_SQL_ENDPOINT")] - sql_endpoint: Option, - - #[clap(long, env = "RW_DASHBOARD_UI_PATH")] - dashboard_ui_path: Option, - - /// For dashboard service to fetch cluster info. - #[clap(long, env = "RW_PROMETHEUS_ENDPOINT")] - prometheus_endpoint: Option, - - /// Endpoint of the connector node, there will be a sidecar connector node - /// colocated with Meta node in the cloud environment - #[clap(long, env = "RW_CONNECTOR_RPC_ENDPOINT")] - pub connector_rpc_endpoint: Option, - - /// Default tag for the endpoint created when creating a privatelink connection. - /// Will be appended to the tags specified in the `tags` field in with clause in `create - /// connection`. - #[clap(long, env = "RW_PRIVATELINK_ENDPOINT_DEFAULT_TAGS")] - pub privatelink_endpoint_default_tags: Option, - - /// The path of `risingwave.toml` configuration file. - /// - /// If empty, default configuration values will be used. - #[clap(long, env = "RW_CONFIG_PATH", default_value = "")] - pub config_path: String, - - #[clap(long, env = "RW_BACKEND", value_enum)] - #[override_opts(path = meta.backend)] - backend: Option, - - /// The interval of periodic barrier. - #[clap(long, env = "RW_BARRIER_INTERVAL_MS")] - #[override_opts(path = system.barrier_interval_ms)] - barrier_interval_ms: Option, - - /// Target size of the Sstable. - #[clap(long, env = "RW_SSTABLE_SIZE_MB")] - #[override_opts(path = system.sstable_size_mb)] - sstable_size_mb: Option, - - /// Size of each block in bytes in SST. - #[clap(long, env = "RW_BLOCK_SIZE_KB")] - #[override_opts(path = system.block_size_kb)] - block_size_kb: Option, - - /// False positive probability of bloom filter. - #[clap(long, env = "RW_BLOOM_FALSE_POSITIVE")] - #[override_opts(path = system.bloom_false_positive)] - bloom_false_positive: Option, - - /// State store url - #[clap(long, env = "RW_STATE_STORE")] - #[override_opts(path = system.state_store)] - state_store: Option, - - /// Remote directory for storing data and metadata objects. - #[clap(long, env = "RW_DATA_DIRECTORY")] - #[override_opts(path = system.data_directory)] - data_directory: Option, - - /// Whether config object storage bucket lifecycle to purge stale data. - #[clap(long, env = "RW_DO_NOT_CONFIG_BUCKET_LIFECYCLE")] - #[override_opts(path = meta.do_not_config_object_storage_lifecycle)] - do_not_config_object_storage_lifecycle: Option, - - /// Remote storage url for storing snapshots. - #[clap(long, env = "RW_BACKUP_STORAGE_URL")] - #[override_opts(path = system.backup_storage_url)] - backup_storage_url: Option, - - /// Remote directory for storing snapshots. - #[clap(long, env = "RW_BACKUP_STORAGE_DIRECTORY")] - #[override_opts(path = system.backup_storage_directory)] - backup_storage_directory: Option, - - #[clap(long, env = "RW_OBJECT_STORE_STREAMING_READ_TIMEOUT_MS", value_enum)] - #[override_opts(path = storage.object_store_streaming_read_timeout_ms)] - pub object_store_streaming_read_timeout_ms: Option, - #[clap(long, env = "RW_OBJECT_STORE_STREAMING_UPLOAD_TIMEOUT_MS", value_enum)] - #[override_opts(path = storage.object_store_streaming_upload_timeout_ms)] - pub object_store_streaming_upload_timeout_ms: Option, - #[clap(long, env = "RW_OBJECT_STORE_UPLOAD_TIMEOUT_MS", value_enum)] - #[override_opts(path = storage.object_store_upload_timeout_ms)] - pub object_store_upload_timeout_ms: Option, - #[clap(long, env = "RW_OBJECT_STORE_READ_TIMEOUT_MS", value_enum)] - #[override_opts(path = storage.object_store_read_timeout_ms)] - pub object_store_read_timeout_ms: Option, - - /// Enable heap profile dump when memory usage is high. - #[clap(long, env = "RW_HEAP_PROFILING_DIR")] - #[override_opts(path = server.heap_profiling.dir)] - pub heap_profiling_dir: Option, -} - -use std::future::Future; -use std::pin::Pin; - -use risingwave_common::config::{load_config, MetaBackend, RwConfig}; -use tracing::info; - -/// Start meta node -pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { - // WARNING: don't change the function signature. Making it `async fn` will cause - // slow compile in release mode. - Box::pin(async move { - info!("Starting meta node"); - info!("> options: {:?}", opts); - let config = load_config(&opts.config_path, &opts); - info!("> config: {:?}", config); - info!("> version: {} ({})", RW_VERSION, GIT_SHA); - let listen_addr = opts.listen_addr.parse().unwrap(); - let dashboard_addr = opts.dashboard_host.map(|x| x.parse().unwrap()); - let prometheus_addr = opts.prometheus_host.map(|x| x.parse().unwrap()); - let backend = match config.meta.backend { - MetaBackend::Etcd => MetaStoreBackend::Etcd { - endpoints: opts - .etcd_endpoints - .split(',') - .map(|x| x.to_string()) - .collect(), - credentials: match opts.etcd_auth { - true => Some((opts.etcd_username, opts.etcd_password)), - false => None, - }, - }, - MetaBackend::Mem => MetaStoreBackend::Mem, - }; - let sql_backend = opts - .sql_endpoint - .map(|endpoint| MetaStoreSqlBackend { endpoint }); - - validate_config(&config); - - let total_memory_bytes = resource_util::memory::system_memory_available_bytes(); - let heap_profiler = - HeapProfiler::new(total_memory_bytes, config.server.heap_profiling.clone()); - // Run a background heap profiler - heap_profiler.start(); - - let max_heartbeat_interval = - Duration::from_secs(config.meta.max_heartbeat_interval_secs as u64); - let max_idle_ms = config.meta.dangerous_max_idle_secs.unwrap_or(0) * 1000; - let in_flight_barrier_nums = config.streaming.in_flight_barrier_nums; - let privatelink_endpoint_default_tags = - opts.privatelink_endpoint_default_tags.map(|tags| { - tags.split(',') - .map(|s| { - let key_val = s.split_once('=').unwrap(); - (key_val.0.to_string(), key_val.1.to_string()) - }) - .collect() - }); - - let add_info = AddressInfo { - advertise_addr: opts.advertise_addr, - listen_addr, - prometheus_addr, - dashboard_addr, - ui_path: opts.dashboard_ui_path, - }; - - let (mut join_handle, leader_lost_handle, shutdown_send) = rpc_serve( - add_info, - backend, - sql_backend, - max_heartbeat_interval, - config.meta.meta_leader_lease_secs, - MetaOpts { - enable_recovery: !config.meta.disable_recovery, - in_flight_barrier_nums, - max_idle_ms, - compaction_deterministic_test: config.meta.enable_compaction_deterministic, - default_parallelism: config.meta.default_parallelism, - vacuum_interval_sec: config.meta.vacuum_interval_sec, - vacuum_spin_interval_ms: config.meta.vacuum_spin_interval_ms, - hummock_version_checkpoint_interval_sec: config - .meta - .hummock_version_checkpoint_interval_sec, - min_delta_log_num_for_hummock_version_checkpoint: config - .meta - .min_delta_log_num_for_hummock_version_checkpoint, - min_sst_retention_time_sec: config.meta.min_sst_retention_time_sec, - full_gc_interval_sec: config.meta.full_gc_interval_sec, - collect_gc_watermark_spin_interval_sec: config - .meta - .collect_gc_watermark_spin_interval_sec, - enable_committed_sst_sanity_check: config.meta.enable_committed_sst_sanity_check, - periodic_compaction_interval_sec: config.meta.periodic_compaction_interval_sec, - node_num_monitor_interval_sec: config.meta.node_num_monitor_interval_sec, - prometheus_endpoint: opts.prometheus_endpoint, - vpc_id: opts.vpc_id, - security_group_id: opts.security_group_id, - connector_rpc_endpoint: opts.connector_rpc_endpoint, - privatelink_endpoint_default_tags, - periodic_space_reclaim_compaction_interval_sec: config - .meta - .periodic_space_reclaim_compaction_interval_sec, - telemetry_enabled: config.server.telemetry_enabled, - periodic_ttl_reclaim_compaction_interval_sec: config - .meta - .periodic_ttl_reclaim_compaction_interval_sec, - periodic_tombstone_reclaim_compaction_interval_sec: config - .meta - .periodic_tombstone_reclaim_compaction_interval_sec, - periodic_split_compact_group_interval_sec: config - .meta - .periodic_split_compact_group_interval_sec, - split_group_size_limit: config.meta.split_group_size_limit, - min_table_split_size: config.meta.move_table_size_limit, - table_write_throughput_threshold: config.meta.table_write_throughput_threshold, - min_table_split_write_throughput: config.meta.min_table_split_write_throughput, - partition_vnode_count: config.meta.partition_vnode_count, - do_not_config_object_storage_lifecycle: config - .meta - .do_not_config_object_storage_lifecycle, - compaction_task_max_heartbeat_interval_secs: config - .meta - .compaction_task_max_heartbeat_interval_secs, - compaction_config: Some(config.meta.compaction_config), - }, - config.system.into_init_system_params(), - ) - .await - .unwrap(); - - tracing::info!("Meta server listening at {}", listen_addr); - - match leader_lost_handle { - None => { - tokio::select! { - _ = tokio::signal::ctrl_c() => { - tracing::info!("receive ctrl+c"); - shutdown_send.send(()).unwrap(); - join_handle.await.unwrap() - } - res = &mut join_handle => res.unwrap(), - }; - } - Some(mut handle) => { - tokio::select! { - _ = &mut handle => { - tracing::info!("receive leader lost signal"); - // When we lose leadership, we will exit as soon as possible. - } - _ = tokio::signal::ctrl_c() => { - tracing::info!("receive ctrl+c"); - shutdown_send.send(()).unwrap(); - join_handle.await.unwrap(); - handle.abort(); - } - res = &mut join_handle => { - res.unwrap(); - handle.abort(); - }, - }; - } - }; - }) -} -fn validate_config(config: &RwConfig) { - if config.meta.meta_leader_lease_secs <= 2 { - let error_msg = "meta leader lease secs should be larger than 2"; - tracing::error!(error_msg); - panic!("{}", error_msg); - } +#[derive(Debug)] +pub enum MetaStoreBackend { + Etcd { + endpoints: Vec, + credentials: Option<(String, String)>, + }, + Mem, } diff --git a/src/meta/src/manager/mod.rs b/src/meta/src/manager/mod.rs index 760c24e0df7bc..35642ed0ec143 100644 --- a/src/meta/src/manager/mod.rs +++ b/src/meta/src/manager/mod.rs @@ -18,20 +18,17 @@ mod env; mod id; mod idle; mod notification; -pub(crate) mod sink_coordination; +pub mod sink_coordination; mod streaming_job; mod system_param; -pub(crate) use catalog::*; -pub use cluster::WorkerKey; -pub(crate) use cluster::*; -pub use env::MetaSrvEnv; -pub(crate) use env::*; -pub(crate) use id::*; -pub(crate) use idle::*; -pub(crate) use notification::*; -pub use notification::{LocalNotification, MessageStatus, NotificationManagerRef}; -pub(crate) use streaming_job::*; -pub(crate) use system_param::*; +pub use catalog::*; +pub use cluster::{WorkerKey, *}; +pub use env::{MetaSrvEnv, *}; +pub use id::*; +pub use idle::*; +pub use notification::{LocalNotification, MessageStatus, NotificationManagerRef, *}; +pub use streaming_job::*; +pub use system_param::*; pub use super::model_v2::prelude; diff --git a/src/meta/src/manager/sink_coordination/coordinator_worker.rs b/src/meta/src/manager/sink_coordination/coordinator_worker.rs index 0185fdc4f4630..79f4f5b753aa2 100644 --- a/src/meta/src/manager/sink_coordination/coordinator_worker.rs +++ b/src/meta/src/manager/sink_coordination/coordinator_worker.rs @@ -46,7 +46,7 @@ macro_rules! send_await_with_err_check { }; } -pub(crate) struct CoordinatorWorker { +pub struct CoordinatorWorker { param: SinkParam, request_streams: Vec, response_senders: Vec, @@ -54,7 +54,7 @@ pub(crate) struct CoordinatorWorker { } impl CoordinatorWorker { - pub(crate) async fn run( + pub async fn run( first_writer_request: NewSinkWriterRequest, request_rx: UnboundedReceiver, ) { @@ -91,7 +91,7 @@ impl CoordinatorWorker { }); } - pub(crate) async fn execute_coordinator( + pub async fn execute_coordinator( first_writer_request: NewSinkWriterRequest, request_rx: UnboundedReceiver, coordinator: impl SinkCommitCoordinator, diff --git a/src/meta/src/manager/sink_coordination/manager.rs b/src/meta/src/manager/sink_coordination/manager.rs index a7d15bd452f34..720a698fa8e72 100644 --- a/src/meta/src/manager/sink_coordination/manager.rs +++ b/src/meta/src/manager/sink_coordination/manager.rs @@ -69,7 +69,7 @@ pub struct SinkCoordinatorManager { } impl SinkCoordinatorManager { - pub(crate) fn start_worker() -> (Self, (JoinHandle<()>, Sender<()>)) { + pub fn start_worker() -> (Self, (JoinHandle<()>, Sender<()>)) { Self::start_worker_with_spawn_worker(|writer_request, manager_request_stream| { tokio::spawn(CoordinatorWorker::run( writer_request, @@ -91,7 +91,7 @@ impl SinkCoordinatorManager { ) } - pub(crate) async fn handle_new_request( + pub async fn handle_new_request( &self, mut request_stream: SinkWriterRequestStream, ) -> Result>, Status> { @@ -143,11 +143,11 @@ impl SinkCoordinatorManager { info!("successfully stop coordinator: {:?}", sink_id); } - pub(crate) async fn reset(&self) { + pub async fn reset(&self) { self.stop_coordinator(None).await; } - pub(crate) async fn stop_sink_coordinator(&self, sink_id: SinkId) { + pub async fn stop_sink_coordinator(&self, sink_id: SinkId) { self.stop_coordinator(Some(sink_id)).await; } } diff --git a/src/meta/src/manager/sink_coordination/mod.rs b/src/meta/src/manager/sink_coordination/mod.rs index 30786c8721e97..fe861e2175343 100644 --- a/src/meta/src/manager/sink_coordination/mod.rs +++ b/src/meta/src/manager/sink_coordination/mod.rs @@ -16,19 +16,19 @@ mod coordinator_worker; mod manager; use futures::stream::BoxStream; -pub(crate) use manager::SinkCoordinatorManager; +pub use manager::SinkCoordinatorManager; use risingwave_common::buffer::Bitmap; use risingwave_connector::sink::SinkParam; use risingwave_pb::connector_service::{CoordinateRequest, CoordinateResponse}; use tokio::sync::mpsc::Sender; use tonic::Status; -pub(crate) type SinkWriterRequestStream = BoxStream<'static, Result>; -pub(crate) type SinkCoordinatorResponseSender = Sender>; +pub type SinkWriterRequestStream = BoxStream<'static, Result>; +pub type SinkCoordinatorResponseSender = Sender>; -pub(crate) struct NewSinkWriterRequest { - pub(crate) request_stream: SinkWriterRequestStream, - pub(crate) response_tx: SinkCoordinatorResponseSender, - pub(crate) param: SinkParam, - pub(crate) vnode_bitmap: Bitmap, +pub struct NewSinkWriterRequest { + pub request_stream: SinkWriterRequestStream, + pub response_tx: SinkCoordinatorResponseSender, + pub param: SinkParam, + pub vnode_bitmap: Bitmap, } diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index 6b3e71fe20092..611de4120a787 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -31,7 +31,7 @@ pub enum StreamingJob { } impl StreamingJob { - pub(crate) fn mark_created(&mut self) { + pub fn mark_created(&mut self) { let created_at_epoch = Some(Epoch::now().0); match self { StreamingJob::MaterializedView(table) => table.created_at_epoch = created_at_epoch, @@ -48,7 +48,7 @@ impl StreamingJob { } } - pub(crate) fn mark_initialized(&mut self) { + pub fn mark_initialized(&mut self) { let initialized_at_epoch = Some(Epoch::now().0); match self { StreamingJob::MaterializedView(table) => { diff --git a/src/meta/src/model/cluster.rs b/src/meta/src/model/cluster.rs index 882f48b6dc8c4..3d654a1d6b8c9 100644 --- a/src/meta/src/model/cluster.rs +++ b/src/meta/src/model/cluster.rs @@ -128,8 +128,14 @@ const CLUSTER_ID_KEY: &[u8] = "cluster_id".as_bytes(); #[derive(Clone, Debug)] pub struct ClusterId(String); +impl Default for ClusterId { + fn default() -> Self { + Self::new() + } +} + impl ClusterId { - pub(crate) fn new() -> Self { + pub fn new() -> Self { Self(Uuid::new_v4().to_string()) } @@ -139,15 +145,13 @@ impl ClusterId { )) } - pub(crate) async fn from_meta_store( + pub async fn from_meta_store( meta_store: &S, ) -> MetadataModelResult> { Self::from_snapshot::(&meta_store.snapshot().await).await } - pub(crate) async fn from_snapshot( - s: &S::Snapshot, - ) -> MetadataModelResult> { + pub async fn from_snapshot(s: &S::Snapshot) -> MetadataModelResult> { match s.get_cf(CLUSTER_ID_CF_NAME, CLUSTER_ID_KEY).await { Ok(bytes) => Ok(Some(Self::from_bytes(bytes)?)), Err(e) => match e { @@ -157,10 +161,7 @@ impl ClusterId { } } - pub(crate) async fn put_at_meta_store( - &self, - meta_store: &S, - ) -> MetadataModelResult<()> { + pub async fn put_at_meta_store(&self, meta_store: &S) -> MetadataModelResult<()> { Ok(meta_store .put_cf( CLUSTER_ID_CF_NAME, diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 5dd8f53e249b0..10fe5abe8aeaa 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -48,22 +48,22 @@ pub struct TableFragments { state: State, /// The table fragments. - pub(crate) fragments: BTreeMap, + pub fragments: BTreeMap, /// The status of actors - pub(crate) actor_status: BTreeMap, + pub actor_status: BTreeMap, /// The splits of actors - pub(crate) actor_splits: HashMap>, + pub actor_splits: HashMap>, /// The environment associated with this stream plan and its fragments - pub(crate) env: StreamEnvironment, + pub env: StreamEnvironment, } #[derive(Debug, Clone, Default)] pub struct StreamEnvironment { /// The timezone used to interpret timestamps and dates for conversion - pub(crate) timezone: Option, + pub timezone: Option, } impl StreamEnvironment { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 3ad5df55a997c..722cce94b866a 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -178,7 +178,7 @@ impl CreatingStreamingJobPermit { } impl DdlController { - pub(crate) async fn new( + pub async fn new( env: MetaSrvEnv, catalog_manager: CatalogManagerRef, stream_manager: GlobalStreamManagerRef, @@ -217,7 +217,7 @@ impl DdlController { /// has been interrupted during executing, the request will be cancelled by tonic. Since we have /// a lot of logic for revert, status management, notification and so on, ensuring consistency /// would be a huge hassle and pain if we don't spawn here. - pub(crate) async fn run_command(&self, command: DdlCommand) -> MetaResult { + pub async fn run_command(&self, command: DdlCommand) -> MetaResult { self.check_barrier_manager_status().await?; let ctrl = self.clone(); let fut = async move { @@ -263,7 +263,7 @@ impl DdlController { tokio::spawn(fut).await.unwrap() } - pub(crate) async fn get_ddl_progress(&self) -> Vec { + pub async fn get_ddl_progress(&self) -> Vec { self.barrier_manager.get_ddl_progress().await } diff --git a/src/meta/src/rpc/election/sql.rs b/src/meta/src/rpc/election/sql.rs index b6bd02b179c26..af8081f170089 100644 --- a/src/meta/src/rpc/election/sql.rs +++ b/src/meta/src/rpc/election/sql.rs @@ -30,13 +30,13 @@ pub struct SqlBackendElectionClient { } #[derive(sqlx::FromRow, Debug)] -pub(crate) struct ElectionRow { +pub struct ElectionRow { service: String, id: String, } #[async_trait::async_trait] -pub(crate) trait SqlDriver: Send + Sync + 'static { +pub trait SqlDriver: Send + Sync + 'static { async fn update_heartbeat(&self, service_name: &str, id: &str) -> MetaResult<()>; async fn try_campaign(&self, service_name: &str, id: &str, ttl: i64) @@ -48,7 +48,7 @@ pub(crate) trait SqlDriver: Send + Sync + 'static { async fn resign(&self, service_name: &str, id: &str) -> MetaResult<()>; } -pub(crate) trait SqlDriverCommon { +pub trait SqlDriverCommon { const ELECTION_LEADER_TABLE_NAME: &'static str = "election_leader"; const ELECTION_MEMBER_TABLE_NAME: &'static str = "election_members"; diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index f11bbe7f41138..3183007753cbd 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -37,7 +37,7 @@ use tokio::task::JoinHandle; use crate::hummock::HummockManagerRef; use crate::manager::{CatalogManagerRef, ClusterManagerRef, FragmentManagerRef}; -use crate::rpc::server::ElectionClientRef; +use crate::rpc::ElectionClientRef; #[derive(Clone)] pub struct MetaMetrics { @@ -690,7 +690,7 @@ impl Default for MetaMetrics { } } -pub async fn start_worker_info_monitor( +pub fn start_worker_info_monitor( cluster_manager: ClusterManagerRef, election_client: Option, interval: Duration, @@ -738,7 +738,7 @@ pub async fn start_worker_info_monitor( (join_handle, shutdown_tx) } -pub async fn start_fragment_info_monitor( +pub fn start_fragment_info_monitor( cluster_manager: ClusterManagerRef, catalog_manager: CatalogManagerRef, fragment_manager: FragmentManagerRef, diff --git a/src/meta/src/rpc/mod.rs b/src/meta/src/rpc/mod.rs index 36380c4d2dafb..99f1b51eaafce 100644 --- a/src/meta/src/rpc/mod.rs +++ b/src/meta/src/rpc/mod.rs @@ -12,19 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod cloud_provider; +pub mod cloud_provider; pub mod ddl_controller; pub mod election; -mod intercept; +pub mod intercept; pub mod metrics; -pub mod server; -pub mod service; + +pub type ElectionClientRef = std::sync::Arc; pub use election::etcd::EtcdElectionClient; pub use election::{ElectionClient, ElectionMember}; -pub use service::cluster_service::ClusterServiceImpl; -pub use service::ddl_service::DdlServiceImpl; -pub use service::heartbeat_service::HeartbeatServiceImpl; -pub use service::hummock_service::HummockServiceImpl; -pub use service::notification_service::NotificationServiceImpl; -pub use service::stream_service::StreamServiceImpl; diff --git a/src/meta/src/serving/mod.rs b/src/meta/src/serving/mod.rs index f6d1a5b1aa714..521a8b9ad1c0d 100644 --- a/src/meta/src/serving/mod.rs +++ b/src/meta/src/serving/mod.rs @@ -103,7 +103,7 @@ fn to_deleted_fragment_parallel_unit_mapping( .collect() } -pub(crate) async fn on_meta_start( +pub async fn on_meta_start( notification_manager: NotificationManagerRef, cluster_manager: ClusterManagerRef, fragment_manager: FragmentManagerRef, @@ -126,7 +126,7 @@ pub(crate) async fn on_meta_start( ); } -pub(crate) async fn start_serving_vnode_mapping_worker( +pub async fn start_serving_vnode_mapping_worker( notification_manager: NotificationManagerRef, cluster_manager: ClusterManagerRef, fragment_manager: FragmentManagerRef, diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index a125d61d91703..afe6186165e22 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -97,7 +97,7 @@ pub struct RescheduleOptions { pub resolve_no_shuffle_upstream: bool, } -pub(crate) struct RescheduleContext { +pub struct RescheduleContext { /// Index used to map `ParallelUnitId` to `WorkerId` parallel_unit_id_to_worker_id: BTreeMap, /// Meta information for all Actors @@ -171,7 +171,7 @@ impl RescheduleContext { /// assert to fail and should be skipped from the upper level. /// /// The return value is the bitmap distribution after scaling, which covers all virtual node indexes -pub(crate) fn rebalance_actor_vnode( +pub fn rebalance_actor_vnode( actors: &[StreamActor], actors_to_remove: &BTreeSet, actors_to_create: &BTreeSet, diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index fa3159f6657ec..1cd666e5d7160 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -47,7 +47,7 @@ pub type SourceManagerRef = Arc; pub type SplitAssignment = HashMap>>; pub struct SourceManager { - pub(crate) paused: Mutex<()>, + pub paused: Mutex<()>, env: MetaSrvEnv, barrier_scheduler: BarrierScheduler, core: Mutex, diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 46d6c5cb1ea22..44198f65635cb 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -159,26 +159,26 @@ pub struct ReplaceTableContext { /// `GlobalStreamManager` manages all the streams in the system. pub struct GlobalStreamManager { - pub(crate) env: MetaSrvEnv, + pub env: MetaSrvEnv, /// Manages definition and status of fragments and actors pub(super) fragment_manager: FragmentManagerRef, /// Broadcasts and collect barriers - pub(crate) barrier_scheduler: BarrierScheduler, + pub barrier_scheduler: BarrierScheduler, /// Maintains information of the cluster - pub(crate) cluster_manager: ClusterManagerRef, + pub cluster_manager: ClusterManagerRef, /// Maintains streaming sources from external system like kafka - pub(crate) source_manager: SourceManagerRef, + pub source_manager: SourceManagerRef, /// Creating streaming job info. creating_job_info: CreatingStreamingJobInfoRef, hummock_manager: HummockManagerRef, - pub(crate) reschedule_lock: RwLock<()>, + pub reschedule_lock: RwLock<()>, } impl GlobalStreamManager { diff --git a/src/meta/src/telemetry.rs b/src/meta/src/telemetry.rs index 774b3cdda8146..fbbc89c2ff0ec 100644 --- a/src/meta/src/telemetry.rs +++ b/src/meta/src/telemetry.rs @@ -35,7 +35,7 @@ struct NodeCount { } #[derive(Debug, Serialize, Deserialize)] -pub(crate) struct MetaTelemetryReport { +pub struct MetaTelemetryReport { #[serde(flatten)] base: TelemetryReportBase, node_count: NodeCount, @@ -45,12 +45,12 @@ pub(crate) struct MetaTelemetryReport { impl TelemetryReport for MetaTelemetryReport {} -pub(crate) struct MetaTelemetryInfoFetcher { +pub struct MetaTelemetryInfoFetcher { tracking_id: ClusterId, } impl MetaTelemetryInfoFetcher { - pub(crate) fn new(tracking_id: ClusterId) -> Self { + pub fn new(tracking_id: ClusterId) -> Self { Self { tracking_id } } } @@ -63,13 +63,13 @@ impl TelemetryInfoFetcher for MetaTelemetryInfoFetcher { } #[derive(Clone)] -pub(crate) struct MetaReportCreator { +pub struct MetaReportCreator { cluster_mgr: Arc, meta_backend: MetaBackend, } impl MetaReportCreator { - pub(crate) fn new(cluster_mgr: Arc, meta_backend: MetaBackend) -> Self { + pub fn new(cluster_mgr: Arc, meta_backend: MetaBackend) -> Self { Self { cluster_mgr, meta_backend, @@ -79,6 +79,7 @@ impl MetaReportCreator { #[async_trait::async_trait] impl TelemetryReportCreator for MetaReportCreator { + #[expect(refining_impl_trait)] async fn create_report( &self, tracking_id: String, diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 6bbcbd2ebf3e2..3e744bb61608d 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -16,7 +16,6 @@ //! response gRPC message structs. #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(result_option_inspect)] #![feature(type_alias_impl_trait)] #![feature(associated_type_defaults)] diff --git a/src/source/Cargo.toml b/src/source/Cargo.toml index bf60bc45f7395..aedb0b9158908 100644 --- a/src/source/Cargo.toml +++ b/src/source/Cargo.toml @@ -15,7 +15,6 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" -easy-ext = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } itertools = "0.11" diff --git a/src/source/src/lib.rs b/src/source/src/lib.rs index 1a4f5d5f47280..1a32888cdf651 100644 --- a/src/source/src/lib.rs +++ b/src/source/src/lib.rs @@ -14,7 +14,6 @@ #![allow(clippy::derive_partial_eq_without_eq)] #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(lint_reasons)] #![feature(result_option_inspect)] #![feature(generators)] diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index 0797c462d61e5..f1022ab2fd935 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -14,7 +14,6 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -anyhow = "1" arc-swap = "1" async-trait = "0.1" auto_enums = { version = "0.8", features = ["futures03"] } diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index 330dfbc4de44c..3e0549db188a2 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -14,7 +14,6 @@ #![allow(clippy::derive_partial_eq_without_eq)] #![feature(trait_alias)] -#![feature(binary_heap_drain_sorted)] #![feature(type_alias_impl_trait)] #![feature(extract_if)] #![feature(custom_test_frameworks)] diff --git a/src/storage/compactor/Cargo.toml b/src/storage/compactor/Cargo.toml index 54d48a88923a6..e6e985b2ba424 100644 --- a/src/storage/compactor/Cargo.toml +++ b/src/storage/compactor/Cargo.toml @@ -15,12 +15,10 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -anyhow = "1" async-trait = "0.1" await-tree = { workspace = true } clap = { version = "4", features = ["derive"] } parking_lot = "0.12" -prometheus = { version = "0.13" } risingwave_common = { workspace = true } risingwave_common_heap_profiling = { workspace = true } risingwave_common_service = { workspace = true } @@ -29,7 +27,6 @@ risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } risingwave_storage = { workspace = true } serde = { version = "1", features = ["derive"] } -serde_json = "1" tokio = { version = "0.2", package = "madsim-tokio", features = [ "fs", "rt", diff --git a/src/storage/src/lib.rs b/src/storage/src/lib.rs index 72b925170a6ef..c5ffe656ab893 100644 --- a/src/storage/src/lib.rs +++ b/src/storage/src/lib.rs @@ -14,7 +14,6 @@ #![feature(allocator_api)] #![feature(arc_unwrap_or_clone)] -#![feature(binary_heap_drain_sorted)] #![feature(bound_as_ref)] #![feature(bound_map)] #![feature(custom_test_frameworks)] diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index 9039f34d89a5d..9e9e77b92ceec 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -21,7 +21,6 @@ async-stream = "0.3" async-trait = "0.1" await-tree = { workspace = true } bytes = "1" -dyn-clone = "1" educe = "0.4" either = "1" enum-as-inner = "0.6" @@ -33,16 +32,13 @@ governor = { version = "0.6", default-features = false, features = [ "jitter", ] } hytra = "0.1.2" -iter-chunks = "0.1" itertools = "0.11" local_stats_alloc = { path = "../utils/local_stats_alloc" } lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } maplit = "1.0.2" memcomparable = "0.2" multimap = "0.9" -num-traits = "0.2" parking_lot = "0.12" -parse-display = "0.8" pin-project = "1" prometheus = { version = "0.13", features = ["process"] } prost = { workspace = true } @@ -57,7 +53,6 @@ risingwave_source = { workspace = true } risingwave_storage = { workspace = true } serde_json = "1" smallvec = "1" -spin = "0.9" static_assertions = "1" thiserror = "1" tokio = { version = "0.2", package = "madsim-tokio", features = [ diff --git a/src/stream/src/lib.rs b/src/stream/src/lib.rs index afa1d3faeee22..5a68b1b712b26 100644 --- a/src/stream/src/lib.rs +++ b/src/stream/src/lib.rs @@ -18,7 +18,6 @@ #![feature(type_alias_impl_trait)] #![feature(more_qualified_paths)] #![feature(lint_reasons)] -#![feature(binary_heap_drain_sorted)] #![feature(let_chains)] #![feature(hash_extract_if)] #![feature(extract_if)] diff --git a/src/tests/compaction_test/Cargo.toml b/src/tests/compaction_test/Cargo.toml index dd3e5d0a53699..87ad5946b26d5 100644 --- a/src/tests/compaction_test/Cargo.toml +++ b/src/tests/compaction_test/Cargo.toml @@ -27,6 +27,7 @@ risingwave_compactor = { workspace = true } risingwave_hummock_sdk = { workspace = true } risingwave_hummock_test = { workspace = true } risingwave_meta = { workspace = true } +risingwave_meta_node = { workspace = true } risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index 3e2f993cf9613..cf3e35b48c692 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -129,7 +129,7 @@ pub async fn compaction_test_main( } pub async fn start_meta_node(listen_addr: String, state_store: String, config_path: String) { - let meta_opts = risingwave_meta::MetaNodeOpts::parse_from([ + let meta_opts = risingwave_meta_node::MetaNodeOpts::parse_from([ "meta-node", "--listen-addr", &listen_addr, @@ -154,7 +154,7 @@ pub async fn start_meta_node(listen_addr: String, state_store: String, config_pa "enable_compaction_deterministic should be set" ); - risingwave_meta::start(meta_opts).await + risingwave_meta_node::start(meta_opts).await } async fn start_compactor_node( diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index c79c324ba51ec..b81be65edae42 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -37,7 +37,7 @@ risingwave_ctl = { workspace = true } risingwave_e2e_extended_mode_test = { path = "../e2e_extended_mode" } risingwave_expr_impl = { workspace = true } risingwave_frontend = { workspace = true } -risingwave_meta = { workspace = true } +risingwave_meta_node = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } risingwave_sqlparser = { workspace = true } diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index 4eb60e7af14dc..6cc6168513cd4 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -255,7 +255,7 @@ impl Cluster { // meta node for i in 1..=conf.meta_nodes { - let opts = risingwave_meta::MetaNodeOpts::parse_from([ + let opts = risingwave_meta_node::MetaNodeOpts::parse_from([ "meta-node", "--config-path", conf.config_path.as_str(), @@ -276,7 +276,7 @@ impl Cluster { .create_node() .name(format!("meta-{i}")) .ip([192, 168, 1, i as u8].into()) - .init(move || risingwave_meta::start(opts.clone())) + .init(move || risingwave_meta_node::start(opts.clone())) .build(); } diff --git a/src/utils/runtime/Cargo.toml b/src/utils/runtime/Cargo.toml index a977f35e67dd2..8bd4e49d808a9 100644 --- a/src/utils/runtime/Cargo.toml +++ b/src/utils/runtime/Cargo.toml @@ -16,10 +16,6 @@ normal = ["workspace-hack"] [dependencies] await-tree = { workspace = true } -chrono = { version = "0.4", default-features = false, features = [ - "clock", - "std", -] } console = "0.15" console-subscriber = "0.2.0" either = "1" @@ -29,7 +25,6 @@ opentelemetry-otlp = { version = "0.13" } opentelemetry-semantic-conventions = "0.12" parking_lot = { version = "0.12", features = ["deadlock_detection"] } pprof = { version = "0.13", features = ["flamegraph"] } -prometheus = { version = "0.13" } risingwave_common = { workspace = true } risingwave_variables = { workspace = true } rlimit = "0.10" From 52f4c630ba60a4a2e2c7915e776744d3cbcfb901 Mon Sep 17 00:00:00 2001 From: Croxx Date: Wed, 18 Oct 2023 03:10:10 -0500 Subject: [PATCH 05/58] fix: don't count file cache buffer pool size if disabled (#12941) Signed-off-by: MrCroxx --- src/compute/src/memory_management/mod.rs | 51 ++++++++++++++++-------- 1 file changed, 35 insertions(+), 16 deletions(-) diff --git a/src/compute/src/memory_management/mod.rs b/src/compute/src/memory_management/mod.rs index 3ae47729b336a..0323cb2ddc093 100644 --- a/src/compute/src/memory_management/mod.rs +++ b/src/compute/src/memory_management/mod.rs @@ -163,22 +163,30 @@ pub fn storage_memory_config( // // Adding more buffers can prevent allocators from waiting for buffers to be freed by flushers. - let data_file_cache_buffer_pool_capacity_mb = storage_config - .data_file_cache - .buffer_pool_size_mb - .unwrap_or( - storage_config.data_file_cache.file_capacity_mb - * (storage_config.data_file_cache.flushers - + 2 * (1 << storage_config.data_file_cache.allocation_bits)), - ); - let meta_file_cache_buffer_pool_capacity_mb = storage_config - .meta_file_cache - .buffer_pool_size_mb - .unwrap_or( - storage_config.meta_file_cache.file_capacity_mb - * (storage_config.meta_file_cache.flushers - + 2 * (1 << storage_config.meta_file_cache.allocation_bits)), - ); + let data_file_cache_buffer_pool_capacity_mb = if storage_config.data_file_cache.dir.is_empty() { + 0 + } else { + storage_config + .data_file_cache + .buffer_pool_size_mb + .unwrap_or( + storage_config.data_file_cache.file_capacity_mb + * (storage_config.data_file_cache.flushers + + 2 * (1 << storage_config.data_file_cache.allocation_bits)), + ) + }; + let meta_file_cache_buffer_pool_capacity_mb = if storage_config.meta_file_cache.dir.is_empty() { + 0 + } else { + storage_config + .meta_file_cache + .buffer_pool_size_mb + .unwrap_or( + storage_config.meta_file_cache.file_capacity_mb + * (storage_config.meta_file_cache.flushers + + 2 * (1 << storage_config.meta_file_cache.allocation_bits)), + ) + }; let compactor_memory_limit_mb = storage_config.compactor_memory_limit_mb.unwrap_or( ((non_reserved_memory_bytes as f64 * compactor_memory_proportion).ceil() as usize) >> 20, @@ -238,6 +246,17 @@ mod tests { let mut storage_config = StorageConfig::default(); let total_non_reserved_memory_bytes = 8 << 30; + let memory_config = + storage_memory_config(total_non_reserved_memory_bytes, true, &storage_config); + assert_eq!(memory_config.block_cache_capacity_mb, 737); + assert_eq!(memory_config.meta_cache_capacity_mb, 860); + assert_eq!(memory_config.shared_buffer_capacity_mb, 737); + assert_eq!(memory_config.data_file_cache_buffer_pool_capacity_mb, 0); + assert_eq!(memory_config.meta_file_cache_buffer_pool_capacity_mb, 0); + assert_eq!(memory_config.compactor_memory_limit_mb, 819); + + storage_config.data_file_cache.dir = "data".to_string(); + storage_config.meta_file_cache.dir = "meta".to_string(); let memory_config = storage_memory_config(total_non_reserved_memory_bytes, true, &storage_config); assert_eq!(memory_config.block_cache_capacity_mb, 737); From 09087404564d23d1e23f29d0f5524261aceea1d2 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Wed, 18 Oct 2023 16:18:29 +0800 Subject: [PATCH 06/58] fix(risedev): fill-in CONNECTOR_LIBS_PATH if set (#12928) --- src/risedevtool/config/src/main.rs | 6 ------ src/risedevtool/src/task/compute_node_service.rs | 8 ++++++++ src/risedevtool/src/task/meta_node_service.rs | 8 ++++++++ 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/risedevtool/config/src/main.rs b/src/risedevtool/config/src/main.rs index ac36d475b83c3..79df34bd815dc 100644 --- a/src/risedevtool/config/src/main.rs +++ b/src/risedevtool/config/src/main.rs @@ -396,12 +396,6 @@ fn main() -> Result<()> { )?; if chosen.contains(&component) { writeln!(file, "{}=true", component.env())?; - if component == Components::BuildConnectorNode { - writeln!( - file, - "CONNECTOR_LIBS_PATH=.risingwave/bin/connector-node/libs/" - )?; - } } else { writeln!(file, "# {}=true", component.env())?; } diff --git a/src/risedevtool/src/task/compute_node_service.rs b/src/risedevtool/src/task/compute_node_service.rs index 5bb9c23cbf21f..ced6bec115f6a 100644 --- a/src/risedevtool/src/task/compute_node_service.rs +++ b/src/risedevtool/src/task/compute_node_service.rs @@ -102,6 +102,14 @@ impl Task for ComputeNodeService { cmd.env("MALLOC_CONF", conf); // unprefixed for linux } + if crate::util::is_env_set("ENABLE_BUILD_RW_CONNECTOR") { + let prefix_bin = env::var("PREFIX_BIN")?; + cmd.env( + "CONNECTOR_LIBS_PATH", + Path::new(&prefix_bin).join("connector-node/libs/"), + ); + } + cmd.arg("--config-path") .arg(Path::new(&prefix_config).join("risingwave.toml")); Self::apply_command_args(&mut cmd, &self.config)?; diff --git a/src/risedevtool/src/task/meta_node_service.rs b/src/risedevtool/src/task/meta_node_service.rs index 1b53af8f9b544..2494a9eceaf16 100644 --- a/src/risedevtool/src/task/meta_node_service.rs +++ b/src/risedevtool/src/task/meta_node_service.rs @@ -186,6 +186,14 @@ impl Task for MetaNodeService { cmd.env("MALLOC_CONF", conf); // unprefixed for linux } + if crate::util::is_env_set("ENABLE_BUILD_RW_CONNECTOR") { + let prefix_bin = env::var("PREFIX_BIN")?; + cmd.env( + "CONNECTOR_LIBS_PATH", + Path::new(&prefix_bin).join("connector-node/libs/"), + ); + } + Self::apply_command_args(&mut cmd, &self.config, HummockInMemoryStrategy::Isolated)?; let prefix_config = env::var("PREFIX_CONFIG")?; From de2b8565e41ab7dd13514eb582107721c9b248f7 Mon Sep 17 00:00:00 2001 From: August Date: Wed, 18 Oct 2023 16:46:57 +0800 Subject: [PATCH 07/58] fix: fix alter index rename (#12944) --- e2e_test/ddl/alter_rename_relation.slt | 5 +++++ src/meta/src/manager/catalog/mod.rs | 1 + 2 files changed, 6 insertions(+) diff --git a/e2e_test/ddl/alter_rename_relation.slt b/e2e_test/ddl/alter_rename_relation.slt index fe4f16fec12bd..df085a273b0a9 100644 --- a/e2e_test/ddl/alter_rename_relation.slt +++ b/e2e_test/ddl/alter_rename_relation.slt @@ -128,6 +128,11 @@ public.mv_on_v1 CREATE MATERIALIZED VIEW mv_on_v1 AS SELECT * FROM v5 AS v1 statement ok ALTER INDEX idx RENAME TO idx1; +query TT +SHOW CREATE INDEX idx1; +---- +public.idx1 CREATE INDEX idx1 ON t2(v1) + statement ok INSERT INTO t2 VALUES(1,(1,(1,2))); diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index e1c13c315a377..be3ed42b01b4d 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1591,6 +1591,7 @@ impl CatalogManager { // 2. rename index name. index.name = index_name.to_string(); index_table.name = index_name.to_string(); + index_table.definition = alter_relation_rename(&index_table.definition, index_name); let mut indexes = BTreeMapTransaction::new(&mut database_core.indexes); let mut tables = BTreeMapTransaction::new(&mut database_core.tables); indexes.insert(index_id, index.clone()); From d7e5772dc1cb7d040ea53ce10782a7e738568e17 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 18 Oct 2023 17:03:53 +0800 Subject: [PATCH 08/58] chore: update deps for feature store demo (#12897) Signed-off-by: TennyZhuang Co-authored-by: TennyZhuang --- integration_tests/feature-store/Dockerfile | 20 +- integration_tests/feature-store/README.md | 16 +- .../feature-store/docker-compose.yml | 1 - .../feature-store/server/Cargo.lock | 1113 ++++++++++------- .../feature-store/server/Cargo.toml | 19 +- .../server/model/requirements.txt | 8 +- .../feature-store/server/src/feature_store.rs | 8 +- .../feature-store/server/src/main.rs | 22 +- .../feature-store/server/src/model.rs | 232 ++-- .../feature-store/server/src/server_pb.rs | 468 ++++--- .../feature-store/server/src/serving.rs | 11 +- .../feature-store/simulator/Cargo.lock | 890 ++++++------- .../feature-store/simulator/Cargo.toml | 17 +- .../feature-store/simulator/rust-toolchain | 3 +- .../feature-store/simulator/src/entities.rs | 6 +- .../simulator/src/entities_taxi.rs | 2 +- .../feature-store/simulator/src/main.rs | 14 +- .../feature-store/simulator/src/server_pb.rs | 442 ++++--- .../feature-store/simulator/src/simulation.rs | 8 +- 19 files changed, 1891 insertions(+), 1409 deletions(-) diff --git a/integration_tests/feature-store/Dockerfile b/integration_tests/feature-store/Dockerfile index dc0d02bd79682..7aadc5fb15d72 100644 --- a/integration_tests/feature-store/Dockerfile +++ b/integration_tests/feature-store/Dockerfile @@ -1,5 +1,6 @@ -FROM rust:1.67 as feature-store-server +FROM ubuntu:20.04 AS feature-store-server ARG BUILD_ARG +ENV DEBIAN_FRONTEND=noninteractive USER root @@ -9,8 +10,11 @@ RUN mkdir -p $WORK_DIR WORKDIR $WORK_DIR RUN apt update -RUN apt install -y python3 python3-pip wget ca-certificates -RUN apt install -y postgresql-client +RUN apt install -y lsof curl openssl libssl-dev pkg-config build-essential \ + cmake \ + python3 python3-pip wget ca-certificates \ + postgresql-client \ + protobuf-compiler ADD ./server/model/requirements.txt $WORK_DIR/model-pipreqs.txt ADD ./generator/requirements.txt $WORK_DIR/generator-pipreqs.txt @@ -19,8 +23,8 @@ RUN pip3 install -r $WORK_DIR/model-pipreqs.txt RUN pip3 install -r $WORK_DIR/generator-pipreqs.txt RUN pip3 install risingwave -RUN apt install -y lsof curl openssl libssl-dev pkg-config build-essential -RUN apt install -y cmake librdkafka-dev +RUN curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh -s -- --no-modify-path -y +ENV PATH /root/.cargo/bin/:$PATH # Install .NET 6.0 RUN wget https://packages.microsoft.com/config/debian/11/packages-microsoft-prod.deb -O packages-microsoft-prod.deb @@ -49,11 +53,11 @@ ADD ./run.sh $WORK_DIR/run-sh/ ADD ./run-mfa.sh $WORK_DIR/run-sh/ RUN if [ "$BUILD_ARG" = "mfa" ]; then \ - cp $WORK_DIR/run-sh/run-mfa.sh $WORK_DIR/run.sh;\ + cp $WORK_DIR/run-sh/run-mfa.sh $WORK_DIR/run.sh;\ else \ - cp $WORK_DIR/run-sh/run.sh $WORK_DIR/run.sh;\ + cp $WORK_DIR/run-sh/run.sh $WORK_DIR/run.sh;\ fi RUN chmod +x $WORK_DIR/run.sh && rm -rf $WORK_DIR/run-sh -CMD ["sh", "-c", "sleep 10 && ./run.sh"] \ No newline at end of file +CMD ["sh", "-c", "sleep 10 && ./run.sh"] diff --git a/integration_tests/feature-store/README.md b/integration_tests/feature-store/README.md index 425efadf27c68..f74c679e73ffb 100644 --- a/integration_tests/feature-store/README.md +++ b/integration_tests/feature-store/README.md @@ -25,7 +25,9 @@ When a user needs to make a prediction using these features, they can provide th 1. Build docker. Kafka RisingWave and Feature Store. -```docker compose up --build``` +``` +docker compose up --build +``` The Feature Store system performs several tasks in sequence: @@ -40,7 +42,9 @@ The Feature Store system performs several tasks in sequence: 2. Then we can get the simulation results for Feature store in `.log`. -```cat .log/simulator_log``` +``` +cat .log/simulator_log +``` # Account change feature store #### Case Description @@ -53,8 +57,12 @@ In this case, we need to calculate the frequency and count of user account chang 1. Build docker. Kafka RisingWave and Feature Store. -```docker compose build --build-arg BUILD_ARG=mfa``` +``` +docker compose build --build-arg BUILD_ARG=mfa +``` 2. Then we can get the simulation results for Feature store in `.log`. -```cat .log/simulator_log``` \ No newline at end of file +``` +cat .log/simulator_log +``` \ No newline at end of file diff --git a/integration_tests/feature-store/docker-compose.yml b/integration_tests/feature-store/docker-compose.yml index d212a80369a38..caa950ea87ccd 100644 --- a/integration_tests/feature-store/docker-compose.yml +++ b/integration_tests/feature-store/docker-compose.yml @@ -66,7 +66,6 @@ services: - "./mfa-start.sql:/mfa-start.sql" - "./mfa-mock.sql:/mfa-mock.sql" feature-store: - image: rust:1.67 build: context: . target: feature-store-server diff --git a/integration_tests/feature-store/server/Cargo.lock b/integration_tests/feature-store/server/Cargo.lock index a678176645772..452bbfe3db14d 100644 --- a/integration_tests/feature-store/server/Cargo.lock +++ b/integration_tests/feature-store/server/Cargo.lock @@ -4,9 +4,9 @@ version = 3 [[package]] name = "addr2line" -version = "0.20.0" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4fa78e18c64fce05e902adecd7a5eed15a5e0a3439f7b0e169f0252214865e3" +checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" dependencies = [ "gimli", ] @@ -19,10 +19,11 @@ checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" [[package]] name = "ahash" -version = "0.7.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcb51a0695d8f838b1ee009b3fbf66bda078cd64590202a864a8f3e8c4315c47" +checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" dependencies = [ + "cfg-if", "getrandom", "once_cell", "version_check", @@ -30,27 +31,72 @@ dependencies = [ [[package]] name = "aho-corasick" -version = "1.0.2" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43f6cb1bf222025340178f382c426f13757b2960e89779dfcb319c32542a5a41" +checksum = "b2969dcb958b36655471fc61f7e416fa76033bdd4bfed0678d8fee1e2d07a1f0" dependencies = [ "memchr", ] [[package]] -name = "ansi_term" -version = "0.12.1" +name = "allocator-api2" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0942ffc6dcaadf03badf6e6a2d0228460359d5e34b57ccdc720b7382dfbd5ec5" + +[[package]] +name = "anstream" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d52a9bb7ec0cf484c551830a7ce27bd20d67eac647e1befb56b0be4ee39a55d2" +checksum = "2ab91ebe16eb252986481c5b62f6098f3b698a45e34b5b98200cf20dd2484a44" dependencies = [ - "winapi", + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" + +[[package]] +name = "anstyle-parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "317b9a89c1868f5ea6ff1d9539a69f45dffc21ce321ac1fd1160dfa48c8e2140" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ca11d4be1bab0c8bc8734a9aa7bf4ee8316d462a08c6ac5052f888fef5b494b" +dependencies = [ + "windows-sys", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0699d10d2f4d628a98ee7b57b289abbc98ff3bad977cb3152709d4bf2330628" +dependencies = [ + "anstyle", + "windows-sys", ] [[package]] name = "anyhow" -version = "1.0.72" +version = "1.0.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b13c32d80ecc7ab747b80c3784bce54ee8a7a0cc4fbda9bf4cda2cf6fe90854" +checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" [[package]] name = "async-stream" @@ -71,40 +117,29 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] name = "async-trait" -version = "0.1.72" +version = "0.1.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6dde6e4ed435a4c1ee4e73592f5ba9da2151af10076cc04858746af9352d09" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] name = "atoi" -version = "0.4.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "616896e05fc0e2649463a93a15183c6a16bf03413a7af88ef1285ddedfa9cda5" +checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" dependencies = [ "num-traits", ] -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi 0.1.19", - "libc", - "winapi", -] - [[package]] name = "autocfg" version = "1.1.0" @@ -113,9 +148,9 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "axum" -version = "0.5.17" +version = "0.6.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acee9fd5073ab6b045a275b3e709c163dd36c90685219cb21804a147b58dba43" +checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" dependencies = [ "async-trait", "axum-core", @@ -131,20 +166,19 @@ dependencies = [ "mime", "percent-encoding", "pin-project-lite", + "rustversion", "serde", "sync_wrapper", - "tokio", "tower", - "tower-http", "tower-layer", "tower-service", ] [[package]] name = "axum-core" -version = "0.2.9" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37e5939e02c56fecd5c017c37df4238c0a839fa76b7f97acdd7efb804fd181cc" +checksum = "759fa577a247914fd3f7f76d62972792636412fbfd634cd452f6a385a74d2d2c" dependencies = [ "async-trait", "bytes", @@ -152,15 +186,16 @@ dependencies = [ "http", "http-body", "mime", + "rustversion", "tower-layer", "tower-service", ] [[package]] name = "backtrace" -version = "0.3.68" +version = "0.3.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4319208da049c43661739c5fade2ba182f09d1dc2299b32298d3a31692b17e12" +checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" dependencies = [ "addr2line", "cc", @@ -173,15 +208,15 @@ dependencies = [ [[package]] name = "base64" -version = "0.13.1" +version = "0.21.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e1b586273c5702936fe7b7d6896644d8be71e6314cfe09d3167c95f712589e8" +checksum = "9ba43ea6f343b788c8764558649e08df62f86c6ef251fdaeb1ffd010a9ae50a2" [[package]] -name = "base64" -version = "0.21.2" +name = "base64ct" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "604178f6c5c21f02dc555784810edfb88d34ac2c73b2eae109655649ee73ce3d" +checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" [[package]] name = "bitflags" @@ -191,9 +226,12 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.3.3" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "630be753d4e58660abd17930c71b647fe46c27ea6b63cc59e1e3851406972e42" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" +dependencies = [ + "serde", +] [[package]] name = "block-buffer" @@ -206,27 +244,27 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.13.0" +version = "3.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3e2c3daef883ecc1b5d58c15adae93470a91d425f3532ba1695849656af3fc1" +checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" [[package]] name = "byteorder" -version = "1.4.3" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89b2fd2a0dcf38d7971e2194b6b6eebab45ae01067456a7fd93d5547a61b70be" +checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" [[package]] name = "cc" -version = "1.0.82" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "305fe645edc1442a0fa8b6726ba61d422798d37a52e12eaecf4b022ebbb88f01" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" dependencies = [ "libc", ] @@ -239,19 +277,31 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "clap" -version = "2.34.0" +version = "4.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0610544180c38b88101fecf2dd634b174a62eef6946f84dfc6a7127512b381c" +checksum = "d04704f56c2cde07f43e8e2c154b43f216dc5c92fc98ada720177362f953b956" dependencies = [ - "ansi_term", - "atty", - "bitflags 1.3.2", + "clap_builder", +] + +[[package]] +name = "clap_builder" +version = "4.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e231faeaca65ebd1ea3c737966bf858971cd38c3849107aa3ea7de90a804e45" +dependencies = [ + "anstream", + "anstyle", + "clap_lex", "strsim", - "textwrap", - "unicode-width", - "vec_map", ] +[[package]] +name = "clap_lex" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd7cc57abe963c6d3b9d8be5b06ba7c8957a930305ca90304f24ef040aa6f961" + [[package]] name = "cmake" version = "0.1.50" @@ -261,6 +311,18 @@ dependencies = [ "cc", ] +[[package]] +name = "colorchoice" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" + +[[package]] +name = "const-oid" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "28c122c3980598d243d63d9a704629a2d748d101f278052ff068be5a4423ab6f" + [[package]] name = "core-foundation" version = "0.9.3" @@ -288,18 +350,18 @@ dependencies = [ [[package]] name = "crc" -version = "2.1.0" +version = "3.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49fc9a695bca7f35f5f4c15cddc84415f66a74ea78eef08e90c5024f2b540e23" +checksum = "86ec7a15cbe22e59248fc7eadb1907dab5ba09372595da4d73dd805ed4417dfe" dependencies = [ "crc-catalog", ] [[package]] name = "crc-catalog" -version = "1.1.1" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccaeedb56da03b09f598226e25e80088cb4cd25f316e6e4df7d695f0feeb1403" +checksum = "9cace84e55f07e7301bae1c519df89cdad8cc3cd868413d3fdbdeca9ff3db484" [[package]] name = "crossbeam-queue" @@ -330,6 +392,17 @@ dependencies = [ "typenum", ] +[[package]] +name = "der" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fffa369a668c8af7dbf8b5e56c9f744fbd399949ed171606040001947de40b1c" +dependencies = [ + "const-oid", + "pem-rfc7468", + "zeroize", +] + [[package]] name = "digest" version = "0.10.7" @@ -337,41 +410,25 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", + "const-oid", "crypto-common", "subtle", ] [[package]] -name = "dirs" -version = "4.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3aa72a6f96ea37bbc5aa912f6788242832f75369bdfdadcb0e38423f100059" -dependencies = [ - "dirs-sys", -] - -[[package]] -name = "dirs-sys" -version = "0.3.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b1d1d91c932ef41c0f2663aa8b0ca0342d444d842c06914aa0a7e352d0bada6" -dependencies = [ - "libc", - "redox_users", - "winapi", -] - -[[package]] -name = "dotenv" -version = "0.15.0" +name = "dotenvy" +version = "0.15.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77c90badedccf4105eca100756a0b1289e191f6fcbdadd3cee1d2f614f97da8f" +checksum = "1aaf95b3e5c8f23aa320147307562d361db0ae0d51242340f558153b4eb2439b" [[package]] name = "either" version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" +dependencies = [ + "serde", +] [[package]] name = "encoding_rs" @@ -390,23 +447,23 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "errno" -version = "0.3.2" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b30f669a7961ef1631673d2766cc92f52d64f7ef354d4fe0ddfd30ed52f0f4f" +checksum = "ac3e13f66a2f95e32a39eaa81f6b95d42878ca0e1db0c7543723dfe12557e860" dependencies = [ - "errno-dragonfly", "libc", "windows-sys", ] [[package]] -name = "errno-dragonfly" -version = "0.1.2" +name = "etcetera" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa68f1b12764fab894d2755d2518754e71b4fd80ecfb822714a1206c2aab39bf" +checksum = "136d1b5283a1ab77bd9257427ffd09d8667ced0570b6f938942bc7568ed5b943" dependencies = [ - "cc", - "libc", + "cfg-if", + "home", + "windows-sys", ] [[package]] @@ -423,9 +480,15 @@ checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" [[package]] name = "fastrand" -version = "2.0.0" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" + +[[package]] +name = "finl_unicode" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6999dc1837253364c2ebb0704ba97994bd874e8f195d665c50b7548f6ea92764" +checksum = "8fcfdc7a0362c9f4444381a9e697c79d435fe65b52a37466fc2c1184cee9edc6" [[package]] name = "fixedbitset" @@ -433,6 +496,17 @@ version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" +[[package]] +name = "flume" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55ac459de2512911e4b674ce33cf20befaba382d05b62b008afc1c8b57cbf181" +dependencies = [ + "futures-core", + "futures-sink", + "spin 0.9.8", +] + [[package]] name = "fnv" version = "1.0.7" @@ -463,21 +537,6 @@ dependencies = [ "percent-encoding", ] -[[package]] -name = "futures" -version = "0.3.28" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23342abe12aba583913b2e62f22225ff9c950774065e4bfb61a19cd9770fec40" -dependencies = [ - "futures-channel", - "futures-core", - "futures-executor", - "futures-io", - "futures-sink", - "futures-task", - "futures-util", -] - [[package]] name = "futures-channel" version = "0.3.28" @@ -507,13 +566,13 @@ dependencies = [ [[package]] name = "futures-intrusive" -version = "0.4.2" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a604f7a68fbf8103337523b1fadc8ade7361ee3f112f7c680ad179651616aed5" +checksum = "1d930c203dd0b6ff06e0201a4a2fe9149b43c684fd4420555b26d21b1a02956f" dependencies = [ "futures-core", "lock_api", - "parking_lot 0.11.2", + "parking_lot", ] [[package]] @@ -530,7 +589,7 @@ checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] @@ -551,7 +610,6 @@ version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533" dependencies = [ - "futures-channel", "futures-core", "futures-io", "futures-macro", @@ -586,15 +644,15 @@ dependencies = [ [[package]] name = "gimli" -version = "0.27.3" +version = "0.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6c80984affa11d98d1b88b66ac8853f143217b399d3c74116778ff8fdb4ed2e" +checksum = "6fb8d784f27acf97159b40fc4db5ecd8aa23b9ad5ef69cdd136d3bc80665f0c0" [[package]] name = "h2" -version = "0.3.20" +version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97ec8491ebaf99c8eaa73058b045fe58073cd6be7f596ac993ced0b0a0c01049" +checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" dependencies = [ "bytes", "fnv", @@ -609,15 +667,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "hashbrown" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab5ef0d4909ef3724cc8cce6ccc8572c5c817592e9285f5464f8e86f8bd3726e" -dependencies = [ - "ahash", -] - [[package]] name = "hashbrown" version = "0.12.3" @@ -629,14 +678,18 @@ name = "hashbrown" version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2c6201b9ff9fd90a5a3bac2e56a830d0caa509576f0e503818ee82c181b3437a" +dependencies = [ + "ahash", + "allocator-api2", +] [[package]] name = "hashlink" -version = "0.7.0" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7249a3129cbc1ffccd74857f81464a323a152173cdb134e0fd81bc803b29facf" +checksum = "e8094feaf31ff591f651a2664fb9cfd92bba7a60ce3197265e9482ebe753c8f7" dependencies = [ - "hashbrown 0.11.2", + "hashbrown 0.14.0", ] [[package]] @@ -650,18 +703,9 @@ dependencies = [ [[package]] name = "hermit-abi" -version = "0.1.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] - -[[package]] -name = "hermit-abi" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "443144c8cdadd93ebf52ddb4056d257f5b52c04d3c804e657d19eb73fc33668b" +checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" [[package]] name = "hex" @@ -687,6 +731,15 @@ dependencies = [ "digest", ] +[[package]] +name = "home" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5444c27eef6923071f7ebcc33e3444508466a76f7a2b93da00ed6e19f30c1ddb" +dependencies = [ + "windows-sys", +] + [[package]] name = "http" version = "0.2.9" @@ -709,12 +762,6 @@ dependencies = [ "pin-project-lite", ] -[[package]] -name = "http-range-header" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "add0ab9360ddbd88cfeb3bd9574a1d85cfdfa14db10b3e21d3700dbc4328758f" - [[package]] name = "httparse" version = "1.8.0" @@ -723,9 +770,9 @@ checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" [[package]] name = "httpdate" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a1e36c821dbe04574f602848a19f742f4fb3c98d40449f11bcad18d6b17421" +checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" [[package]] name = "hyper" @@ -806,15 +853,6 @@ dependencies = [ "hashbrown 0.14.0", ] -[[package]] -name = "instant" -version = "0.1.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" -dependencies = [ - "cfg-if", -] - [[package]] name = "ipnet" version = "2.8.0" @@ -823,9 +861,9 @@ checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" [[package]] name = "itertools" -version = "0.10.5" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" dependencies = [ "either", ] @@ -850,12 +888,32 @@ name = "lazy_static" version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +dependencies = [ + "spin 0.5.2", +] [[package]] name = "libc" -version = "0.2.147" +version = "0.2.149" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a08173bc88b7955d1b3145aa561539096c421ac8debde8cbc3612ec635fee29b" + +[[package]] +name = "libm" +version = "0.2.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058" + +[[package]] +name = "libsqlite3-sys" +version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4668fb0ea861c1df094127ac5f1da3409a82116a4ba74fca2e58ef927159bb3" +checksum = "afc22eff61b133b115c6e8c74e818c628d6d5e7a502afea6f64dee076dd94326" +dependencies = [ + "cc", + "pkg-config", + "vcpkg", +] [[package]] name = "libz-sys" @@ -871,9 +929,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.4.5" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57bcfdad1b858c2db7c38303a6d2ad4dfaf5eb53dfeb0910128b2c26d6158503" +checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" [[package]] name = "lock_api" @@ -887,15 +945,15 @@ dependencies = [ [[package]] name = "log" -version = "0.4.19" +version = "0.4.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" [[package]] name = "matchit" -version = "0.5.0" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73cbba799671b762df5a175adf59ce145165747bb891505c43d09aefbbf38beb" +checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" [[package]] name = "md-5" @@ -908,9 +966,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.5.0" +version = "2.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" +checksum = "f665ee40bc4a3c5590afb1e9677db74a508659dfd71e126420da8274909a0167" [[package]] name = "mime" @@ -978,13 +1036,52 @@ dependencies = [ "minimal-lexical", ] +[[package]] +name = "num-bigint-dig" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc84195820f291c7697304f3cbdadd1cb7199c0efc917ff5eafd71225c136151" +dependencies = [ + "byteorder", + "lazy_static", + "libm", + "num-integer", + "num-iter", + "num-traits", + "rand", + "smallvec", + "zeroize", +] + +[[package]] +name = "num-integer" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "225d3389fb3509a24c93f5c29eb6bde2586b98d9f016636dff58d7c6f7569cd9" +dependencies = [ + "autocfg", + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d03e6c028c5dc5cac6e2dec0efda81fc887605bb3d884578bb6d6bf7514e252" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + [[package]] name = "num-traits" -version = "0.2.16" +version = "0.2.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f30b0abd723be7e2ffca1272140fac1a2f084c77ec3e123c192b66af1ee9e6c2" +checksum = "39e3200413f237f41ab11ad6d161bc7239c84dcb631773ccd7de3dfe4b5c267c" dependencies = [ "autocfg", + "libm", ] [[package]] @@ -993,7 +1090,7 @@ version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi 0.3.2", + "hermit-abi", "libc", ] @@ -1020,9 +1117,9 @@ dependencies = [ [[package]] name = "object" -version = "0.31.1" +version = "0.32.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bda667d9f2b5051b8833f59f3bf748b28ef54f850f4fcb389a252aa383866d1" +checksum = "9cf5f9dd3933bd50a9e1f149ec995f39ae2c496d31fd772c1fd45ebc27e902b0" dependencies = [ "memchr", ] @@ -1035,11 +1132,11 @@ checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" [[package]] name = "openssl" -version = "0.10.56" +version = "0.10.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "729b745ad4a5575dd06a3e1af1414bd330ee561c01b3899eb584baeaa8def17e" +checksum = "bac25ee399abb46215765b1cb35bc0212377e58a061560d8b29b024fd0430e7c" dependencies = [ - "bitflags 1.3.2", + "bitflags 2.4.1", "cfg-if", "foreign-types", "libc", @@ -1056,7 +1153,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] @@ -1067,9 +1164,9 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "openssl-sys" -version = "0.9.91" +version = "0.9.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "866b5f16f90776b9bb8dc1e1802ac6f0513de3a7a7465867bfbc563dc737faac" +checksum = "db4d56a4c0478783083cfafcc42493dd4a981d41669da64b4572a2a089b51b1d" dependencies = [ "cc", "libc", @@ -1077,17 +1174,6 @@ dependencies = [ "vcpkg", ] -[[package]] -name = "parking_lot" -version = "0.11.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d17b78036a60663b797adeaee46f5c9dfebb86948d1255007a1d6be0271ff99" -dependencies = [ - "instant", - "lock_api", - "parking_lot_core 0.8.6", -] - [[package]] name = "parking_lot" version = "0.12.1" @@ -1095,21 +1181,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" dependencies = [ "lock_api", - "parking_lot_core 0.9.8", -] - -[[package]] -name = "parking_lot_core" -version = "0.8.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60a2cfe6f0ad2bfc16aefa463b497d5c7a5ecd44a23efa72aa342d90177356dc" -dependencies = [ - "cfg-if", - "instant", - "libc", - "redox_syscall 0.2.16", - "smallvec", - "winapi", + "parking_lot_core", ] [[package]] @@ -1120,7 +1192,7 @@ checksum = "93f00c865fe7cabf650081affecd3871070f26767e7b2070a3ffae14c654b447" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.3.5", + "redox_syscall", "smallvec", "windows-targets", ] @@ -1131,6 +1203,15 @@ version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" +[[package]] +name = "pem-rfc7468" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88b39c9bfcfc231068454382784bb460aae594343fb030d46e9f50a645418412" +dependencies = [ + "base64ct", +] + [[package]] name = "percent-encoding" version = "2.3.0" @@ -1139,12 +1220,12 @@ checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94" [[package]] name = "petgraph" -version = "0.6.3" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dd7d28ee937e54fe3080c91faa1c3a46c06de6252988a7f4592ba2310ef22a4" +checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" dependencies = [ "fixedbitset", - "indexmap 1.9.3", + "indexmap 2.0.0", ] [[package]] @@ -1182,14 +1263,14 @@ checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] name = "pin-project-lite" -version = "0.2.11" +version = "0.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c516611246607d0c04186886dbb3a754368ef82c79e9827a802c6d836dd111c" +checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" [[package]] name = "pin-utils" @@ -1197,6 +1278,27 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" +[[package]] +name = "pkcs1" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8ffb9f10fa047879315e6625af03c164b16962a5368d724ed16323b68ace47f" +dependencies = [ + "der", + "pkcs8", + "spki", +] + +[[package]] +name = "pkcs8" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" +dependencies = [ + "der", + "spki", +] + [[package]] name = "pkg-config" version = "0.3.27" @@ -1205,11 +1307,11 @@ checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" [[package]] name = "postgres-protocol" -version = "0.6.5" +version = "0.6.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78b7fa9f396f51dffd61546fd8573ee20592287996568e6175ceb0f8699ad75d" +checksum = "49b6c5ef183cd3ab4ba005f1ca64c21e8bd97ce4699cfea9e8d9a2c4958ca520" dependencies = [ - "base64 0.21.2", + "base64", "byteorder", "bytes", "fallible-iterator", @@ -1223,9 +1325,9 @@ dependencies = [ [[package]] name = "postgres-types" -version = "0.2.5" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f028f05971fe20f512bcc679e2c10227e57809a3af86a7606304435bc8896cd6" +checksum = "8d2234cdee9408b523530a9b6d2d6b373d1db34f6a8e51dc03ded1828d7fb67c" dependencies = [ "bytes", "fallible-iterator", @@ -1240,12 +1342,12 @@ checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "prettyplease" -version = "0.1.25" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8646e95016a7a6c4adea95bafa8a16baab64b583356217f2c85db4a39d9a86" +checksum = "ae005bd773ab59b4725093fd7df83fd7892f7d8eafb48dbd7de6e024e4215f9d" dependencies = [ "proc-macro2", - "syn 1.0.109", + "syn 2.0.38", ] [[package]] @@ -1260,18 +1362,18 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.66" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18fb31db3f9bddb2ea821cde30a9f70117e3f119938b5ee630b7403aa6e2ead9" +checksum = "134c189feb4956b20f6f547d2cf727d4c0fe06722b20a0eec87ed445a97f92da" dependencies = [ "unicode-ident", ] [[package]] name = "prost" -version = "0.10.4" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71adf41db68aa0daaefc69bb30bcd68ded9b9abaad5d1fbb6304c4fb390e083e" +checksum = "f4fdd22f3b9c31b53c060df4a0613a1c7f062d4115a2b984dd15b1858f7e340d" dependencies = [ "bytes", "prost-derive", @@ -1279,54 +1381,53 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.10.4" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ae5a4388762d5815a9fc0dea33c56b021cdc8dde0c55e0c9ca57197254b0cab" +checksum = "8bdf592881d821b83d471f8af290226c8d51402259e9bb5be7f9f8bdebbb11ac" dependencies = [ "bytes", - "cfg-if", - "cmake", "heck", "itertools", - "lazy_static", "log", "multimap", + "once_cell", "petgraph", + "prettyplease", "prost", "prost-types", "regex", + "syn 2.0.38", "tempfile", "which", ] [[package]] name = "prost-derive" -version = "0.10.1" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b670f45da57fb8542ebdbb6105a925fe571b67f9e7ed9f47a06a84e72b4e7cc" +checksum = "265baba7fabd416cf5078179f7d2cbeca4ce7a9041111900675ea7c4cb8a4c32" dependencies = [ "anyhow", "itertools", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.38", ] [[package]] name = "prost-types" -version = "0.10.1" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d0a014229361011dc8e69c8a1ec6c2e8d0f2af7c91e3ea3f5b2170298461e68" +checksum = "e081b29f63d83a4bc75cfc9f3fe424f9156cf92d8a4f0c9407cce9a1b67327cf" dependencies = [ - "bytes", "prost", ] [[package]] name = "quote" -version = "1.0.32" +version = "1.0.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50f3b39ccfb720540debaa0164757101c08ecb8d326b15358ce76a62c7e85965" +checksum = "5267fca4496028628a95160fc423a33e8b2e6af8a5302579e322e4b520293cae" dependencies = [ "proc-macro2", ] @@ -1363,11 +1464,12 @@ dependencies = [ [[package]] name = "rdkafka" -version = "0.25.0" +version = "0.34.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8acd8f5c5482fdf89e8878227bafa442d8c4409f6287391c85549ca83626c27" +checksum = "053adfa02fab06e86c01d586cc68aa47ee0ff4489a59469081dc12cbcde578bf" dependencies = [ - "futures", + "futures-channel", + "futures-util", "libc", "log", "rdkafka-sys", @@ -1380,25 +1482,17 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "3.0.0+1.6.0" +version = "4.6.0+2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca35e95c88e08cdc643b25744e38ccee7c93c7e90d1ac6850fe74cbaa40803c3" +checksum = "ad63c279fca41a27c231c450a2d2ad18288032e9cbb159ad16c9d96eba35aaaf" dependencies = [ + "cmake", "libc", "libz-sys", "num_enum", "pkg-config", ] -[[package]] -name = "redox_syscall" -version = "0.2.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb5a58c1855b4b6819d59012155603f0b22ad30cad752600aadfcb695265519a" -dependencies = [ - "bitflags 1.3.2", -] - [[package]] name = "redox_syscall" version = "0.3.5" @@ -1408,22 +1502,11 @@ dependencies = [ "bitflags 1.3.2", ] -[[package]] -name = "redox_users" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b033d837a7cf162d7993aded9304e30a83213c648b6e389db233191f891e5c2b" -dependencies = [ - "getrandom", - "redox_syscall 0.2.16", - "thiserror", -] - [[package]] name = "regex" -version = "1.9.3" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81bc1d4caf89fac26a70747fe603c130093b53c773888797a6329091246d651a" +checksum = "d119d7c7ca818f8a53c300863d4f87566aac09943aef5b355bb83969dae75d87" dependencies = [ "aho-corasick", "memchr", @@ -1433,9 +1516,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.3.6" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fed1ceff11a1dddaee50c9dc8e4938bd106e9d89ae372f192311e7da498e3b69" +checksum = "465c6fc0621e4abc4187a2bda0937bfd4f722c2730b29562e19689ea796c9a4b" dependencies = [ "aho-corasick", "memchr", @@ -1444,9 +1527,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.7.4" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" +checksum = "56d84fdd47036b038fc80dd333d10b6aab10d5d31f4a366e20014def75328d33" [[package]] name = "reqwest" @@ -1454,7 +1537,7 @@ version = "0.11.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cde824a14b7c14f85caff81225f411faacc04a2013f41670f41443742b1c1c55" dependencies = [ - "base64 0.21.2", + "base64", "bytes", "encoding_rs", "futures-core", @@ -1485,6 +1568,28 @@ dependencies = [ "winreg", ] +[[package]] +name = "rsa" +version = "0.9.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ab43bb47d23c1a631b4b680199a45255dce26fa9ab2fa902581f624ff13e6a8" +dependencies = [ + "byteorder", + "const-oid", + "digest", + "num-bigint-dig", + "num-integer", + "num-iter", + "num-traits", + "pkcs1", + "pkcs8", + "rand_core", + "signature", + "spki", + "subtle", + "zeroize", +] + [[package]] name = "rustc-demangle" version = "0.1.23" @@ -1493,17 +1598,23 @@ checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" [[package]] name = "rustix" -version = "0.38.7" +version = "0.38.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "172891ebdceb05aa0005f533a6cbfca599ddd7d966f6f5d4d9b2e70478e70399" +checksum = "745ecfa778e66b2b63c88a61cb36e0eea109e803b0b86bf9879fbc77c70e86ed" dependencies = [ - "bitflags 2.3.3", + "bitflags 2.4.1", "errno", "libc", "linux-raw-sys", "windows-sys", ] +[[package]] +name = "rustversion" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" + [[package]] name = "ryu" version = "1.0.15" @@ -1550,29 +1661,29 @@ dependencies = [ [[package]] name = "serde" -version = "1.0.183" +version = "1.0.189" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32ac8da02677876d532745a130fc9d8e6edfa81a269b107c5b00829b91d8eb3c" +checksum = "8e422a44e74ad4001bdc8eede9a4570ab52f71190e9c076d14369f38b9200537" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.183" +version = "1.0.189" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aafe972d60b0b9bee71a91b92fee2d4fb3c9d7e8f6b179aa99f27203d99a4816" +checksum = "1e48d1f918009ce3145511378cf68d613e3b3d9137d67272562080d68a2b32d5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] name = "serde_json" -version = "1.0.104" +version = "1.0.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "076066c5f1078eac5b722a31827a8832fe108bed65dfa75e233c89f8206e976c" +checksum = "6b420ce6e3d8bd882e9b243c6eed35dbc9a6110c9769e74b584e0d68d1f20c65" dependencies = [ "itoa", "ryu", @@ -1608,10 +1719,10 @@ dependencies = [ ] [[package]] -name = "sha-1" -version = "0.10.1" +name = "sha1" +version = "0.10.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5058ada175748e33390e40e872bd0fe59a19f265d0158daa551c5a88a76009c" +checksum = "e3bf829a2d51ab4a5ddf1352d8470c140cadc8301b2ae1789db023f01cedd6ba" dependencies = [ "cfg-if", "cpufeatures", @@ -1638,26 +1749,36 @@ dependencies = [ "libc", ] +[[package]] +name = "signature" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e1788eed21689f9cf370582dfc467ef36ed9c707f073528ddafa8d83e3b8500" +dependencies = [ + "digest", + "rand_core", +] + [[package]] name = "siphasher" -version = "0.3.10" +version = "0.3.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7bd3e3206899af3f8b12af284fafc038cc1dc2b41d1b89dd17297221c5d225de" +checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" [[package]] name = "slab" -version = "0.4.8" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6528351c9bc8ab22353f9d776db39a20288e8d6c37ef8cfe3317cf875eecfc2d" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" dependencies = [ "autocfg", ] [[package]] name = "smallvec" -version = "1.11.0" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62bb4feee49fdd9f707ef802e22365a35de4b7b299de4763d44bfea899442ff9" +checksum = "942b4a808e05215192e39f4ab80813e599068285906cc91aa64f923db842bd5a" [[package]] name = "socket2" @@ -1671,19 +1792,44 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.3" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2538b18701741680e0322a2302176d3253a35388e2e62f172f64f4f16605f877" +checksum = "4031e820eb552adee9295814c0ced9e5cf38ddf1e8b7d566d6de8e2538ea989e" dependencies = [ "libc", "windows-sys", ] +[[package]] +name = "spin" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" + +[[package]] +name = "spin" +version = "0.9.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" +dependencies = [ + "lock_api", +] + +[[package]] +name = "spki" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d1e996ef02c474957d681f1b05213dfb0abab947b446a62d37770b23500184a" +dependencies = [ + "base64ct", + "der", +] + [[package]] name = "sqlformat" -version = "0.1.8" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4b7922be017ee70900be125523f38bdd644f4f06a1b16e8fa5a8ee8c34bffd4" +checksum = "6b7b278788e7be4d0d29c0f39497a0eef3fba6bbc8e70d8bf7fde46edeaa9e85" dependencies = [ "itertools", "nom", @@ -1692,109 +1838,216 @@ dependencies = [ [[package]] name = "sqlx" -version = "0.5.13" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "551873805652ba0d912fec5bbb0f8b4cdd96baf8e2ebf5970e5671092966019b" +checksum = "0e50c216e3624ec8e7ecd14c6a6a6370aad6ee5d8cfc3ab30b5162eeeef2ed33" dependencies = [ "sqlx-core", "sqlx-macros", + "sqlx-mysql", + "sqlx-postgres", + "sqlx-sqlite", ] [[package]] name = "sqlx-core" -version = "0.5.13" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e48c61941ccf5ddcada342cd59e3e5173b007c509e1e8e990dafc830294d9dc5" +checksum = "8d6753e460c998bbd4cd8c6f0ed9a64346fcca0723d6e75e52fdc351c5d2169d" dependencies = [ "ahash", "atoi", - "base64 0.13.1", - "bitflags 1.3.2", "byteorder", "bytes", "crc", "crossbeam-queue", - "dirs", + "dotenvy", "either", "event-listener", "futures-channel", "futures-core", "futures-intrusive", + "futures-io", "futures-util", "hashlink", "hex", - "hkdf", - "hmac", - "indexmap 1.9.3", - "itoa", - "libc", + "indexmap 2.0.0", "log", - "md-5", "memchr", + "native-tls", "once_cell", "paste", "percent-encoding", - "rand", "serde", "serde_json", - "sha-1", "sha2", "smallvec", "sqlformat", - "sqlx-rt", - "stringprep", "thiserror", + "tokio", "tokio-stream", + "tracing", "url", - "whoami", ] [[package]] name = "sqlx-macros" -version = "0.5.13" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a793bb3ba331ec8359c1853bd39eed32cdd7baaf22c35ccf5c92a7e8d1189ec" +dependencies = [ + "proc-macro2", + "quote", + "sqlx-core", + "sqlx-macros-core", + "syn 1.0.109", +] + +[[package]] +name = "sqlx-macros-core" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc0fba2b0cae21fc00fe6046f8baa4c7fcb49e379f0f592b04696607f69ed2e1" +checksum = "0a4ee1e104e00dedb6aa5ffdd1343107b0a4702e862a84320ee7cc74782d96fc" dependencies = [ - "dotenv", + "dotenvy", "either", "heck", + "hex", "once_cell", "proc-macro2", "quote", + "serde", + "serde_json", "sha2", "sqlx-core", - "sqlx-rt", + "sqlx-mysql", + "sqlx-postgres", + "sqlx-sqlite", "syn 1.0.109", + "tempfile", + "tokio", "url", ] [[package]] -name = "sqlx-rt" -version = "0.5.13" +name = "sqlx-mysql" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4db708cd3e459078f85f39f96a00960bd841f66ee2a669e90bf36907f5a79aae" +checksum = "864b869fdf56263f4c95c45483191ea0af340f9f3e3e7b4d57a61c7c87a970db" dependencies = [ - "native-tls", + "atoi", + "base64", + "bitflags 2.4.1", + "byteorder", + "bytes", + "crc", + "digest", + "dotenvy", + "either", + "futures-channel", + "futures-core", + "futures-io", + "futures-util", + "generic-array", + "hex", + "hkdf", + "hmac", + "itoa", + "log", + "md-5", + "memchr", "once_cell", - "tokio", - "tokio-native-tls", + "percent-encoding", + "rand", + "rsa", + "serde", + "sha1", + "sha2", + "smallvec", + "sqlx-core", + "stringprep", + "thiserror", + "tracing", + "whoami", +] + +[[package]] +name = "sqlx-postgres" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eb7ae0e6a97fb3ba33b23ac2671a5ce6e3cabe003f451abd5a56e7951d975624" +dependencies = [ + "atoi", + "base64", + "bitflags 2.4.1", + "byteorder", + "crc", + "dotenvy", + "etcetera", + "futures-channel", + "futures-core", + "futures-io", + "futures-util", + "hex", + "hkdf", + "hmac", + "home", + "itoa", + "log", + "md-5", + "memchr", + "once_cell", + "rand", + "serde", + "serde_json", + "sha1", + "sha2", + "smallvec", + "sqlx-core", + "stringprep", + "thiserror", + "tracing", + "whoami", +] + +[[package]] +name = "sqlx-sqlite" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d59dc83cf45d89c555a577694534fcd1b55c545a816c816ce51f20bbe56a4f3f" +dependencies = [ + "atoi", + "flume", + "futures-channel", + "futures-core", + "futures-executor", + "futures-intrusive", + "futures-util", + "libsqlite3-sys", + "log", + "percent-encoding", + "serde", + "sqlx-core", + "tracing", + "url", ] [[package]] name = "stringprep" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db3737bde7edce97102e0e2b15365bf7a20bfdb5f60f4f9e8d7004258a51a8da" +checksum = "bb41d74e231a107a1b4ee36bd1214b11285b77768d2e3824aedafa988fd36ee6" dependencies = [ + "finl_unicode", "unicode-bidi", "unicode-normalization", ] [[package]] name = "strsim" -version = "0.8.0" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ea5119cdb4c55b55d432abb513a0429384878c15dde60cc77b1c99de1a95a6a" +checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "subtle" @@ -1815,9 +2068,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.28" +version = "2.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04361975b3f5e348b2189d8dc55bc942f278b2d482a6a0365de5bdd62d351567" +checksum = "e96b79aaa137db8f61e26363a0c9b47d8b4ec75da28b7d1d614c2303e232408b" dependencies = [ "proc-macro2", "quote", @@ -1832,44 +2085,35 @@ checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" [[package]] name = "tempfile" -version = "3.7.1" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc02fddf48964c42031a0b3fe0428320ecf3a73c401040fc0096f97794310651" +checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" dependencies = [ "cfg-if", "fastrand", - "redox_syscall 0.3.5", + "redox_syscall", "rustix", "windows-sys", ] -[[package]] -name = "textwrap" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d326610f408c7a4eb6f51c37c330e496b08506c9457c9d34287ecc38809fb060" -dependencies = [ - "unicode-width", -] - [[package]] name = "thiserror" -version = "1.0.44" +version = "1.0.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "611040a08a0439f8248d1990b111c95baa9c704c805fa1f62104b39655fd7f90" +checksum = "1177e8c6d7ede7afde3585fd2513e611227efd6481bd78d2e82ba1ce16557ed4" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.44" +version = "1.0.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "090198534930841fab3a5d1bb637cde49e339654e606195f8d9c76eeb081dc96" +checksum = "10712f02019e9288794769fba95cd6847df9874d49d871d062172f9dd41bc4cc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] @@ -1889,20 +2133,19 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.29.1" +version = "1.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "532826ff75199d5833b9d2c5fe410f29235e25704ee5f0ef599fb51c21f4a4da" +checksum = "4f38200e3ef7995e5ef13baec2f432a6da0aa9ac495b2c0e8f3b7eec2c92d653" dependencies = [ - "autocfg", "backtrace", "bytes", "libc", "mio", "num_cpus", - "parking_lot 0.12.1", + "parking_lot", "pin-project-lite", "signal-hook-registry", - "socket2 0.4.9", + "socket2 0.5.4", "tokio-macros", "windows-sys", ] @@ -1925,7 +2168,7 @@ checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] @@ -1940,9 +2183,9 @@ dependencies = [ [[package]] name = "tokio-postgres" -version = "0.7.8" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e89f6234aa8fd43779746012fcf53603cdb91fdd8399aa0de868c2d56b6dde1" +checksum = "d340244b32d920260ae7448cb72b6e238bddc3d4f7603394e7dd46ed8e48f5b8" dependencies = [ "async-trait", "byteorder", @@ -1951,15 +2194,17 @@ dependencies = [ "futures-channel", "futures-util", "log", - "parking_lot 0.12.1", + "parking_lot", "percent-encoding", "phf", "pin-project-lite", "postgres-protocol", "postgres-types", - "socket2 0.5.3", + "rand", + "socket2 0.5.4", "tokio", "tokio-util", + "whoami", ] [[package]] @@ -1975,9 +2220,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.8" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "806fe8c2c87eccc8b3267cbae29ed3ab2d0bd37fca70ab622e46aaa9375ddb7d" +checksum = "1d68074620f57a0b21594d9735eb2e98ab38b17f80d3fcb189fca266771ca60d" dependencies = [ "bytes", "futures-core", @@ -1995,9 +2240,9 @@ checksum = "7cda73e2f1397b1262d6dfdcef8aafae14d1de7748d66822d3bfeeb6d03e5e4b" [[package]] name = "toml_edit" -version = "0.19.14" +version = "0.19.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8123f27e969974a3dfba720fdb560be359f57b44302d280ba72e76a74480e8a" +checksum = "1b5bb770da30e5cbfde35a2d7b9b8a2c4b8ef89548a7a6aeab5c9a576e3e7421" dependencies = [ "indexmap 2.0.0", "toml_datetime", @@ -2006,17 +2251,15 @@ dependencies = [ [[package]] name = "tonic" -version = "0.7.2" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5be9d60db39854b30b835107500cf0aca0b0d14d6e1c3de124217c23a29c2ddb" +checksum = "d560933a0de61cf715926b9cac824d4c883c2c43142f787595e48280c40a1d0e" dependencies = [ "async-stream", "async-trait", "axum", - "base64 0.13.1", + "base64", "bytes", - "futures-core", - "futures-util", "h2", "http", "http-body", @@ -2025,28 +2268,25 @@ dependencies = [ "percent-encoding", "pin-project", "prost", - "prost-derive", "tokio", "tokio-stream", - "tokio-util", "tower", "tower-layer", "tower-service", "tracing", - "tracing-futures", ] [[package]] name = "tonic-build" -version = "0.7.2" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9263bf4c9bfaae7317c1c2faf7f18491d2fe476f70c414b73bf5d445b00ffa1" +checksum = "9d021fc044c18582b9a2408cd0dd05b1596e3ecdb5c4df822bb0183545683889" dependencies = [ "prettyplease", "proc-macro2", "prost-build", "quote", - "syn 1.0.109", + "syn 2.0.38", ] [[package]] @@ -2069,25 +2309,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "tower-http" -version = "0.3.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f873044bf02dd1e8239e9c1293ea39dad76dc594ec16185d0a1bf31d8dc8d858" -dependencies = [ - "bitflags 1.3.2", - "bytes", - "futures-core", - "futures-util", - "http", - "http-body", - "http-range-header", - "pin-project-lite", - "tower", - "tower-layer", - "tower-service", -] - [[package]] name = "tower-layer" version = "0.3.2" @@ -2102,11 +2323,10 @@ checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" -version = "0.1.37" +version = "0.1.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +checksum = "ee2ef2af84856a50c1d430afce2fdded0a4ec7eda868db86409b4543df0797f9" dependencies = [ - "cfg-if", "log", "pin-project-lite", "tracing-attributes", @@ -2115,34 +2335,24 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", ] [[package]] name = "tracing-core" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" dependencies = [ "once_cell", ] -[[package]] -name = "tracing-futures" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97d095ae15e245a057c8e8451bab9b3ee1e1f68e9ba2b4fbc18d0ac5237835f2" -dependencies = [ - "pin-project", - "tracing", -] - [[package]] name = "try-lock" version = "0.2.4" @@ -2151,9 +2361,9 @@ checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" [[package]] name = "typenum" -version = "1.16.0" +version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "497961ef93d974e23eb6f433eb5fe1b7930b659f06d12dec6fc44a8f554c0bba" +checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-bidi" @@ -2163,9 +2373,9 @@ checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" [[package]] name = "unicode-ident" -version = "1.0.11" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "301abaae475aa91687eb82514b328ab47a211a533026cb25fc3e519b86adfc3c" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" [[package]] name = "unicode-normalization" @@ -2182,12 +2392,6 @@ version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1dd624098567895118886609431a7c3b8f516e41d30e0643f03d94592a147e36" -[[package]] -name = "unicode-width" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" - [[package]] name = "unicode_categories" version = "0.1.1" @@ -2196,9 +2400,9 @@ checksum = "39ec24b3121d976906ece63c9daad25b85969647682eee313cb5779fdd69e14e" [[package]] name = "url" -version = "2.4.0" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50bff7831e19200a85b17131d085c25d7811bc4e186efdaf54bbd132994a88cb" +checksum = "143b538f18257fac9cad154828a57c6bf5157e1aa604d4816b5995bf6de87ae5" dependencies = [ "form_urlencoded", "idna", @@ -2206,16 +2410,16 @@ dependencies = [ ] [[package]] -name = "vcpkg" -version = "0.2.15" +name = "utf8parse" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" +checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] -name = "vec_map" -version = "0.8.2" +name = "vcpkg" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1bddf1187be692e79c5ffeab891132dfb0f236ed36a43c7ed39f1165ee20191" +checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" [[package]] name = "version_check" @@ -2259,7 +2463,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", "wasm-bindgen-shared", ] @@ -2293,7 +2497,7 @@ checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.28", + "syn 2.0.38", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -2316,13 +2520,14 @@ dependencies = [ [[package]] name = "which" -version = "4.4.0" +version = "4.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2441c784c52b289a054b7201fc93253e288f094e2f4be9058343127c4226a269" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" dependencies = [ "either", - "libc", + "home", "once_cell", + "rustix", ] [[package]] @@ -2368,9 +2573,9 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.48.1" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05d4b17490f70499f20b9e791dcf6a299785ce8af4d709018206dc5b4953e95f" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" dependencies = [ "windows_aarch64_gnullvm", "windows_aarch64_msvc", @@ -2383,51 +2588,51 @@ dependencies = [ [[package]] name = "windows_aarch64_gnullvm" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91ae572e1b79dba883e0d315474df7305d12f569b400fcf90581b06062f7e1bc" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_msvc" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2ef27e0d7bdfcfc7b868b317c1d32c641a6fe4629c171b8928c7b08d98d7cf3" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_i686_gnu" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "622a1962a7db830d6fd0a69683c80a18fda201879f0f447f065a3b7467daa241" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_msvc" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4542c6e364ce21bf45d69fdd2a8e455fa38d316158cfd43b3ac1c5b1b19f8e00" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_x86_64_gnu" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca2b8a661f7628cbd23440e50b05d705db3686f894fc9580820623656af974b1" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnullvm" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7896dbc1f41e08872e9d5e8f8baa8fdd2677f29468c4e156210174edc7f7b953" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_msvc" -version = "0.48.0" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "winnow" -version = "0.5.4" +version = "0.5.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acaaa1190073b2b101e15083c38ee8ec891b5e05cbee516521e94ec008f61e64" +checksum = "7c2e3184b9c4e92ad5167ca73039d0c42476302ab603e2fec4487511f38ccefc" dependencies = [ "memchr", ] @@ -2440,3 +2645,9 @@ checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d" dependencies = [ "winapi", ] + +[[package]] +name = "zeroize" +version = "1.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2a0956f1ba7c7909bfb66c2e9e4124ab6f6482560f6628b5aaeba39207c9aad9" diff --git a/integration_tests/feature-store/server/Cargo.toml b/integration_tests/feature-store/server/Cargo.toml index 123f089f5e7a3..ce43a3f8af506 100644 --- a/integration_tests/feature-store/server/Cargo.toml +++ b/integration_tests/feature-store/server/Cargo.toml @@ -9,23 +9,20 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -sqlx = { version = "0.5", features = [ "runtime-tokio-native-tls" , "postgres" ] } +sqlx = { version = "0.7", features = ["runtime-tokio-native-tls", "postgres"] } tokio = { version = "1", features = ["full"] } -tonic = "0.7.1" +tonic = "0.10.2" reqwest = { version = "0.11", features = ["blocking"] } -rdkafka = { version = "0.25", features = ["dynamic-linking"] } +rdkafka = { version = "0.34", features = ["cmake-build"] } serde_json = "1.0" -prost = "0.10" -clap = "2.26.0" -tokio-postgres = "0.7.8" -tonic-build = "0.7.1" +prost = "0.12" +clap = "4.4.6" +tokio-postgres = "0.7.10" +tonic-build = "0.10.2" [build-dependencies] -tonic-build = "0.7.1" +tonic-build = "0.10.2" [[bin]] name = "server" path = "src/main.rs" - -[lints] -workspace = true diff --git a/integration_tests/feature-store/server/model/requirements.txt b/integration_tests/feature-store/server/model/requirements.txt index bc40c0ead3fbe..276361b0e1f83 100644 --- a/integration_tests/feature-store/server/model/requirements.txt +++ b/integration_tests/feature-store/server/model/requirements.txt @@ -1,6 +1,6 @@ -grpcio==1.48.0 -numpy==1.21.4 -protobuf==4.21.5 +grpcio==1.53.0 +numpy==1.24 +protobuf==4.21.6 psycopg==3.0.16 scikit-learn==1.3.0 -pandas==1.4.0 +pandas==2.0 diff --git a/integration_tests/feature-store/server/src/feature_store.rs b/integration_tests/feature-store/server/src/feature_store.rs index 883a7c332767d..30bb92debe5aa 100644 --- a/integration_tests/feature-store/server/src/feature_store.rs +++ b/integration_tests/feature-store/server/src/feature_store.rs @@ -57,7 +57,13 @@ impl Server for FeatureStoreServer { request: tonic::Request, ) -> Result, tonic::Status> { let do_location_id = request.into_inner(); - let fare_amount = self.get_taxi_amount(do_location_id.do_location_id.clone(),do_location_id.pu_location_id.clone()).await.unwrap(); + let fare_amount = self + .get_taxi_amount( + do_location_id.do_location_id.clone(), + do_location_id.pu_location_id.clone(), + ) + .await + .unwrap(); Ok(Response::new(GetTaxiAmountResponse { fare_amount: fare_amount as f64, })) diff --git a/integration_tests/feature-store/server/src/main.rs b/integration_tests/feature-store/server/src/main.rs index adab174aaa227..73fad5122c40f 100644 --- a/integration_tests/feature-store/server/src/main.rs +++ b/integration_tests/feature-store/server/src/main.rs @@ -1,6 +1,6 @@ use std::net::{IpAddr, Ipv4Addr, SocketAddr}; -use clap::{App, Arg, ArgMatches}; +use clap::{Arg, ArgMatches, Command}; use crate::feature_store::FeatureStoreServer; use crate::kafka::KafkaSink; @@ -17,14 +17,14 @@ async fn main() { println!("Reading args"); let args = get_args(); let kafka_sink = KafkaSink::new( - args.value_of("brokers") + args.get_one::("brokers") .expect("failed to decode brokers") .to_string(), - args.value_of("output-topic") + args.get_one::("output-topic") .expect("failed to decode output_topics") .to_string(), ); - println!("Testing Kafka payload,args{:?}",args); + println!("Testing Kafka payload,args{:?}", args); tokio::spawn(KafkaSink::mock_consume()); kafka_sink .send("0".to_string(), "{init: true}".to_string()) @@ -41,23 +41,23 @@ async fn main() { .unwrap() } -fn get_args<'a>() -> ArgMatches<'a> { - App::new("feature-store") +fn get_args() -> ArgMatches { + Command::new("feature-store") .about("Feature store") .arg( - Arg::with_name("brokers") - .short("b") + Arg::new("brokers") + .short('b') .long("brokers") .help("Kafka broker list") - .takes_value(true) + .num_args(1) .default_value("kafka:9092"), ) .arg( - Arg::with_name("output-topic") + Arg::new("output-topic") .long("output-topics") .help("Output topics names") .default_value("taxi") - .takes_value(true), + .num_args(1), ) .get_matches() } diff --git a/integration_tests/feature-store/server/src/model.rs b/integration_tests/feature-store/server/src/model.rs index 2b141fafce0d1..4e7ae9464ca4e 100644 --- a/integration_tests/feature-store/server/src/model.rs +++ b/integration_tests/feature-store/server/src/model.rs @@ -1,24 +1,27 @@ +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct TrainingRequest { -} +pub struct TrainingRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct TrainingResponse { -} +pub struct TrainingResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetAmountRequest { - #[prost(int64, tag="1")] + #[prost(int64, tag = "1")] pub do_location_id: i64, - #[prost(int64, tag="2")] + #[prost(int64, tag = "2")] pub pu_location_id: i64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetAmountResponse { - #[prost(float, tag="1")] + #[prost(float, tag = "1")] pub amount: f32, } /// Generated client implementations. pub mod model_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::http::Uri; use tonic::codegen::*; #[derive(Debug, Clone)] pub struct ModelClient { @@ -28,7 +31,7 @@ pub mod model_client { /// Attempt to create a new client by connecting to a given endpoint. pub async fn connect(dst: D) -> Result where - D: std::convert::TryInto, + D: TryInto, D::Error: Into, { let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; @@ -46,6 +49,12 @@ pub mod model_client { let inner = tonic::client::Grpc::new(inner); Self { inner } } + + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( inner: T, interceptor: F, @@ -59,60 +68,81 @@ pub mod model_client { >::ResponseBody, >, >, - , - >>::Error: Into + Send + Sync, + >>::Error: + Into + Send + Sync, { ModelClient::new(InterceptedService::new(inner, interceptor)) } - /// Compress requests with `gzip`. + + /// Compress requests with the given encoding. /// /// This requires the server to support it otherwise it might respond with an /// error. #[must_use] - pub fn send_gzip(mut self) -> Self { - self.inner = self.inner.send_gzip(); + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); self } - /// Enable decompressing responses with `gzip`. + + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` #[must_use] - pub fn accept_gzip(mut self) -> Self { - self.inner = self.inner.accept_gzip(); + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); self } + + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn get_amount( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static("/model.Model/GetAmount"); - self.inner.unary(request.into_request(), path, codec).await + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("model.Model", "GetAmount")); + self.inner.unary(req, path, codec).await } + pub async fn training( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static("/model.Model/Training"); - self.inner.unary(request.into_request(), path, codec).await + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("model.Model", "Training")); + self.inner.unary(req, path, codec).await } } } @@ -120,46 +150,81 @@ pub mod model_client { pub mod model_server { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] use tonic::codegen::*; - ///Generated trait containing gRPC methods that should be implemented for use with ModelServer. + /// Generated trait containing gRPC methods that should be implemented for use with ModelServer. #[async_trait] pub trait Model: Send + Sync + 'static { async fn get_amount( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn training( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; } #[derive(Debug)] pub struct ModelServer { inner: _Inner, - accept_compression_encodings: (), - send_compression_encodings: (), + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, } struct _Inner(Arc); impl ModelServer { pub fn new(inner: T) -> Self { Self::from_arc(Arc::new(inner)) } + pub fn from_arc(inner: Arc) -> Self { let inner = _Inner(inner); Self { inner, accept_compression_encodings: Default::default(), send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, } } - pub fn with_interceptor( - inner: T, - interceptor: F, - ) -> InterceptedService + + pub fn with_interceptor(inner: T, interceptor: F) -> InterceptedService where F: tonic::service::Interceptor, { InterceptedService::new(Self::new(inner), interceptor) } + + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } } impl tonic::codegen::Service> for ModelServer where @@ -167,39 +232,41 @@ pub mod model_server { B: Body + Send + 'static, B::Error: Into + Send + 'static, { - type Response = http::Response; type Error = std::convert::Infallible; type Future = BoxFuture; + type Response = http::Response; + fn poll_ready( &mut self, _cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll> { Poll::Ready(Ok(())) } + fn call(&mut self, req: http::Request) -> Self::Future { let inner = self.inner.clone(); match req.uri().path() { "/model.Model/GetAmount" => { #[allow(non_camel_case_types)] struct GetAmountSvc(pub Arc); - impl tonic::server::UnaryService - for GetAmountSvc { + impl tonic::server::UnaryService for GetAmountSvc { + type Future = BoxFuture, tonic::Status>; type Response = super::GetAmountResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { (*inner).get_amount(request).await }; + let inner = Arc::clone(&self.0); + let fut = + async move { ::get_amount(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -209,6 +276,10 @@ pub mod model_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -218,24 +289,23 @@ pub mod model_server { "/model.Model/Training" => { #[allow(non_camel_case_types)] struct TrainingSvc(pub Arc); - impl tonic::server::UnaryService - for TrainingSvc { + impl tonic::server::UnaryService for TrainingSvc { + type Future = BoxFuture, tonic::Status>; type Response = super::TrainingResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { (*inner).training(request).await }; + let inner = Arc::clone(&self.0); + let fut = async move { ::training(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -245,24 +315,24 @@ pub mod model_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) }; Box::pin(fut) } - _ => { - Box::pin(async move { - Ok( - http::Response::builder() - .status(200) - .header("grpc-status", "12") - .header("content-type", "application/grpc") - .body(empty_body()) - .unwrap(), - ) - }) - } + _ => Box::pin(async move { + Ok(http::Response::builder() + .status(200) + .header("grpc-status", "12") + .header("content-type", "application/grpc") + .body(empty_body()) + .unwrap()) + }), } } } @@ -273,12 +343,14 @@ pub mod model_server { inner, accept_compression_encodings: self.accept_compression_encodings, send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, } } } impl Clone for _Inner { fn clone(&self) -> Self { - Self(self.0.clone()) + Self(Arc::clone(&self.0)) } } impl std::fmt::Debug for _Inner { @@ -286,7 +358,7 @@ pub mod model_server { write!(f, "{:?}", self.0) } } - impl tonic::transport::NamedService for ModelServer { + impl tonic::server::NamedService for ModelServer { const NAME: &'static str = "model.Model"; } } diff --git a/integration_tests/feature-store/server/src/server_pb.rs b/integration_tests/feature-store/server/src/server_pb.rs index 1f2912d868412..e4e25439fb29d 100644 --- a/integration_tests/feature-store/server/src/server_pb.rs +++ b/integration_tests/feature-store/server/src/server_pb.rs @@ -1,96 +1,104 @@ +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReportActionRequest { - #[prost(string, tag="1")] + #[prost(string, tag = "1")] pub userid: ::prost::alloc::string::String, - #[prost(string, tag="2")] + #[prost(string, tag = "2")] pub eventtype: ::prost::alloc::string::String, - #[prost(int64, tag="3")] + #[prost(int64, tag = "3")] pub changenum: i64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReportActionResponse { - #[prost(uint64, tag="1")] + #[prost(uint64, tag = "1")] pub timestamp: u64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetFeatureRequest { - #[prost(string, tag="1")] + #[prost(string, tag = "1")] pub userid: ::prost::alloc::string::String, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetFeatureResponse { - #[prost(uint64, tag="1")] + #[prost(uint64, tag = "1")] pub count: u64, - #[prost(int64, tag="2")] + #[prost(int64, tag = "2")] pub sum: i64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReportTaxiActionRequest { - #[prost(int32, tag="1")] + #[prost(int32, tag = "1")] pub vendor_id: i32, - #[prost(string, tag="2")] + #[prost(string, tag = "2")] pub lpep_pickup_datetime: ::prost::alloc::string::String, - #[prost(string, tag="3")] + #[prost(string, tag = "3")] pub lpep_dropoff_datetime: ::prost::alloc::string::String, - #[prost(bool, tag="4")] + #[prost(bool, tag = "4")] pub store_and_fwd_flag: bool, - #[prost(double, tag="5")] + #[prost(double, tag = "5")] pub ratecode_id: f64, - #[prost(int64, tag="6")] + #[prost(int64, tag = "6")] pub pu_location_id: i64, - #[prost(int64, tag="7")] + #[prost(int64, tag = "7")] pub do_location_id: i64, - #[prost(double, tag="8")] + #[prost(double, tag = "8")] pub passenger_count: f64, - #[prost(double, tag="9")] + #[prost(double, tag = "9")] pub trip_distance: f64, - #[prost(double, tag="10")] + #[prost(double, tag = "10")] pub fare_amount: f64, - #[prost(double, tag="11")] + #[prost(double, tag = "11")] pub extra: f64, - #[prost(double, tag="12")] + #[prost(double, tag = "12")] pub mta_tax: f64, - #[prost(double, tag="13")] + #[prost(double, tag = "13")] pub tip_amount: f64, - #[prost(double, tag="14")] + #[prost(double, tag = "14")] pub tolls_amount: f64, - #[prost(double, tag="15")] + #[prost(double, tag = "15")] pub ehail_fee: f64, - #[prost(double, tag="16")] + #[prost(double, tag = "16")] pub improvement_surcharge: f64, - #[prost(double, tag="17")] + #[prost(double, tag = "17")] pub total_amount: f64, - #[prost(double, tag="18")] + #[prost(double, tag = "18")] pub payment_type: f64, - #[prost(double, tag="19")] + #[prost(double, tag = "19")] pub trip_type: f64, - #[prost(double, tag="20")] + #[prost(double, tag = "20")] pub congestion_surcharge: f64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct ReportTaxiActionResponse { -} +pub struct ReportTaxiActionResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetTaxiAmountRequest { - #[prost(int64, tag="1")] + #[prost(int64, tag = "1")] pub do_location_id: i64, - #[prost(int64, tag="2")] + #[prost(int64, tag = "2")] pub pu_location_id: i64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetTaxiAmountResponse { - #[prost(double, tag="1")] + #[prost(double, tag = "1")] pub fare_amount: f64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct StartTrainingRequest { -} +pub struct StartTrainingRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct StartTrainingResponse { -} +pub struct StartTrainingResponse {} /// Generated client implementations. pub mod server_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::http::Uri; use tonic::codegen::*; #[derive(Debug, Clone)] pub struct ServerClient { @@ -100,7 +108,7 @@ pub mod server_client { /// Attempt to create a new client by connecting to a given endpoint. pub async fn connect(dst: D) -> Result where - D: std::convert::TryInto, + D: TryInto, D::Error: Into, { let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; @@ -118,6 +126,12 @@ pub mod server_client { let inner = tonic::client::Grpc::new(inner); Self { inner } } + + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( inner: T, interceptor: F, @@ -131,121 +145,140 @@ pub mod server_client { >::ResponseBody, >, >, - , - >>::Error: Into + Send + Sync, + >>::Error: + Into + Send + Sync, { ServerClient::new(InterceptedService::new(inner, interceptor)) } - /// Compress requests with `gzip`. + + /// Compress requests with the given encoding. /// /// This requires the server to support it otherwise it might respond with an /// error. #[must_use] - pub fn send_gzip(mut self) -> Self { - self.inner = self.inner.send_gzip(); + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); self } - /// Enable decompressing responses with `gzip`. + + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` #[must_use] - pub fn accept_gzip(mut self) -> Self { - self.inner = self.inner.accept_gzip(); + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); self } + pub async fn get_feature( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/GetFeature", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/GetFeature"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "GetFeature")); + self.inner.unary(req, path, codec).await } + pub async fn report_action( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/ReportAction", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/ReportAction"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "ReportAction")); + self.inner.unary(req, path, codec).await } + pub async fn report_taxi_action( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/ReportTaxiAction", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/ReportTaxiAction"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "ReportTaxiAction")); + self.inner.unary(req, path, codec).await } + pub async fn get_taxi_amount( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/GetTaxiAmount", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/GetTaxiAmount"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "GetTaxiAmount")); + self.inner.unary(req, path, codec).await } + pub async fn start_training( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/StartTraining", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/StartTraining"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "StartTraining")); + self.inner.unary(req, path, codec).await } } } @@ -253,58 +286,93 @@ pub mod server_client { pub mod server_server { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] use tonic::codegen::*; - ///Generated trait containing gRPC methods that should be implemented for use with ServerServer. + /// Generated trait containing gRPC methods that should be implemented for use with ServerServer. #[async_trait] pub trait Server: Send + Sync + 'static { async fn get_feature( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn report_action( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn report_taxi_action( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn get_taxi_amount( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn start_training( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; } #[derive(Debug)] pub struct ServerServer { inner: _Inner, - accept_compression_encodings: (), - send_compression_encodings: (), + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, } struct _Inner(Arc); impl ServerServer { pub fn new(inner: T) -> Self { Self::from_arc(Arc::new(inner)) } + pub fn from_arc(inner: Arc) -> Self { let inner = _Inner(inner); Self { inner, accept_compression_encodings: Default::default(), send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, } } - pub fn with_interceptor( - inner: T, - interceptor: F, - ) -> InterceptedService + + pub fn with_interceptor(inner: T, interceptor: F) -> InterceptedService where F: tonic::service::Interceptor, { InterceptedService::new(Self::new(inner), interceptor) } + + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } } impl tonic::codegen::Service> for ServerServer where @@ -312,39 +380,41 @@ pub mod server_server { B: Body + Send + 'static, B::Error: Into + Send + 'static, { - type Response = http::Response; type Error = std::convert::Infallible; type Future = BoxFuture; + type Response = http::Response; + fn poll_ready( &mut self, _cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll> { Poll::Ready(Ok(())) } + fn call(&mut self, req: http::Request) -> Self::Future { let inner = self.inner.clone(); match req.uri().path() { "/server_pb.Server/GetFeature" => { #[allow(non_camel_case_types)] struct GetFeatureSvc(pub Arc); - impl tonic::server::UnaryService - for GetFeatureSvc { + impl tonic::server::UnaryService for GetFeatureSvc { + type Future = BoxFuture, tonic::Status>; type Response = super::GetFeatureResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { (*inner).get_feature(request).await }; + let inner = Arc::clone(&self.0); + let fut = + async move { ::get_feature(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -354,6 +424,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -363,28 +437,24 @@ pub mod server_server { "/server_pb.Server/ReportAction" => { #[allow(non_camel_case_types)] struct ReportActionSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for ReportActionSvc { + impl tonic::server::UnaryService for ReportActionSvc { + type Future = BoxFuture, tonic::Status>; type Response = super::ReportActionResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { - (*inner).report_action(request).await - }; + let inner = Arc::clone(&self.0); + let fut = + async move { ::report_action(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -394,6 +464,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -403,28 +477,27 @@ pub mod server_server { "/server_pb.Server/ReportTaxiAction" => { #[allow(non_camel_case_types)] struct ReportTaxiActionSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for ReportTaxiActionSvc { + impl tonic::server::UnaryService + for ReportTaxiActionSvc + { + type Future = BoxFuture, tonic::Status>; type Response = super::ReportTaxiActionResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); + let inner = Arc::clone(&self.0); let fut = async move { - (*inner).report_taxi_action(request).await + ::report_taxi_action(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -434,6 +507,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -443,28 +520,25 @@ pub mod server_server { "/server_pb.Server/GetTaxiAmount" => { #[allow(non_camel_case_types)] struct GetTaxiAmountSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for GetTaxiAmountSvc { + impl tonic::server::UnaryService for GetTaxiAmountSvc { + type Future = BoxFuture, tonic::Status>; type Response = super::GetTaxiAmountResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); + let inner = Arc::clone(&self.0); let fut = async move { - (*inner).get_taxi_amount(request).await + ::get_taxi_amount(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -474,6 +548,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -483,28 +561,24 @@ pub mod server_server { "/server_pb.Server/StartTraining" => { #[allow(non_camel_case_types)] struct StartTrainingSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for StartTrainingSvc { + impl tonic::server::UnaryService for StartTrainingSvc { + type Future = BoxFuture, tonic::Status>; type Response = super::StartTrainingResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { - (*inner).start_training(request).await - }; + let inner = Arc::clone(&self.0); + let fut = + async move { ::start_training(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -514,24 +588,24 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) }; Box::pin(fut) } - _ => { - Box::pin(async move { - Ok( - http::Response::builder() - .status(200) - .header("grpc-status", "12") - .header("content-type", "application/grpc") - .body(empty_body()) - .unwrap(), - ) - }) - } + _ => Box::pin(async move { + Ok(http::Response::builder() + .status(200) + .header("grpc-status", "12") + .header("content-type", "application/grpc") + .body(empty_body()) + .unwrap()) + }), } } } @@ -542,12 +616,14 @@ pub mod server_server { inner, accept_compression_encodings: self.accept_compression_encodings, send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, } } } impl Clone for _Inner { fn clone(&self) -> Self { - Self(self.0.clone()) + Self(Arc::clone(&self.0)) } } impl std::fmt::Debug for _Inner { @@ -555,7 +631,7 @@ pub mod server_server { write!(f, "{:?}", self.0) } } - impl tonic::transport::NamedService for ServerServer { + impl tonic::server::NamedService for ServerServer { const NAME: &'static str = "server_pb.Server"; } } diff --git a/integration_tests/feature-store/server/src/serving.rs b/integration_tests/feature-store/server/src/serving.rs index c00d8041492c0..d1137e534e7f9 100644 --- a/integration_tests/feature-store/server/src/serving.rs +++ b/integration_tests/feature-store/server/src/serving.rs @@ -53,8 +53,15 @@ impl FeatureStoreServer { } } - pub async fn get_taxi_amount(&self, do_location_id: i64,pu_location_id: i64) -> Result { - let request = GetAmountRequest { do_location_id ,pu_location_id}; + pub async fn get_taxi_amount( + &self, + do_location_id: i64, + pu_location_id: i64, + ) -> Result { + let request = GetAmountRequest { + do_location_id, + pu_location_id, + }; let mut model_client = ModelClient::connect("http://localhost:8080") .await .expect("Failed to connect to model server"); diff --git a/integration_tests/feature-store/simulator/Cargo.lock b/integration_tests/feature-store/simulator/Cargo.lock index c2be1809ce1bd..9d7b0fc4a1f82 100644 --- a/integration_tests/feature-store/simulator/Cargo.lock +++ b/integration_tests/feature-store/simulator/Cargo.lock @@ -3,35 +3,90 @@ version = 3 [[package]] -name = "ansi_term" -version = "0.12.1" +name = "addr2line" +version = "0.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d52a9bb7ec0cf484c551830a7ce27bd20d67eac647e1befb56b0be4ee39a55d2" +checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" dependencies = [ - "winapi", + "gimli", +] + +[[package]] +name = "adler" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" + +[[package]] +name = "anstream" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ab91ebe16eb252986481c5b62f6098f3b698a45e34b5b98200cf20dd2484a44" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7079075b41f533b8c61d2a4d073c4676e1f8b249ff94a393b0595db304e0dd87" + +[[package]] +name = "anstyle-parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "317b9a89c1868f5ea6ff1d9539a69f45dffc21ce321ac1fd1160dfa48c8e2140" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ca11d4be1bab0c8bc8734a9aa7bf4ee8316d462a08c6ac5052f888fef5b494b" +dependencies = [ + "windows-sys", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0699d10d2f4d628a98ee7b57b289abbc98ff3bad977cb3152709d4bf2330628" +dependencies = [ + "anstyle", + "windows-sys", ] [[package]] name = "anyhow" -version = "1.0.58" +version = "1.0.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb07d2053ccdbe10e2af2995a2f116c1330396493dc1269f6a91d0ae82e19704" +checksum = "a4668cab20f66d8d020e1fbc0ebe47217433c1b6c8f2040faf858554e394ace6" [[package]] name = "async-stream" -version = "0.3.3" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dad5c83079eae9969be7fadefe640a1c566901f05ff91ab221de4b6f68d9507e" +checksum = "cd56dd203fef61ac097dd65721a419ddccb106b2d2b70ba60a6b529f03961a51" dependencies = [ "async-stream-impl", "futures-core", + "pin-project-lite", ] [[package]] name = "async-stream-impl" -version = "0.3.3" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10f203db73a71dfa2fb6dd22763990fa26f3d2625a6da2da900d23b87d26be27" +checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", @@ -40,26 +95,15 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.56" +version = "0.1.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96cf8829f67d2eab0b2dfa42c5d0ef737e0724e4a82b01b3e292456202b19716" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", "syn", ] -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi", - "libc", - "winapi", -] - [[package]] name = "autocfg" version = "1.1.0" @@ -68,13 +112,13 @@ checksum = "d468802bab17cbc0cc575e9b053f41e72aa36bfa6b7f55e3529ffa43161b97fa" [[package]] name = "axum" -version = "0.5.13" +version = "0.6.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b9496f0c1d1afb7a2af4338bbe1d969cddfead41d87a9fb3aaa6d0bbc7af648" +checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" dependencies = [ "async-trait", "axum-core", - "bitflags", + "bitflags 1.3.2", "bytes", "futures-util", "http", @@ -86,20 +130,19 @@ dependencies = [ "mime", "percent-encoding", "pin-project-lite", + "rustversion", "serde", "sync_wrapper", - "tokio", "tower", - "tower-http", "tower-layer", "tower-service", ] [[package]] name = "axum-core" -version = "0.2.7" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4f44a0e6200e9d11a1cdc989e4b358f6e3d354fbf48478f345a17f4e43f8635" +checksum = "759fa577a247914fd3f7f76d62972792636412fbfd634cd452f6a385a74d2d2c" dependencies = [ "async-trait", "bytes", @@ -107,13 +150,31 @@ dependencies = [ "http", "http-body", "mime", + "rustversion", + "tower-layer", + "tower-service", +] + +[[package]] +name = "backtrace" +version = "0.3.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2089b7e3f35b9dd2d0ed921ead4f6d318c27680d4a5bd167b3ee120edb105837" +dependencies = [ + "addr2line", + "cc", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", ] [[package]] name = "base64" -version = "0.13.0" +version = "0.21.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "904dfeac50f3cdaba28fc6f57fdcddb75f49ed61346676a78c4ffe55877802fd" +checksum = "9ba43ea6f343b788c8764558649e08df62f86c6ef251fdaeb1ffd010a9ae50a2" [[package]] name = "bitflags" @@ -121,23 +182,32 @@ version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" +[[package]] +name = "bitflags" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" + [[package]] name = "bumpalo" -version = "3.10.0" +version = "3.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ccbd214614c6783386c1af30caf03192f17891059cecc394b4fb119e363de3" +checksum = "7f30e7476521f6f8af1a1c4c0b8cc94f0bee37d91763d0ca2665f299b6cd8aec" [[package]] name = "bytes" -version = "1.1.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4872d67bab6358e59559027aa3b9157c53d9358c51423c17554809a8858e0f8" +checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" [[package]] name = "cc" -version = "1.0.73" +version = "1.0.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fff2a6927b3bb87f9595d67196a70493f627687a71d87a0d692242c33f58c11" +checksum = "f1174fb0b6ec23863f8b971027804a42614e347eafb0a95bf0b12cdae21fc4d0" +dependencies = [ + "libc", +] [[package]] name = "cfg-if" @@ -147,19 +217,37 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "clap" -version = "2.34.0" +version = "4.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d04704f56c2cde07f43e8e2c154b43f216dc5c92fc98ada720177362f953b956" +dependencies = [ + "clap_builder", +] + +[[package]] +name = "clap_builder" +version = "4.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0610544180c38b88101fecf2dd634b174a62eef6946f84dfc6a7127512b381c" +checksum = "0e231faeaca65ebd1ea3c737966bf858971cd38c3849107aa3ea7de90a804e45" dependencies = [ - "ansi_term", - "atty", - "bitflags", + "anstream", + "anstyle", + "clap_lex", "strsim", - "textwrap", - "unicode-width", - "vec_map", ] +[[package]] +name = "clap_lex" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd7cc57abe963c6d3b9d8be5b06ba7c8957a930305ca90304f24ef040aa6f961" + +[[package]] +name = "colorchoice" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" + [[package]] name = "core-foundation" version = "0.9.3" @@ -172,15 +260,15 @@ dependencies = [ [[package]] name = "core-foundation-sys" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5827cebf4670468b8772dd191856768aedcb1b0278a04f989f7766351917b9dc" +checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" [[package]] name = "csv" -version = "1.2.2" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "626ae34994d3d8d668f4269922248239db4ae42d538b14c398b74a52208e8086" +checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" dependencies = [ "csv-core", "itoa", @@ -190,37 +278,44 @@ dependencies = [ [[package]] name = "csv-core" -version = "0.1.10" +version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b2466559f260f48ad25fe6317b3c8dac77b5bdb5763ac7d9d6103530663bc90" +checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" dependencies = [ "memchr", ] [[package]] name = "either" -version = "1.7.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f107b87b6afc2a64fd13cac55fe06d6c8859f12d4b14cbcdd2c67d0976781be" +checksum = "a26ae43d7bcc3b814de94796a5e736d4029efb0ee900c12e2d54c993ad1a1e07" [[package]] name = "encoding_rs" -version = "0.8.31" +version = "0.8.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9852635589dc9f9ea1b6fe9f05b50ef208c85c834a562f0c6abb1c475736ec2b" +checksum = "7268b386296a025e474d5140678f75d6de9493ae55a5d709eeb9dd08149945e1" dependencies = [ "cfg-if", ] [[package]] -name = "fastrand" -version = "1.7.0" +name = "errno" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3fcf0cee53519c866c09b5de1f6c56ff9d647101f81c1964fa632e148896cdf" +checksum = "ac3e13f66a2f95e32a39eaa81f6b95d42878ca0e1db0c7543723dfe12557e860" dependencies = [ - "instant", + "libc", + "windows-sys", ] +[[package]] +name = "fastrand" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25cbce373ec4653f1a01a31e8a5e5ec0c622dc27ff9c4e6606eefef5cbbed4a5" + [[package]] name = "fnv" version = "1.0.7" @@ -244,19 +339,18 @@ checksum = "00b0228411908ca8685dba7fc2cdd70ec9990a6e753e89b6ac91a84c40fbaf4b" [[package]] name = "form_urlencoded" -version = "1.0.1" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fc25a87fa4fd2094bffb06925852034d90a17f0d1e05197d4956d3555752191" +checksum = "a62bc1cf6f830c2ec14a513a9fb124d0a213a629668a4186f329db21fe045652" dependencies = [ - "matches", "percent-encoding", ] [[package]] name = "futures" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f73fe65f54d1e12b726f517d3e2135ca3125a437b6d998caf1962961f7172d9e" +checksum = "23342abe12aba583913b2e62f22225ff9c950774065e4bfb61a19cd9770fec40" dependencies = [ "futures-channel", "futures-core", @@ -269,9 +363,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3083ce4b914124575708913bca19bfe887522d6e2e6d0952943f5eac4a74010" +checksum = "955518d47e09b25bbebc7a18df10b81f0c766eaf4c4f1cccef2fca5f2a4fb5f2" dependencies = [ "futures-core", "futures-sink", @@ -279,15 +373,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c09fd04b7e4073ac7156a9539b57a484a8ea920f79c7c675d05d289ab6110d3" +checksum = "4bca583b7e26f571124fe5b7561d49cb2868d79116cfa0eefce955557c6fee8c" [[package]] name = "futures-executor" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9420b90cfa29e327d0429f19be13e7ddb68fa1cccb09d65e5706b8c7a749b8a6" +checksum = "ccecee823288125bd88b4d7f565c9e58e41858e47ab72e8ea2d64e93624386e0" dependencies = [ "futures-core", "futures-task", @@ -296,15 +390,15 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc4045962a5a5e935ee2fdedaa4e08284547402885ab326734432bed5d12966b" +checksum = "4fff74096e71ed47f8e023204cfd0aa1289cd54ae5430a9523be060cdb849964" [[package]] name = "futures-macro" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33c1e13800337f4d4d7a316bf45a567dbcb6ffe087f16424852d97e97a91f512" +checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", @@ -313,21 +407,21 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21163e139fa306126e6eedaf49ecdb4588f939600f0b1e770f4205ee4b7fa868" +checksum = "f43be4fe21a13b9781a69afa4985b0f6ee0e1afab2c6f454a8cf30e2b2237b6e" [[package]] name = "futures-task" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57c66a976bf5909d801bbef33416c41372779507e7a6b3a5e25e4749c58f776a" +checksum = "76d3d132be6c0e6aa1534069c705a74a5997a356c0dc2f86a47765e5617c5b65" [[package]] name = "futures-util" -version = "0.3.21" +version = "0.3.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8b7abd5d659d9b90c8cba917f6ec750a74e2dc23902ef9cd4cc8c8b22e6036a" +checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533" dependencies = [ "futures-channel", "futures-core", @@ -343,31 +437,26 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.1.16" +version = "0.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fc3cb4d91f53b50155bdcfd23f6a4c39ae1969c2ae85982b135750cccaf5fce" +checksum = "be4136b2a15dd319360be1c07d9933517ccf0be8f16bf62a3bee4f0d618df427" dependencies = [ "cfg-if", "libc", - "wasi 0.9.0+wasi-snapshot-preview1", + "wasi", ] [[package]] -name = "getrandom" -version = "0.2.7" +name = "gimli" +version = "0.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4eb1a864a501629691edf6c15a593b7a51eebaa1e8468e9ddc623de7c9b58ec6" -dependencies = [ - "cfg-if", - "libc", - "wasi 0.11.0+wasi-snapshot-preview1", -] +checksum = "6fb8d784f27acf97159b40fc4db5ecd8aa23b9ad5ef69cdd136d3bc80665f0c0" [[package]] name = "h2" -version = "0.3.13" +version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37a82c6d637fc9515a4694bbf1cb2457b79d81ce52b3108bdeea58b07dd34a57" +checksum = "91fc23aa11be92976ef4729127f1a74adf36d8436f7816b185d18df956790833" dependencies = [ "bytes", "fnv", @@ -390,18 +479,15 @@ checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" [[package]] name = "hermit-abi" -version = "0.1.19" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] +checksum = "d77f7ec81a6d05a3abb01ab6eb7590f6083d08449fe5a1c8b1e620283546ccb7" [[package]] name = "http" -version = "0.2.8" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75f43d41e26995c17e71ee126451dd3941010b0514a81a9d11f3b341debc2399" +checksum = "bd6effc99afb63425aff9b05836f029929e345a6148a14b7ecd5ab67af944482" dependencies = [ "bytes", "fnv", @@ -419,29 +505,23 @@ dependencies = [ "pin-project-lite", ] -[[package]] -name = "http-range-header" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bfe8eed0a9285ef776bb792479ea3834e8b94e13d615c2f66d03dd50a435a29" - [[package]] name = "httparse" -version = "1.7.1" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "496ce29bb5a52785b44e0f7ca2847ae0bb839c9bd28f69acac9b99d461c0c04c" +checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" [[package]] name = "httpdate" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a1e36c821dbe04574f602848a19f742f4fb3c98d40449f11bcad18d6b17421" +checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" [[package]] name = "hyper" -version = "0.14.20" +version = "0.14.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02c929dc5c39e335a03c405292728118860721b10190d98c2a0f0efd5baafbac" +checksum = "ffb1cfd654a8219eaef89881fdb3bb3b1cdc5fa75ded05d6933b2b382e395468" dependencies = [ "bytes", "futures-channel", @@ -454,7 +534,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2", + "socket2 0.4.9", "tokio", "tower-service", "tracing", @@ -488,60 +568,50 @@ dependencies = [ [[package]] name = "idna" -version = "0.2.3" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "418a0a6fab821475f634efe3ccc45c013f742efe03d853e8d3355d5cb850ecf8" +checksum = "7d20d6b07bfbc108882d88ed8e37d39636dcc260e15e30c45e6ba089610b917c" dependencies = [ - "matches", "unicode-bidi", "unicode-normalization", ] [[package]] name = "indexmap" -version = "1.9.1" +version = "1.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10a35a97730320ffe8e2d410b5d3b69279b98d2c14bdb8b70ea89ecf7888d41e" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" dependencies = [ "autocfg", "hashbrown", ] -[[package]] -name = "instant" -version = "0.1.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" -dependencies = [ - "cfg-if", -] - [[package]] name = "ipnet" -version = "2.5.0" +version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879d54834c8c76457ef4293a689b2a8c59b076067ad77b15efafbb05f92a592b" +checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" [[package]] name = "itertools" -version = "0.10.3" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9a9d19fa1e79b6215ff29b9d6880b706147f16e9b1dbb1e4e5947b5b02bc5e3" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" dependencies = [ "either", ] [[package]] name = "itoa" -version = "1.0.2" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "112c678d4050afce233f4f2852bb2eb519230b3cf12f33585275537d7e41578d" +checksum = "af150ab688ff2122fcef229be89cb50dd66af9e01a4ff320cc137eecc9bacc38" [[package]] name = "js-sys" -version = "0.3.58" +version = "0.3.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3fac17f7123a73ca62df411b1bf727ccc805daa070338fda671c86dac1bdc27" +checksum = "c5f195fe497f702db0f318b07fdd68edb16955aed830df8363d837542f8f935a" dependencies = [ "wasm-bindgen", ] @@ -554,60 +624,65 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.126" +version = "0.2.149" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "349d5a591cd28b49e1d1037471617a32ddcda5731b99419008085f72d5a53836" +checksum = "a08173bc88b7955d1b3145aa561539096c421ac8debde8cbc3612ec635fee29b" [[package]] -name = "log" -version = "0.4.17" +name = "linux-raw-sys" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abb12e687cfb44aa40f41fc3978ef76448f9b6038cad6aef4259d3c095a2382e" -dependencies = [ - "cfg-if", -] +checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" [[package]] -name = "matches" -version = "0.1.9" +name = "log" +version = "0.4.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3e378b66a060d48947b590737b30a1be76706c8dd7b8ba0f2fe3989c68a853f" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" [[package]] name = "matchit" -version = "0.5.0" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73cbba799671b762df5a175adf59ce145165747bb891505c43d09aefbbf38beb" +checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" [[package]] name = "memchr" -version = "2.5.0" +version = "2.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" +checksum = "f665ee40bc4a3c5590afb1e9677db74a508659dfd71e126420da8274909a0167" [[package]] name = "mime" -version = "0.3.16" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" + +[[package]] +name = "miniz_oxide" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a60c7ce501c71e03a9c9c0d35b861413ae925bd979cc7a4e30d060069aaac8d" +checksum = "e7810e0be55b428ada41041c41f32c9f1a42817901b4ccf45fa3d4b6561e74c7" +dependencies = [ + "adler", +] [[package]] name = "mio" -version = "0.8.4" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57ee1c23c7c63b0c9250c339ffdc69255f110b298b901b9f6c82547b7b87caaf" +checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2" dependencies = [ "libc", - "log", - "wasi 0.11.0+wasi-snapshot-preview1", + "wasi", "windows-sys", ] [[package]] name = "native-tls" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd7e2f3618557f980e0b17e8856252eee3c97fa12c54dff0ca290fb6266ca4a9" +checksum = "07226173c32f2926027b63cce4bcd8076c3552846cbe7925f3aaffeac0a3b92e" dependencies = [ "lazy_static", "libc", @@ -623,27 +698,36 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.13.1" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19e64526ebdee182341572e50e9ad03965aa510cd94427a4549448f285e957a1" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ "hermit-abi", "libc", ] +[[package]] +name = "object" +version = "0.32.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cf5f9dd3933bd50a9e1f149ec995f39ae2c496d31fd772c1fd45ebc27e902b0" +dependencies = [ + "memchr", +] + [[package]] name = "once_cell" -version = "1.13.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18a6dbe30758c9f83eb00cbea4ac95966305f5a7772f3f42ebfc7fc7eddbd8e1" +checksum = "dd8b5dd2ae5ed71462c540258bedcb51965123ad7e7ccf4b9a8cafaa4a63576d" [[package]] name = "openssl" -version = "0.10.41" +version = "0.10.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "618febf65336490dfcf20b73f885f5651a0c89c64c2d4a8c3662585a70bf5bd0" +checksum = "bac25ee399abb46215765b1cb35bc0212377e58a061560d8b29b024fd0430e7c" dependencies = [ - "bitflags", + "bitflags 2.4.1", "cfg-if", "foreign-types", "libc", @@ -654,9 +738,9 @@ dependencies = [ [[package]] name = "openssl-macros" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b501e44f11665960c7e7fcf062c7d96a14ade4aa98116c004b2e37b5be7d736c" +checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", @@ -671,11 +755,10 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "openssl-sys" -version = "0.9.75" +version = "0.9.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5f9bd0c2710541a3cda73d6f9ac4f1b240de4ae261065d309dbe73d9dceb42f" +checksum = "db4d56a4c0478783083cfafcc42493dd4a981d41669da64b4572a2a089b51b1d" dependencies = [ - "autocfg", "cc", "libc", "pkg-config", @@ -684,24 +767,24 @@ dependencies = [ [[package]] name = "percent-encoding" -version = "2.1.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" +checksum = "9b2a4787296e9989611394c33f193f676704af1686e70b8f8033ab5ba9a35a94" [[package]] name = "pin-project" -version = "1.0.11" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78203e83c48cffbe01e4a2d35d566ca4de445d79a85372fc64e378bfc812a260" +checksum = "fda4ed1c6c173e3fc7a83629421152e01d7b1f9b7f65fb301e490e8cfc656422" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.0.11" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "710faf75e1b33345361201d36d04e98ac1ed8909151a017ed384700836104c74" +checksum = "4359fd9c9171ec6e8c62926d6faaf553a8dc3f64e1507e76da7911b4f6a04405" dependencies = [ "proc-macro2", "quote", @@ -710,9 +793,9 @@ dependencies = [ [[package]] name = "pin-project-lite" -version = "0.2.9" +version = "0.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0a7ae3ac2f1173085d398531c705756c94a4c56843785df85a60c1a0afac116" +checksum = "8afb450f006bf6385ca15ef45d71d2288452bc3683ce2e2cacc0d18e4be60b58" [[package]] name = "pin-utils" @@ -722,30 +805,30 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pkg-config" -version = "0.3.25" +version = "0.3.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1df8c4ec4b0627e53bdf214615ad287367e482558cf84b109250b37464dc03ae" +checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" [[package]] name = "ppv-lite86" -version = "0.2.16" +version = "0.2.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb9f9e6e233e5c4a35559a617bf40a4ec447db2e84c20b55a6f83167b7e57872" +checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" [[package]] name = "proc-macro2" -version = "1.0.40" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd96a1e8ed2596c337f8eae5f24924ec83f5ad5ab21ea8e455d3566c69fbcaf7" +checksum = "134c189feb4956b20f6f547d2cf727d4c0fe06722b20a0eec87ed445a97f92da" dependencies = [ "unicode-ident", ] [[package]] name = "prost" -version = "0.10.4" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71adf41db68aa0daaefc69bb30bcd68ded9b9abaad5d1fbb6304c4fb390e083e" +checksum = "f4fdd22f3b9c31b53c060df4a0613a1c7f062d4115a2b984dd15b1858f7e340d" dependencies = [ "bytes", "prost-derive", @@ -753,9 +836,9 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.10.1" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b670f45da57fb8542ebdbb6105a925fe571b67f9e7ed9f47a06a84e72b4e7cc" +checksum = "265baba7fabd416cf5078179f7d2cbeca4ce7a9041111900675ea7c4cb8a4c32" dependencies = [ "anyhow", "itertools", @@ -766,26 +849,13 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.20" +version = "1.0.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3bcdf212e9776fbcb2d23ab029360416bb1706b1aea2d1a5ba002727cbcab804" +checksum = "5267fca4496028628a95160fc423a33e8b2e6af8a5302579e322e4b520293cae" dependencies = [ "proc-macro2", ] -[[package]] -name = "rand" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a6b1679d49b24bbfe0c803429aa1874472f50d9b363131f0e89fc356b544d03" -dependencies = [ - "getrandom 0.1.16", - "libc", - "rand_chacha 0.2.2", - "rand_core 0.5.1", - "rand_hc", -] - [[package]] name = "rand" version = "0.8.5" @@ -793,18 +863,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" dependencies = [ "libc", - "rand_chacha 0.3.1", - "rand_core 0.6.3", -] - -[[package]] -name = "rand_chacha" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4c8ed856279c9737206bf725bf36935d8666ead7aa69b52be55af369d193402" -dependencies = [ - "ppv-lite86", - "rand_core 0.5.1", + "rand_chacha", + "rand_core", ] [[package]] @@ -814,59 +874,32 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" dependencies = [ "ppv-lite86", - "rand_core 0.6.3", -] - -[[package]] -name = "rand_core" -version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90bde5296fc891b0cef12a6d03ddccc162ce7b2aff54160af9338f8d40df6d19" -dependencies = [ - "getrandom 0.1.16", + "rand_core", ] [[package]] name = "rand_core" -version = "0.6.3" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d34f1408f55294453790c48b2f1ebbb1c5b4b7563eb1f418bcfcfdbb06ebb4e7" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" dependencies = [ - "getrandom 0.2.7", -] - -[[package]] -name = "rand_hc" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3129af7b92a17112d59ad498c6f81eaf463253766b90396d39ea7a39d6613c" -dependencies = [ - "rand_core 0.5.1", + "getrandom", ] [[package]] name = "redox_syscall" -version = "0.2.13" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62f25bc4c7e55e0b0b7a1d43fb893f4fa1361d0abe38b9ce4f323c2adfe6ef42" +checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" dependencies = [ - "bitflags", -] - -[[package]] -name = "remove_dir_all" -version = "0.5.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3acd125665422973a33ac9d3dd2df85edad0f4ae9b00dafb1a05e43a9f5ef8e7" -dependencies = [ - "winapi", + "bitflags 1.3.2", ] [[package]] name = "reqwest" -version = "0.11.11" +version = "0.11.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b75aa69a3f06bbcc66ede33af2af253c6f7a86b1ca0033f60c580a27074fbf92" +checksum = "046cd98826c46c2ac8ddecae268eb5c2e58628688a5fc7a2643704a73faba95b" dependencies = [ "base64", "bytes", @@ -880,15 +913,16 @@ dependencies = [ "hyper-tls", "ipnet", "js-sys", - "lazy_static", "log", "mime", "native-tls", + "once_cell", "percent-encoding", "pin-project-lite", "serde", "serde_json", "serde_urlencoded", + "system-configuration", "tokio", "tokio-native-tls", "tower-service", @@ -899,29 +933,53 @@ dependencies = [ "winreg", ] +[[package]] +name = "rustc-demangle" +version = "0.1.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" + +[[package]] +name = "rustix" +version = "0.38.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "745ecfa778e66b2b63c88a61cb36e0eea109e803b0b86bf9879fbc77c70e86ed" +dependencies = [ + "bitflags 2.4.1", + "errno", + "libc", + "linux-raw-sys", + "windows-sys", +] + +[[package]] +name = "rustversion" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ffc183a10b4478d04cbbbfc96d0873219d962dd5accaff2ffbd4ceb7df837f4" + [[package]] name = "ryu" -version = "1.0.10" +version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3f6f92acf49d1b98f7a81226834412ada05458b7364277387724a237f062695" +checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" [[package]] name = "schannel" -version = "0.1.20" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88d6731146462ea25d9244b2ed5fd1d716d25c52e4d54aa4fb0f3c4e9854dbe2" +checksum = "0c3733bf4cf7ea0880754e19cb5a462007c4a8c1914bff372ccc95b464f1df88" dependencies = [ - "lazy_static", "windows-sys", ] [[package]] name = "security-framework" -version = "2.6.1" +version = "2.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dc14f172faf8a0194a3aded622712b0de276821addc574fa54fc0a1167e10dc" +checksum = "05b64fb303737d99b81884b2c63433e9ae28abebe5eb5045dcdd175dc2ecf4de" dependencies = [ - "bitflags", + "bitflags 1.3.2", "core-foundation", "core-foundation-sys", "libc", @@ -930,9 +988,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.6.1" +version = "2.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0160a13a177a45bfb43ce71c01580998474f556ad854dcbca936dd2841a5c556" +checksum = "e932934257d3b408ed8f30db49d85ea163bfe74961f017f405b025af298f0c7a" dependencies = [ "core-foundation-sys", "libc", @@ -940,18 +998,18 @@ dependencies = [ [[package]] name = "serde" -version = "1.0.139" +version = "1.0.189" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0171ebb889e45aa68b44aee0859b3eede84c6f5f5c228e6f140c0b2a0a46cad6" +checksum = "8e422a44e74ad4001bdc8eede9a4570ab52f71190e9c076d14369f38b9200537" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.139" +version = "1.0.189" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc1d3230c1de7932af58ad8ffbe1d784bd55efd5a9d84ac24f69c72d83543dfb" +checksum = "1e48d1f918009ce3145511378cf68d613e3b3d9137d67272562080d68a2b32d5" dependencies = [ "proc-macro2", "quote", @@ -960,9 +1018,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.82" +version = "1.0.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82c2c1fdcd807d1098552c5b9a36e425e42e9fbd7c6a37a8425f390f781f7fa7" +checksum = "6b420ce6e3d8bd882e9b243c6eed35dbc9a6110c9769e74b584e0d68d1f20c65" dependencies = [ "itoa", "ryu", @@ -989,7 +1047,7 @@ dependencies = [ "csv", "futures", "prost", - "rand 0.7.3", + "rand", "reqwest", "serde", "serde_derive", @@ -1000,31 +1058,44 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.6" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb703cfe953bccee95685111adeedb76fabe4e97549a58d16f03ea7b9367bb32" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] [[package]] name = "socket2" -version = "0.4.4" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66d72b759436ae32898a2af0a14218dbf55efde3feeb170eb623637db85ee1e0" +checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662" dependencies = [ "libc", "winapi", ] +[[package]] +name = "socket2" +version = "0.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4031e820eb552adee9295814c0ced9e5cf38ddf1e8b7d566d6de8e2538ea989e" +dependencies = [ + "libc", + "windows-sys", +] + [[package]] name = "strsim" -version = "0.8.0" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ea5119cdb4c55b55d432abb513a0429384878c15dde60cc77b1c99de1a95a6a" +checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" [[package]] name = "syn" -version = "1.0.98" +version = "2.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c50aef8a904de4c23c788f104b7dddc7d6f79c647c7c8ce4cc8f73eb0ca773dd" +checksum = "e96b79aaa137db8f61e26363a0c9b47d8b4ec75da28b7d1d614c2303e232408b" dependencies = [ "proc-macro2", "quote", @@ -1033,31 +1104,42 @@ dependencies = [ [[package]] name = "sync_wrapper" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20518fe4a4c9acf048008599e464deb21beeae3d3578418951a189c235a7a9a8" +checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" [[package]] -name = "tempfile" -version = "3.3.0" +name = "system-configuration" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cdb1ef4eaeeaddc8fbd371e5017057064af0911902ef36b39801f67cc6d79e4" +checksum = "ba3a3adc5c275d719af8cb4272ea1c4a6d668a777f37e115f6d11ddbc1c8e0e7" dependencies = [ - "cfg-if", - "fastrand", + "bitflags 1.3.2", + "core-foundation", + "system-configuration-sys", +] + +[[package]] +name = "system-configuration-sys" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75fb188eb626b924683e3b95e3a48e63551fcfb51949de2f06a9d91dbee93c9" +dependencies = [ + "core-foundation-sys", "libc", - "redox_syscall", - "remove_dir_all", - "winapi", ] [[package]] -name = "textwrap" -version = "0.11.0" +name = "tempfile" +version = "3.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d326610f408c7a4eb6f51c37c330e496b08506c9457c9d34287ecc38809fb060" +checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" dependencies = [ - "unicode-width", + "cfg-if", + "fastrand", + "redox_syscall", + "rustix", + "windows-sys", ] [[package]] @@ -1071,27 +1153,25 @@ dependencies = [ [[package]] name = "tinyvec_macros" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.20.0" +version = "1.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57aec3cfa4c296db7255446efb4928a6be304b431a806216105542a67b6ca82e" +checksum = "4f38200e3ef7995e5ef13baec2f432a6da0aa9ac495b2c0e8f3b7eec2c92d653" dependencies = [ - "autocfg", + "backtrace", "bytes", "libc", - "memchr", "mio", "num_cpus", - "once_cell", "pin-project-lite", - "socket2", + "socket2 0.5.4", "tokio-macros", - "winapi", + "windows-sys", ] [[package]] @@ -1106,9 +1186,9 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "1.8.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9724f9a975fb987ef7a3cd9be0350edcbe130698af5b8f7a631e23d42d052484" +checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", @@ -1117,9 +1197,9 @@ dependencies = [ [[package]] name = "tokio-native-tls" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7d995660bd2b7f8c1568414c1126076c13fbb725c40112dc0120b78eb9b717b" +checksum = "bbae76ab933c85776efabc971569dd6119c580d8f5d448769dec1764bf796ef2" dependencies = [ "native-tls", "tokio", @@ -1127,9 +1207,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.9" +version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df54d54117d6fdc4e4fea40fe1e4e566b3505700e148a6827e59b34b0d2600d9" +checksum = "397c988d37662c7dda6d2208364a706264bf3d6138b11d436cbac0ad38832842" dependencies = [ "futures-core", "pin-project-lite", @@ -1138,9 +1218,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.3" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc463cd8deddc3770d20f9852143d50bf6094e640b485cb2e189a2099085ff45" +checksum = "1d68074620f57a0b21594d9735eb2e98ab38b17f80d3fcb189fca266771ca60d" dependencies = [ "bytes", "futures-core", @@ -1152,17 +1232,15 @@ dependencies = [ [[package]] name = "tonic" -version = "0.7.2" +version = "0.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5be9d60db39854b30b835107500cf0aca0b0d14d6e1c3de124217c23a29c2ddb" +checksum = "d560933a0de61cf715926b9cac824d4c883c2c43142f787595e48280c40a1d0e" dependencies = [ "async-stream", "async-trait", "axum", "base64", "bytes", - "futures-core", - "futures-util", "h2", "http", "http-body", @@ -1171,15 +1249,12 @@ dependencies = [ "percent-encoding", "pin-project", "prost", - "prost-derive", "tokio", "tokio-stream", - "tokio-util", "tower", "tower-layer", "tower-service", "tracing", - "tracing-futures", ] [[package]] @@ -1193,7 +1268,7 @@ dependencies = [ "indexmap", "pin-project", "pin-project-lite", - "rand 0.8.5", + "rand", "slab", "tokio", "tokio-util", @@ -1202,30 +1277,11 @@ dependencies = [ "tracing", ] -[[package]] -name = "tower-http" -version = "0.3.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c530c8675c1dbf98facee631536fa116b5fb6382d7dd6dc1b118d970eafe3ba" -dependencies = [ - "bitflags", - "bytes", - "futures-core", - "futures-util", - "http", - "http-body", - "http-range-header", - "pin-project-lite", - "tower", - "tower-layer", - "tower-service", -] - [[package]] name = "tower-layer" -version = "0.3.1" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "343bc9466d3fe6b0f960ef45960509f84480bf4fd96f92901afe7ff3df9d3a62" +checksum = "c20c8dbed6283a09604c3e69b4b7eeb54e298b8a600d4d5ecb5ad39de609f1d0" [[package]] name = "tower-service" @@ -1235,12 +1291,10 @@ checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" -version = "0.1.35" +version = "0.1.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a400e31aa60b9d44a52a8ee0343b5b18566b03a8321e0d321f695cf56e940160" +checksum = "ee2ef2af84856a50c1d430afce2fdded0a4ec7eda868db86409b4543df0797f9" dependencies = [ - "cfg-if", - "log", "pin-project-lite", "tracing-attributes", "tracing-core", @@ -1248,9 +1302,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.22" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11c75893af559bc8e10716548bdef5cb2b983f8e637db9d0e15126b61b484ee2" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", @@ -1259,96 +1313,72 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.28" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b7358be39f2f274f322d2aaed611acc57f382e8eb1e5b48cb9ae30933495ce7" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" dependencies = [ "once_cell", ] -[[package]] -name = "tracing-futures" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97d095ae15e245a057c8e8451bab9b3ee1e1f68e9ba2b4fbc18d0ac5237835f2" -dependencies = [ - "pin-project", - "tracing", -] - [[package]] name = "try-lock" -version = "0.2.3" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59547bce71d9c38b83d9c0e92b6066c4253371f15005def0c30d9657f50c7642" +checksum = "3528ecfd12c466c6f163363caf2d02a71161dd5e1cc6ae7b34207ea2d42d81ed" [[package]] name = "unicode-bidi" -version = "0.3.8" +version = "0.3.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "099b7128301d285f79ddd55b9a83d5e6b9e97c92e0ea0daebee7263e932de992" +checksum = "92888ba5573ff080736b3648696b70cafad7d250551175acbaa4e0385b3e1460" [[package]] name = "unicode-ident" -version = "1.0.2" +version = "1.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15c61ba63f9235225a22310255a29b806b907c9b8c964bcbd0a2c70f3f2deea7" +checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" [[package]] name = "unicode-normalization" -version = "0.1.21" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "854cbdc4f7bc6ae19c820d44abdc3277ac3e1b2b93db20a636825d9322fb60e6" +checksum = "5c5713f0fc4b5db668a2ac63cdb7bb4469d8c9fed047b1d0292cc7b0ce2ba921" dependencies = [ "tinyvec", ] -[[package]] -name = "unicode-width" -version = "0.1.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0edd1e5b14653f783770bce4a4dabb4a5108a5370a5f5d8cfe8710c361f6c8b" - [[package]] name = "url" -version = "2.2.2" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a507c383b2d33b5fc35d1861e77e6b383d158b2da5e14fe51b83dfedf6fd578c" +checksum = "143b538f18257fac9cad154828a57c6bf5157e1aa604d4816b5995bf6de87ae5" dependencies = [ "form_urlencoded", "idna", - "matches", "percent-encoding", ] [[package]] -name = "vcpkg" -version = "0.2.15" +name = "utf8parse" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" +checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] -name = "vec_map" -version = "0.8.2" +name = "vcpkg" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1bddf1187be692e79c5ffeab891132dfb0f236ed36a43c7ed39f1165ee20191" +checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" [[package]] name = "want" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ce8a968cb1cd110d136ff8b819a556d6fb6d919363c61534f6860c7eb172ba0" +checksum = "bfa7760aed19e106de2c7c0b581b509f2f25d3dacaf737cb82ac61bc6d760b0e" dependencies = [ - "log", "try-lock", ] -[[package]] -name = "wasi" -version = "0.9.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cccddf32554fecc6acb585f82a32a72e28b48f8c4c1883ddfeeeaa96f7d8e519" - [[package]] name = "wasi" version = "0.11.0+wasi-snapshot-preview1" @@ -1357,9 +1387,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.81" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c53b543413a17a202f4be280a7e5c62a1c69345f5de525ee64f8cfdbc954994" +checksum = "7706a72ab36d8cb1f80ffbf0e071533974a60d0a308d01a5d0375bf60499a342" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -1367,13 +1397,13 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.81" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5491a68ab4500fa6b4d726bd67408630c3dbe9c4fe7bda16d5c82a1fd8c7340a" +checksum = "5ef2b6d3c510e9625e5fe6f509ab07d66a760f0885d858736483c32ed7809abd" dependencies = [ "bumpalo", - "lazy_static", "log", + "once_cell", "proc-macro2", "quote", "syn", @@ -1382,9 +1412,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.31" +version = "0.4.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de9a9cec1733468a8c657e57fa2413d2ae2c0129b95e87c5b72b8ace4d13f31f" +checksum = "c02dbc21516f9f1f04f187958890d7e6026df8d16540b7ad9492bc34a67cea03" dependencies = [ "cfg-if", "js-sys", @@ -1394,9 +1424,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.81" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c441e177922bc58f1e12c022624b6216378e5febc2f0533e41ba443d505b80aa" +checksum = "dee495e55982a3bd48105a7b947fd2a9b4a8ae3010041b9e0faab3f9cd028f1d" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -1404,9 +1434,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.81" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d94ac45fcf608c1f45ef53e748d35660f168490c10b23704c7779ab8f5c3048" +checksum = "54681b18a46765f095758388f2d0cf16eb8d4169b639ab575a8f5693af210c7b" dependencies = [ "proc-macro2", "quote", @@ -1417,15 +1447,15 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.81" +version = "0.2.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a89911bd99e5f3659ec4acf9c4d93b0a90fe4a2a11f15328472058edc5261be" +checksum = "ca6ad05a4870b2bf5fe995117d3728437bd27d7cd5f06f13c17443ef369775a1" [[package]] name = "web-sys" -version = "0.3.58" +version = "0.3.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fed94beee57daf8dd7d51f2b15dc2bcde92d7a72304cdf662a4371008b71b90" +checksum = "9b85cbef8c220a6abc02aefd892dfc0fc23afb1c6a426316ec33253a3877249b" dependencies = [ "js-sys", "wasm-bindgen", @@ -1455,52 +1485,76 @@ checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] name = "windows-sys" -version = "0.36.1" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" +dependencies = [ + "windows-targets", +] + +[[package]] +name = "windows-targets" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea04155a16a59f9eab786fe12a4a450e75cdb175f9e0d80da1e17db09f55b8d2" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" dependencies = [ + "windows_aarch64_gnullvm", "windows_aarch64_msvc", "windows_i686_gnu", "windows_i686_msvc", "windows_x86_64_gnu", + "windows_x86_64_gnullvm", "windows_x86_64_msvc", ] +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" + [[package]] name = "windows_aarch64_msvc" -version = "0.36.1" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9bb8c3fd39ade2d67e9874ac4f3db21f0d710bee00fe7cab16949ec184eeaa47" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_i686_gnu" -version = "0.36.1" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "180e6ccf01daf4c426b846dfc66db1fc518f074baa793aa7d9b9aaeffad6a3b6" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_msvc" -version = "0.36.1" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2e7917148b2812d1eeafaeb22a97e4813dfa60a3f8f78ebe204bcc88f12f024" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_x86_64_gnu" -version = "0.36.1" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dcd171b8776c41b97521e5da127a2d86ad280114807d0b2ab1e462bc764d9e1" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_msvc" -version = "0.36.1" +version = "0.48.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c811ca4a8c853ef420abd8592ba53ddbbac90410fab6903b3e79972a631f7680" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "winreg" -version = "0.10.1" +version = "0.50.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80d0f4e272c85def139476380b12f9ac60926689dd2e01d4923222f40580869d" +checksum = "524e57b2c537c0f9b1e69f1965311ec12182b4122e45035b1508cd24d2adadb1" dependencies = [ - "winapi", + "cfg-if", + "windows-sys", ] diff --git a/integration_tests/feature-store/simulator/Cargo.toml b/integration_tests/feature-store/simulator/Cargo.toml index 1fd9609ba2d1a..03264cde563bb 100644 --- a/integration_tests/feature-store/simulator/Cargo.toml +++ b/integration_tests/feature-store/simulator/Cargo.toml @@ -9,17 +9,14 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -tokio = { version = "1", features=["rt", "rt-multi-thread"]} -tonic = "0.7.1" +tokio = { version = "1", features=["macros","rt", "rt-multi-thread"]} +tonic = "0.10.2" reqwest = { version = "0.11"} serde_json = "1.0" serde_derive = "1.0" -rand = "0.7" -clap = "2.26.0" -prost = "0.10" +rand = "0.8" +clap = "4.4.6" +prost = "0.12" serde = { version = "1", features = ["derive"] } -futures = "0.3.0" -csv = "1.2.2" - -[lints] -workspace = true +futures = "0.3.28" +csv = "1.3.0" diff --git a/integration_tests/feature-store/simulator/rust-toolchain b/integration_tests/feature-store/simulator/rust-toolchain index 35bda38a1dcfb..292fe499e3b25 100644 --- a/integration_tests/feature-store/simulator/rust-toolchain +++ b/integration_tests/feature-store/simulator/rust-toolchain @@ -1 +1,2 @@ -nightly-2022-06-20 +[toolchain] +channel = "stable" diff --git a/integration_tests/feature-store/simulator/src/entities.rs b/integration_tests/feature-store/simulator/src/entities.rs index 114a1e5c1ee0f..0bdf4fe0235d3 100644 --- a/integration_tests/feature-store/simulator/src/entities.rs +++ b/integration_tests/feature-store/simulator/src/entities.rs @@ -38,7 +38,7 @@ impl User { &'a self, client: &'a mut ServerClient, ) -> Result { - let changenum: i64 = rand::thread_rng().gen_range(0, 90); + let changenum: i64 = rand::thread_rng().gen_range(0..90); let (changenum, event_type) = { if changenum > 0 && changenum < 30 { (changenum, "mfa+") @@ -60,7 +60,7 @@ impl User { Ok(ActionHistory { userid: self.userid.clone(), - changenum: changenum, + changenum, event_type: event_type.to_string(), timestamp, }) @@ -93,5 +93,5 @@ pub fn parse_user_metadata() -> Result, ()> { let users = read_users_json(Path::new(&*generator_path).join("users.json")).unwrap(); - return Ok(users); + Ok(users) } diff --git a/integration_tests/feature-store/simulator/src/entities_taxi.rs b/integration_tests/feature-store/simulator/src/entities_taxi.rs index 8ef9e1f358e4b..b5800f707d0e0 100644 --- a/integration_tests/feature-store/simulator/src/entities_taxi.rs +++ b/integration_tests/feature-store/simulator/src/entities_taxi.rs @@ -101,7 +101,7 @@ impl TaxiFeature { mta_tax: self.mta_tax, tip_amount: self.tip_amount, tolls_amount: self.tolls_amount, - ehail_fee: self.ehail_fee.unwrap_or_else(|| 0.0), + ehail_fee: self.ehail_fee.unwrap_or(0.0), improvement_surcharge: self.improvement_surcharge, total_amount: self.total_amount, payment_type: self.payment_type, diff --git a/integration_tests/feature-store/simulator/src/main.rs b/integration_tests/feature-store/simulator/src/main.rs index daa6c30af1c74..f062b63d9cc84 100644 --- a/integration_tests/feature-store/simulator/src/main.rs +++ b/integration_tests/feature-store/simulator/src/main.rs @@ -1,4 +1,4 @@ -use clap::{App, Arg, ArgMatches}; +use clap::{Arg, ArgMatches, Command}; mod entities; mod entities_taxi; @@ -9,22 +9,22 @@ mod simulation; async fn main() { let args = get_args(); simulation::main_loop( - args.value_of("types") + args.get_one::("types") .expect("failed to decode brokers") .to_string(), ) .await; } -fn get_args<'a>() -> ArgMatches<'a> { - App::new("simulator") +fn get_args() -> ArgMatches { + Command::new("simulator") .about("The simulator") .arg( - Arg::with_name("types") - .short("t") + Arg::new("types") + .short('t') .long("types") .help("mfa or taxi") - .takes_value(true) + .num_args(1) .default_value("taxi"), ) .get_matches() diff --git a/integration_tests/feature-store/simulator/src/server_pb.rs b/integration_tests/feature-store/simulator/src/server_pb.rs index 1f2912d868412..697c2870b7df8 100644 --- a/integration_tests/feature-store/simulator/src/server_pb.rs +++ b/integration_tests/feature-store/simulator/src/server_pb.rs @@ -1,96 +1,104 @@ +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReportActionRequest { - #[prost(string, tag="1")] + #[prost(string, tag = "1")] pub userid: ::prost::alloc::string::String, - #[prost(string, tag="2")] + #[prost(string, tag = "2")] pub eventtype: ::prost::alloc::string::String, - #[prost(int64, tag="3")] + #[prost(int64, tag = "3")] pub changenum: i64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReportActionResponse { - #[prost(uint64, tag="1")] + #[prost(uint64, tag = "1")] pub timestamp: u64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetFeatureRequest { - #[prost(string, tag="1")] + #[prost(string, tag = "1")] pub userid: ::prost::alloc::string::String, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetFeatureResponse { - #[prost(uint64, tag="1")] + #[prost(uint64, tag = "1")] pub count: u64, - #[prost(int64, tag="2")] + #[prost(int64, tag = "2")] pub sum: i64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReportTaxiActionRequest { - #[prost(int32, tag="1")] + #[prost(int32, tag = "1")] pub vendor_id: i32, - #[prost(string, tag="2")] + #[prost(string, tag = "2")] pub lpep_pickup_datetime: ::prost::alloc::string::String, - #[prost(string, tag="3")] + #[prost(string, tag = "3")] pub lpep_dropoff_datetime: ::prost::alloc::string::String, - #[prost(bool, tag="4")] + #[prost(bool, tag = "4")] pub store_and_fwd_flag: bool, - #[prost(double, tag="5")] + #[prost(double, tag = "5")] pub ratecode_id: f64, - #[prost(int64, tag="6")] + #[prost(int64, tag = "6")] pub pu_location_id: i64, - #[prost(int64, tag="7")] + #[prost(int64, tag = "7")] pub do_location_id: i64, - #[prost(double, tag="8")] + #[prost(double, tag = "8")] pub passenger_count: f64, - #[prost(double, tag="9")] + #[prost(double, tag = "9")] pub trip_distance: f64, - #[prost(double, tag="10")] + #[prost(double, tag = "10")] pub fare_amount: f64, - #[prost(double, tag="11")] + #[prost(double, tag = "11")] pub extra: f64, - #[prost(double, tag="12")] + #[prost(double, tag = "12")] pub mta_tax: f64, - #[prost(double, tag="13")] + #[prost(double, tag = "13")] pub tip_amount: f64, - #[prost(double, tag="14")] + #[prost(double, tag = "14")] pub tolls_amount: f64, - #[prost(double, tag="15")] + #[prost(double, tag = "15")] pub ehail_fee: f64, - #[prost(double, tag="16")] + #[prost(double, tag = "16")] pub improvement_surcharge: f64, - #[prost(double, tag="17")] + #[prost(double, tag = "17")] pub total_amount: f64, - #[prost(double, tag="18")] + #[prost(double, tag = "18")] pub payment_type: f64, - #[prost(double, tag="19")] + #[prost(double, tag = "19")] pub trip_type: f64, - #[prost(double, tag="20")] + #[prost(double, tag = "20")] pub congestion_surcharge: f64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct ReportTaxiActionResponse { -} +pub struct ReportTaxiActionResponse {} +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetTaxiAmountRequest { - #[prost(int64, tag="1")] + #[prost(int64, tag = "1")] pub do_location_id: i64, - #[prost(int64, tag="2")] + #[prost(int64, tag = "2")] pub pu_location_id: i64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct GetTaxiAmountResponse { - #[prost(double, tag="1")] + #[prost(double, tag = "1")] pub fare_amount: f64, } +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct StartTrainingRequest { -} +pub struct StartTrainingRequest {} +#[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct StartTrainingResponse { -} +pub struct StartTrainingResponse {} /// Generated client implementations. pub mod server_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::http::Uri; use tonic::codegen::*; #[derive(Debug, Clone)] pub struct ServerClient { @@ -100,7 +108,7 @@ pub mod server_client { /// Attempt to create a new client by connecting to a given endpoint. pub async fn connect(dst: D) -> Result where - D: std::convert::TryInto, + D: TryInto, D::Error: Into, { let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; @@ -118,6 +126,10 @@ pub mod server_client { let inner = tonic::client::Grpc::new(inner); Self { inner } } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } pub fn with_interceptor( inner: T, interceptor: F, @@ -131,121 +143,131 @@ pub mod server_client { >::ResponseBody, >, >, - , - >>::Error: Into + Send + Sync, + >>::Error: + Into + Send + Sync, { ServerClient::new(InterceptedService::new(inner, interceptor)) } - /// Compress requests with `gzip`. + /// Compress requests with the given encoding. /// /// This requires the server to support it otherwise it might respond with an /// error. #[must_use] - pub fn send_gzip(mut self) -> Self { - self.inner = self.inner.send_gzip(); + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); self } - /// Enable decompressing responses with `gzip`. + /// Enable decompressing responses. #[must_use] - pub fn accept_gzip(mut self) -> Self { - self.inner = self.inner.accept_gzip(); + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); self } pub async fn get_feature( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/GetFeature", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/GetFeature"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "GetFeature")); + self.inner.unary(req, path, codec).await } pub async fn report_action( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/ReportAction", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/ReportAction"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "ReportAction")); + self.inner.unary(req, path, codec).await } pub async fn report_taxi_action( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/ReportTaxiAction", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/ReportTaxiAction"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "ReportTaxiAction")); + self.inner.unary(req, path, codec).await } pub async fn get_taxi_amount( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/GetTaxiAmount", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/GetTaxiAmount"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "GetTaxiAmount")); + self.inner.unary(req, path, codec).await } pub async fn start_training( &mut self, request: impl tonic::IntoRequest, - ) -> Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; + ) -> std::result::Result, tonic::Status> + { + self.inner.ready().await.map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/server_pb.Server/StartTraining", - ); - self.inner.unary(request.into_request(), path, codec).await + let path = http::uri::PathAndQuery::from_static("/server_pb.Server/StartTraining"); + let mut req = request.into_request(); + req.extensions_mut() + .insert(GrpcMethod::new("server_pb.Server", "StartTraining")); + self.inner.unary(req, path, codec).await } } } @@ -253,35 +275,37 @@ pub mod server_client { pub mod server_server { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] use tonic::codegen::*; - ///Generated trait containing gRPC methods that should be implemented for use with ServerServer. + /// Generated trait containing gRPC methods that should be implemented for use with ServerServer. #[async_trait] pub trait Server: Send + Sync + 'static { async fn get_feature( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn report_action( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn report_taxi_action( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn get_taxi_amount( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; async fn start_training( &self, request: tonic::Request, - ) -> Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; } #[derive(Debug)] pub struct ServerServer { inner: _Inner, - accept_compression_encodings: (), - send_compression_encodings: (), + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, } struct _Inner(Arc); impl ServerServer { @@ -294,17 +318,44 @@ pub mod server_server { inner, accept_compression_encodings: Default::default(), send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, } } - pub fn with_interceptor( - inner: T, - interceptor: F, - ) -> InterceptedService + pub fn with_interceptor(inner: T, interceptor: F) -> InterceptedService where F: tonic::service::Interceptor, { InterceptedService::new(Self::new(inner), interceptor) } + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } } impl tonic::codegen::Service> for ServerServer where @@ -318,7 +369,7 @@ pub mod server_server { fn poll_ready( &mut self, _cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll> { Poll::Ready(Ok(())) } fn call(&mut self, req: http::Request) -> Self::Future { @@ -327,24 +378,23 @@ pub mod server_server { "/server_pb.Server/GetFeature" => { #[allow(non_camel_case_types)] struct GetFeatureSvc(pub Arc); - impl tonic::server::UnaryService - for GetFeatureSvc { + impl tonic::server::UnaryService for GetFeatureSvc { type Response = super::GetFeatureResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + type Future = BoxFuture, tonic::Status>; fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { (*inner).get_feature(request).await }; + let inner = Arc::clone(&self.0); + let fut = + async move { ::get_feature(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -354,6 +404,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -363,28 +417,23 @@ pub mod server_server { "/server_pb.Server/ReportAction" => { #[allow(non_camel_case_types)] struct ReportActionSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for ReportActionSvc { + impl tonic::server::UnaryService for ReportActionSvc { type Response = super::ReportActionResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + type Future = BoxFuture, tonic::Status>; fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { - (*inner).report_action(request).await - }; + let inner = Arc::clone(&self.0); + let fut = + async move { ::report_action(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -394,6 +443,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -403,28 +456,26 @@ pub mod server_server { "/server_pb.Server/ReportTaxiAction" => { #[allow(non_camel_case_types)] struct ReportTaxiActionSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for ReportTaxiActionSvc { + impl tonic::server::UnaryService + for ReportTaxiActionSvc + { type Response = super::ReportTaxiActionResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + type Future = BoxFuture, tonic::Status>; fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); + let inner = Arc::clone(&self.0); let fut = async move { - (*inner).report_taxi_action(request).await + ::report_taxi_action(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -434,6 +485,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -443,28 +498,24 @@ pub mod server_server { "/server_pb.Server/GetTaxiAmount" => { #[allow(non_camel_case_types)] struct GetTaxiAmountSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for GetTaxiAmountSvc { + impl tonic::server::UnaryService for GetTaxiAmountSvc { type Response = super::GetTaxiAmountResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + type Future = BoxFuture, tonic::Status>; fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); + let inner = Arc::clone(&self.0); let fut = async move { - (*inner).get_taxi_amount(request).await + ::get_taxi_amount(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -474,6 +525,10 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) @@ -483,28 +538,23 @@ pub mod server_server { "/server_pb.Server/StartTraining" => { #[allow(non_camel_case_types)] struct StartTrainingSvc(pub Arc); - impl< - T: Server, - > tonic::server::UnaryService - for StartTrainingSvc { + impl tonic::server::UnaryService for StartTrainingSvc { type Response = super::StartTrainingResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; + type Future = BoxFuture, tonic::Status>; fn call( &mut self, request: tonic::Request, ) -> Self::Future { - let inner = self.0.clone(); - let fut = async move { - (*inner).start_training(request).await - }; + let inner = Arc::clone(&self.0); + let fut = + async move { ::start_training(&inner, request).await }; Box::pin(fut) } } let accept_compression_encodings = self.accept_compression_encodings; let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; let inner = self.inner.clone(); let fut = async move { let inner = inner.0; @@ -514,24 +564,24 @@ pub mod server_server { .apply_compression_config( accept_compression_encodings, send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, ); let res = grpc.unary(method, req).await; Ok(res) }; Box::pin(fut) } - _ => { - Box::pin(async move { - Ok( - http::Response::builder() - .status(200) - .header("grpc-status", "12") - .header("content-type", "application/grpc") - .body(empty_body()) - .unwrap(), - ) - }) - } + _ => Box::pin(async move { + Ok(http::Response::builder() + .status(200) + .header("grpc-status", "12") + .header("content-type", "application/grpc") + .body(empty_body()) + .unwrap()) + }), } } } @@ -542,12 +592,14 @@ pub mod server_server { inner, accept_compression_encodings: self.accept_compression_encodings, send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, } } } impl Clone for _Inner { fn clone(&self) -> Self { - Self(self.0.clone()) + Self(Arc::clone(&self.0)) } } impl std::fmt::Debug for _Inner { @@ -555,7 +607,7 @@ pub mod server_server { write!(f, "{:?}", self.0) } } - impl tonic::transport::NamedService for ServerServer { + impl tonic::server::NamedService for ServerServer { const NAME: &'static str = "server_pb.Server"; } } diff --git a/integration_tests/feature-store/simulator/src/simulation.rs b/integration_tests/feature-store/simulator/src/simulation.rs index 96f3c31921a5e..8a693e9227809 100644 --- a/integration_tests/feature-store/simulator/src/simulation.rs +++ b/integration_tests/feature-store/simulator/src/simulation.rs @@ -4,7 +4,6 @@ use std::thread::sleep; use std::time::Duration; use futures::future::join_all; -use rand; use rand::Rng; use tokio::sync::Mutex; use tonic::transport::Channel; @@ -14,8 +13,7 @@ use crate::server_pb::StartTrainingRequest; use crate::{entities, entities_taxi}; fn get_delay_mills(delay_val: f64) -> u64 { - let turbulence = - rand::thread_rng().gen_range((delay_val * 0.6) as f64, (delay_val * 1.1) as f64) as f64; + let turbulence = rand::thread_rng().gen_range((delay_val * 0.6)..(delay_val * 1.1)); (turbulence * 10000.0) as u64 } @@ -33,7 +31,7 @@ pub async fn main_loop(simulator_type: String) { } } -async fn mock_taxi(client: Arc>>) -> () { +async fn mock_taxi(client: Arc>>) { let (offline_features, online_features) = entities_taxi::parse_taxi_metadata(); println!("Write training data len is {:?}", offline_features.len()); let mut threads = vec![]; @@ -87,7 +85,7 @@ async fn mock_taxi(client: Arc>>) -> () { } #[allow(dead_code)] -async fn mock_user_mfa(client: Arc>>) -> () { +async fn mock_user_mfa(client: Arc>>) { let users = entities::parse_user_metadata().unwrap(); let mut threads = vec![]; for user in users { From 878ebac88d6af6df286440e50df2b453960dc2ca Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Wed, 18 Oct 2023 17:35:11 +0800 Subject: [PATCH 09/58] feat(ctl): support rebuild table stats with approximate value (#12946) --- proto/hummock.proto | 5 + src/ctl/src/cmd_impl/hummock/list_version.rs | 6 + src/ctl/src/lib.rs | 5 + src/meta/service/src/hummock_service.rs | 8 + src/meta/src/hummock/manager/versioning.rs | 156 ++++++++++++++++++- src/rpc_client/src/meta_client.rs | 7 + 6 files changed, 183 insertions(+), 4 deletions(-) diff --git a/proto/hummock.proto b/proto/hummock.proto index 6957d7e060524..1b557ca5fb31b 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -681,6 +681,10 @@ message ListHummockMetaConfigResponse { map configs = 1; } +message RiseCtlRebuildTableStatsRequest {} + +message RiseCtlRebuildTableStatsResponse {} + service HummockManagerService { rpc UnpinVersionBefore(UnpinVersionBeforeRequest) returns (UnpinVersionBeforeResponse); rpc GetCurrentVersion(GetCurrentVersionRequest) returns (GetCurrentVersionResponse); @@ -706,6 +710,7 @@ service HummockManagerService { rpc RiseCtlPauseVersionCheckpoint(RiseCtlPauseVersionCheckpointRequest) returns (RiseCtlPauseVersionCheckpointResponse); rpc RiseCtlResumeVersionCheckpoint(RiseCtlResumeVersionCheckpointRequest) returns (RiseCtlResumeVersionCheckpointResponse); rpc RiseCtlGetCheckpointVersion(RiseCtlGetCheckpointVersionRequest) returns (RiseCtlGetCheckpointVersionResponse); + rpc RiseCtlRebuildTableStats(RiseCtlRebuildTableStatsRequest) returns (RiseCtlRebuildTableStatsResponse); rpc InitMetadataForReplay(InitMetadataForReplayRequest) returns (InitMetadataForReplayResponse); rpc PinVersion(PinVersionRequest) returns (PinVersionResponse); rpc SplitCompactionGroup(SplitCompactionGroupRequest) returns (SplitCompactionGroupResponse); diff --git a/src/ctl/src/cmd_impl/hummock/list_version.rs b/src/ctl/src/cmd_impl/hummock/list_version.rs index 6935dcf604142..3973860d9e30e 100644 --- a/src/ctl/src/cmd_impl/hummock/list_version.rs +++ b/src/ctl/src/cmd_impl/hummock/list_version.rs @@ -148,3 +148,9 @@ pub async fn list_pinned_snapshots(context: &CtlContext) -> anyhow::Result<()> { } Ok(()) } + +pub async fn rebuild_table_stats(context: &CtlContext) -> anyhow::Result<()> { + let meta_client = context.meta_client().await?; + meta_client.risectl_rebuild_table_stats().await?; + Ok(()) +} diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index 4908e8cdb952e..cf194884d52f3 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -246,6 +246,8 @@ enum HummockCommands { }, /// Validate the current HummockVersion. ValidateVersion, + /// Rebuild table stats + RebuildTableStats, } #[derive(Subcommand)] @@ -608,6 +610,9 @@ pub async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { Commands::Hummock(HummockCommands::ValidateVersion) => { cmd_impl::hummock::validate_version(context).await?; } + Commands::Hummock(HummockCommands::RebuildTableStats) => { + cmd_impl::hummock::rebuild_table_stats(context).await?; + } Commands::Table(TableCommands::Scan { mv_name, data_dir }) => { cmd_impl::table::scan(context, mv_name, data_dir).await? } diff --git a/src/meta/service/src/hummock_service.rs b/src/meta/service/src/hummock_service.rs index 74dc37b82d21e..56bb78f19249f 100644 --- a/src/meta/service/src/hummock_service.rs +++ b/src/meta/service/src/hummock_service.rs @@ -594,6 +594,14 @@ impl HummockManagerService for HummockServiceImpl { ); Ok(Response::new(ListHummockMetaConfigResponse { configs })) } + + async fn rise_ctl_rebuild_table_stats( + &self, + _request: Request, + ) -> Result, Status> { + self.hummock_manager.rebuild_table_stats().await?; + Ok(Response::new(RiseCtlRebuildTableStatsResponse {})) + } } #[cfg(test)] diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 1e939513bbf3d..e1ed8a5d716c2 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -23,6 +23,7 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ HummockVersionExt, }; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; +use risingwave_hummock_sdk::table_stats::add_prost_table_stats_map; use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockSstableObjectId, HummockVersionId, FIRST_VERSION_ID, }; @@ -30,15 +31,18 @@ use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ CompactionConfig, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersion, - HummockVersionCheckpoint, HummockVersionDelta, HummockVersionStats, + HummockVersionCheckpoint, HummockVersionDelta, HummockVersionStats, SstableInfo, TableStats, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; +use crate::hummock::error::Result; use crate::hummock::manager::worker::{HummockManagerEvent, HummockManagerEventSender}; -use crate::hummock::manager::{read_lock, write_lock}; +use crate::hummock::manager::{commit_multi_var, read_lock, write_lock}; use crate::hummock::metrics_utils::{trigger_safepoint_stat, trigger_write_stop_stats}; use crate::hummock::model::CompactionGroup; use crate::hummock::HummockManager; +use crate::model::{ValTransaction, VarTransaction}; +use crate::storage::Transaction; /// `HummockVersionSafePoint` prevents hummock versions GE than it from being GC. /// It's used by meta node itself to temporarily pin versions. @@ -277,6 +281,16 @@ impl HummockManager { let guard = read_lock!(self, versioning).await; guard.branched_ssts.clone() } + + #[named] + pub async fn rebuild_table_stats(&self) -> Result<()> { + let mut versioning = write_lock!(self, versioning).await; + let new_stats = rebuild_table_stats(&versioning.current_version); + let mut version_stats = VarTransaction::new(&mut versioning.version_stats); + *version_stats = new_stats; + commit_multi_var!(self, None, Transaction::default(), version_stats)?; + Ok(()) + } } /// Calculates write limits for `target_groups`. @@ -338,6 +352,47 @@ pub(super) fn create_init_version(default_compaction_config: CompactionConfig) - init_version } +/// Rebuilds table stats from the given version. +/// Note that the result is approximate value. See `estimate_table_stats`. +fn rebuild_table_stats(version: &HummockVersion) -> HummockVersionStats { + let mut stats = HummockVersionStats { + hummock_version_id: version.id, + table_stats: Default::default(), + }; + for level in version.get_combined_levels() { + for sst in &level.table_infos { + let changes = estimate_table_stats(sst); + add_prost_table_stats_map(&mut stats.table_stats, &changes); + } + } + stats +} + +/// Estimates table stats change from the given file. +/// - The file stats is evenly distributed among multiple tables within the file. +/// - The total key size and total value size are estimated based on key range and file size. +/// - Branched files may lead to an overestimation. +fn estimate_table_stats(sst: &SstableInfo) -> HashMap { + let mut changes: HashMap = HashMap::default(); + let weighted_value = + |value: i64| -> i64 { (value as f64 / sst.table_ids.len() as f64).ceil() as i64 }; + let key_range = sst.key_range.as_ref().unwrap(); + let estimated_key_size: u64 = (key_range.left.len() + key_range.right.len()) as u64 / 2; + let mut estimated_total_key_size = estimated_key_size * sst.total_key_count; + if estimated_total_key_size > sst.uncompressed_file_size { + estimated_total_key_size = sst.uncompressed_file_size / 2; + tracing::warn!(sst.sst_id, "Calculated estimated_total_key_size {} > uncompressed_file_size {}. Use uncompressed_file_size/2 as estimated_total_key_size instead.", estimated_total_key_size, sst.uncompressed_file_size); + } + let estimated_total_value_size = sst.uncompressed_file_size - estimated_total_key_size; + for table_id in &sst.table_ids { + let e = changes.entry(*table_id).or_default(); + e.total_key_count += weighted_value(sst.total_key_count as i64); + e.total_key_size += weighted_value(estimated_total_key_size as i64); + e.total_value_size += weighted_value(estimated_total_value_size as i64); + } + changes +} + #[cfg(test)] mod tests { use std::collections::HashMap; @@ -346,10 +401,15 @@ mod tests { use risingwave_hummock_sdk::{CompactionGroupId, HummockVersionId}; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::write_limits::WriteLimit; - use risingwave_pb::hummock::{HummockPinnedVersion, HummockVersion, Level, OverlappingLevel}; + use risingwave_pb::hummock::{ + HummockPinnedVersion, HummockVersion, HummockVersionStats, KeyRange, Level, + OverlappingLevel, SstableInfo, + }; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; - use crate::hummock::manager::versioning::{calc_new_write_limits, Versioning}; + use crate::hummock::manager::versioning::{ + calc_new_write_limits, estimate_table_stats, rebuild_table_stats, Versioning, + }; use crate::hummock::model::CompactionGroup; #[test] @@ -470,4 +530,92 @@ mod tests { "too many L0 sub levels: 11 > 5" ); } + + #[test] + fn test_estimate_table_stats() { + let sst = SstableInfo { + key_range: Some(KeyRange { + left: vec![1; 10], + right: vec![1; 20], + ..Default::default() + }), + table_ids: vec![1, 2, 3], + total_key_count: 6000, + uncompressed_file_size: 6_000_000, + ..Default::default() + }; + let changes = estimate_table_stats(&sst); + assert_eq!(changes.len(), 3); + for stats in changes.values() { + assert_eq!(stats.total_key_count, 6000 / 3); + assert_eq!(stats.total_key_size, (10 + 20) / 2 * 6000 / 3); + assert_eq!( + stats.total_value_size, + (6_000_000 - (10 + 20) / 2 * 6000) / 3 + ); + } + + let mut version = HummockVersion { + id: 123, + levels: Default::default(), + max_committed_epoch: 0, + safe_epoch: 0, + }; + for cg in 1..3 { + version.levels.insert( + cg, + Levels { + levels: vec![Level { + table_infos: vec![sst.clone()], + ..Default::default() + }], + l0: Some(Default::default()), + ..Default::default() + }, + ); + } + let HummockVersionStats { + hummock_version_id, + table_stats, + } = rebuild_table_stats(&version); + assert_eq!(hummock_version_id, version.id); + assert_eq!(table_stats.len(), 3); + for (tid, stats) in table_stats { + assert_eq!( + stats.total_key_count, + changes.get(&tid).unwrap().total_key_count * 2 + ); + assert_eq!( + stats.total_key_size, + changes.get(&tid).unwrap().total_key_size * 2 + ); + assert_eq!( + stats.total_value_size, + changes.get(&tid).unwrap().total_value_size * 2 + ); + } + } + + #[test] + fn test_estimate_table_stats_large_key_range() { + let sst = SstableInfo { + key_range: Some(KeyRange { + left: vec![1; 1000], + right: vec![1; 2000], + ..Default::default() + }), + table_ids: vec![1, 2, 3], + total_key_count: 6000, + uncompressed_file_size: 60_000, + ..Default::default() + }; + let changes = estimate_table_stats(&sst); + assert_eq!(changes.len(), 3); + for t in &sst.table_ids { + let stats = changes.get(t).unwrap(); + assert_eq!(stats.total_key_count, 6000 / 3); + assert_eq!(stats.total_key_size, 60_000 / 2 / 3); + assert_eq!(stats.total_value_size, (60_000 - 60_000 / 2) / 3); + } + } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 74a80f8e9f3e6..73664b6540a45 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -1045,6 +1045,12 @@ impl MetaClient { )) } + pub async fn risectl_rebuild_table_stats(&self) -> Result<()> { + let req = RiseCtlRebuildTableStatsRequest {}; + let _resp = self.inner.rise_ctl_rebuild_table_stats(req).await?; + Ok(()) + } + pub async fn list_branched_object(&self) -> Result> { let req = ListBranchedObjectRequest {}; let resp = self.inner.list_branched_object(req).await?; @@ -1728,6 +1734,7 @@ macro_rules! for_all_meta_rpc { ,{ hummock_client, init_metadata_for_replay, InitMetadataForReplayRequest, InitMetadataForReplayResponse } ,{ hummock_client, split_compaction_group, SplitCompactionGroupRequest, SplitCompactionGroupResponse } ,{ hummock_client, rise_ctl_list_compaction_status, RiseCtlListCompactionStatusRequest, RiseCtlListCompactionStatusResponse } + ,{ hummock_client, rise_ctl_rebuild_table_stats, RiseCtlRebuildTableStatsRequest, RiseCtlRebuildTableStatsResponse } ,{ hummock_client, subscribe_compaction_event, impl tonic::IntoStreamingRequest, Streaming } ,{ hummock_client, list_branched_object, ListBranchedObjectRequest, ListBranchedObjectResponse } ,{ hummock_client, list_active_write_limit, ListActiveWriteLimitRequest, ListActiveWriteLimitResponse } From 399cbc07c422a32022abab43d8aa7ba961afc942 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Wed, 18 Oct 2023 05:31:05 -0500 Subject: [PATCH 10/58] feat(dashboard): dump await-tree for all compute nodes (#12947) Signed-off-by: Bugen Zhao --- Cargo.lock | 4 +- Cargo.toml | 2 +- Makefile.toml | 1 + dashboard/pages/await_tree.tsx | 31 +++++-- .../src/rpc/service/monitor_service.rs | 12 +-- src/ctl/src/cmd_impl.rs | 2 +- .../src/cmd_impl/{trace.rs => await_tree.rs} | 88 +++++++++---------- src/ctl/src/lib.rs | 7 +- src/meta/src/dashboard/mod.rs | 43 +++++++-- src/prost/build.rs | 5 ++ src/storage/compactor/src/rpc.rs | 3 +- 11 files changed, 121 insertions(+), 77 deletions(-) rename src/ctl/src/cmd_impl/{trace.rs => await_tree.rs} (50%) diff --git a/Cargo.lock b/Cargo.lock index 247852213c422..a337219231016 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4587,9 +4587,9 @@ dependencies = [ [[package]] name = "madsim-tonic-build" -version = "0.4.1+0.10.0" +version = "0.4.2+0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3434b3d59001abcce56b9170fbd7982377858d8b931e8472056bf0c894ab257" +checksum = "4a2ad2776ba20221ccbe4e136e2fa0f7ab90eebd608373177f3e74a198a288ec" dependencies = [ "prettyplease 0.2.15", "proc-macro2", diff --git a/Cargo.toml b/Cargo.toml index 74a097d4eb9d7..ef09221b818a2 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -109,7 +109,7 @@ hashbrown = { version = "0.14.0", features = [ ] } criterion = { version = "0.5", features = ["async_futures"] } tonic = { package = "madsim-tonic", version = "0.4.0" } -tonic-build = { package = "madsim-tonic-build", version = "0.4.0" } +tonic-build = { package = "madsim-tonic-build", version = "0.4.2" } prost = { version = "0.12" } icelake = { git = "https://github.com/icelake-io/icelake", rev = "16dab0e36ab337e58ee8002d828def2d212fa116" } arrow-array = "47" diff --git a/Makefile.toml b/Makefile.toml index a0c6213a63c82..1203a847e1e94 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -406,6 +406,7 @@ condition = { env_set = [ "ENABLE_BUILD_DASHBOARD", ], files_modified = { input = [ "./dashboard/**/*.js", + "./dashboard/**/*.ts*", "./dashboard/package.json", "./dashboard/next.config.js", ], output = [ diff --git a/dashboard/pages/await_tree.tsx b/dashboard/pages/await_tree.tsx index 8908e121deabd..3db6b6677274a 100644 --- a/dashboard/pages/await_tree.tsx +++ b/dashboard/pages/await_tree.tsx @@ -36,22 +36,32 @@ import { getClusterInfoComputeNode } from "./api/cluster" import useFetch from "./api/fetch" const SIDEBAR_WIDTH = 200 +const ALL_COMPUTE_NODES = "" export default function AwaitTreeDump() { const { response: computeNodes } = useFetch(getClusterInfoComputeNode) - const [computeNodeId, setComputeNodeId] = useState() - const [dump, setDump] = useState("") + const [computeNodeId, setComputeNodeId] = useState() + const [dump, setDump] = useState("") useEffect(() => { - if (computeNodes && !computeNodeId && computeNodes.length > 0) { - setComputeNodeId(computeNodes[0].id) + if (computeNodes && !computeNodeId) { + setComputeNodeId(ALL_COMPUTE_NODES) } }, [computeNodes, computeNodeId]) const dumpTree = async () => { - const title = `Await-Tree Dump of Compute Node ${computeNodeId}:` - setDump(undefined) + if (computeNodeId === undefined) { + return + } + + let title + if (computeNodeId === ALL_COMPUTE_NODES) { + title = "Await-Tree Dump of All Compute Nodes:" + } else { + title = `Await-Tree Dump of Compute Node ${computeNodeId}:` + } + setDump("Loading...") let result @@ -92,10 +102,13 @@ export default function AwaitTreeDump() { Compute Nodes