From 440c09324686bcd768f095ced72b1601369f536d Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Fri, 25 Aug 2023 16:47:57 +0300 Subject: [PATCH 01/15] chore(cubestore): Job processing refactoring --- .../src/cluster/ingestion/job_processor.rs | 209 +++++++++ .../src/cluster/ingestion/job_runner.rs | 401 ++++++++++++++++++ .../cubestore/src/cluster/ingestion/mod.rs | 3 + .../cubestore/src/cluster/ingestion/worker.rs | 46 ++ rust/cubestore/cubestore/src/cluster/mod.rs | 391 +---------------- .../cubestore/src/cluster/worker_pool.rs | 9 +- 6 files changed, 683 insertions(+), 376 deletions(-) create mode 100644 rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs create mode 100644 rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs create mode 100644 rust/cubestore/cubestore/src/cluster/ingestion/mod.rs create mode 100644 rust/cubestore/cubestore/src/cluster/ingestion/worker.rs diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs b/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs new file mode 100644 index 0000000000000..68e0d5fcdd2af --- /dev/null +++ b/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs @@ -0,0 +1,209 @@ +#[cfg(not(target_os = "windows"))] +use crate::cluster::ingestion::worker::{IngestionWorkerMessage, IngestionWorkerProcessor}; +#[cfg(not(target_os = "windows"))] +use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; +use crate::config::{Config, ConfigObj}; +use crate::import::ImportService; +use crate::metastore::job::{Job, JobStatus, JobType}; +use crate::metastore::table::Table; +use crate::metastore::{MetaStore, RowKey, TableId}; +use crate::queryplanner::trace_data_loaded::DataLoadedSize; +use crate::store::compaction::CompactionService; +use crate::CubeError; +use serde::{Deserialize, Serialize}; +use std::sync::Arc; +use std::time::Duration; + +#[derive(Clone, Serialize, Deserialize, Debug)] +pub struct JobProcessResult { + data_loaded_size: usize, +} + +impl JobProcessResult { + pub fn new(data_loaded_size: usize) -> Self { + Self { data_loaded_size } + } + + pub fn data_loaded_size(&self) -> usize { + self.data_loaded_size + } +} + +impl Default for JobProcessResult { + fn default() -> Self { + Self { + data_loaded_size: 0, + } + } +} + +#[cfg(not(target_os = "windows"))] +pub struct JobProcessor { + process_pool: + Arc>, +} + +#[cfg(not(target_os = "windows"))] +impl JobProcessor { + pub fn new(pool_size: usize, timeout: Duration) -> Arc { + Arc::new(Self { + process_pool: Arc::new(WorkerPool::new(pool_size, timeout)), + }) + } + pub async fn wait_processing_loops(&self) { + self.process_pool.wait_processing_loops().await + } + pub async fn stop_processing_loops(&self) -> Result<(), CubeError> { + self.process_pool.stop_workers().await + } + pub async fn process_job(&self, job: Job) -> Result { + println!("^^^^^^^ "); + self.process_pool.process(IngestionWorkerMessage::Job(job)).await + } +} + +//TODO +#[cfg(target_os = "windows")] +pub struct JobProcessor { + process_pool: + Arc>, +} + +#[cfg(target_os = "windows")] +impl JobProcessor { + pub fn new(pool_size: usize, timeout: Duration) -> Arc { + Arc::new(Self { + process_pool: Arc::new(WorkerPool::new(pool_size, timeout)), + }) + } + pub async fn wait_processing_loops(&self) { + self.process_pool.wait_processing_loops().await + } + pub async fn stop_processing_loops(&self) -> Result<(), CubeError> { + self.process_pool.stop_workers().await + } + pub async fn process_job(&self, job: Job) -> Result { + self.process_pool.process(IngestionWorkerMessage::Job(job)).await + } +} + +pub struct JobIsolatedProcessor { + config_obj: Arc, + meta_store: Arc, + compaction_service: Arc, + import_service: Arc, +} + +impl JobIsolatedProcessor { + pub fn new( + config_obj: Arc, + meta_store: Arc, + compaction_service: Arc, + import_service: Arc, + ) -> Arc { + Arc::new(Self { + config_obj, + meta_store, + compaction_service, + import_service, + }) + } + + pub async fn new_from_config(config: &Config) -> Arc { + Self::new( + config.config_obj(), + config.injector().get_service_typed().await, + config.injector().get_service_typed().await, + config.injector().get_service_typed().await, + ) + } + + pub async fn process_separate_job(&self, job: &Job) -> Result { + match job.job_type() { + JobType::PartitionCompaction => { + if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { + let compaction_service = self.compaction_service.clone(); + let partition_id = *partition_id; + let data_loaded_size = DataLoadedSize::new(); + compaction_service + .compact(partition_id, data_loaded_size.clone()) + .await?; + Ok(JobProcessResult::new(data_loaded_size.get())) + } else { + Self::fail_job_row_key(job) + } + } + JobType::MultiPartitionSplit => { + if let RowKey::Table(TableId::MultiPartitions, id) = job.row_reference() { + let compaction_service = self.compaction_service.clone(); + let id = *id; + compaction_service.split_multi_partition(id).await?; + Ok(JobProcessResult::default()) + } else { + Self::fail_job_row_key(job) + } + } + JobType::FinishMultiSplit => { + if let RowKey::Table(TableId::MultiPartitions, multi_part_id) = job.row_reference() + { + let meta_store = self.meta_store.clone(); + let compaction_service = self.compaction_service.clone(); + let multi_part_id = *multi_part_id; + for p in meta_store.find_unsplit_partitions(multi_part_id).await? { + compaction_service + .finish_multi_split(multi_part_id, p) + .await? + } + + Ok(JobProcessResult::default()) + } else { + Self::fail_job_row_key(job) + } + } + JobType::TableImport => { + if let RowKey::Table(TableId::Tables, table_id) = job.row_reference() { + let import_service = self.import_service.clone(); + let table_id = *table_id; + import_service.import_table(table_id).await?; + Ok(JobProcessResult::default()) + } else { + Self::fail_job_row_key(job) + } + } + JobType::TableImportCSV(location) => { + if Table::is_stream_location(&location) { + return Err(CubeError::internal( + "Streaming import cannot be processed in separate process".to_string(), + )); + } + if let RowKey::Table(TableId::Tables, table_id) = job.row_reference() { + let table_id = *table_id; + let import_service = self.import_service.clone(); + let location = location.to_string(); + let data_loaded_size = Some(DataLoadedSize::new()); + import_service + .clone() + .import_table_part(table_id, &location, data_loaded_size.clone()) + .await?; + Ok(JobProcessResult::new( + data_loaded_size.map_or(0, |d| d.get()), + )) + } else { + Self::fail_job_row_key(job) + } + } + _ => Err(CubeError::internal(format!( + "Job {:?} cannot be processed in separate process", + job.job_type() + ))), + } + } + + fn fail_job_row_key(job: &Job) -> Result { + Err(CubeError::internal(format!( + "Incorrect row key for {:?}: {:?}", + job, + job.row_reference() + ))) + } +} diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs b/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs new file mode 100644 index 0000000000000..4c642c8f20982 --- /dev/null +++ b/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs @@ -0,0 +1,401 @@ +use crate::config::{Config, ConfigObj}; +use crate::metastore::{ + deactivate_table_on_corrupt_data, Chunk, IdRow, MetaStore, MetaStoreEvent, Partition, RowKey, + TableId, +}; +use crate::metastore::job::{Job, JobStatus, JobType}; +use crate::import::ImportService; +use crate::store::ChunkDataStore; +use crate::store::compaction::CompactionService; +use tokio_util::sync::CancellationToken; +use crate::queryplanner::trace_data_loaded::DataLoadedSize; +use crate::metastore::table::Table; +use tokio::time::timeout; +use tokio::task::JoinHandle; +use crate::util::aborting_join_handle::AbortingJoinHandle; +use std::time::SystemTime; +use crate::cluster::rate_limiter::{ProcessRateLimiter, TaskType, TraceIndex}; +use futures_timer::Delay; +use std::time::Duration; +use crate::CubeError; +use std::sync::Arc; +use log::{debug, error, info, warn}; +use tokio::sync::{oneshot, watch, Notify, RwLock}; +use core::mem; +use datafusion::cube_ext; +use crate::cluster::ingestion::job_processor::JobProcessor; + +pub struct JobRunner { + pub config_obj: Arc, + pub meta_store: Arc, + pub chunk_store: Arc, + pub compaction_service: Arc, + pub import_service: Arc, + pub process_rate_limiter: Arc, + pub server_name: String, + pub notify: Arc, + pub stop_token: CancellationToken, + pub is_long_term: bool, + pub job_processor: Arc, +} + +impl JobRunner { + pub async fn processing_loop(&self) { + loop { + let res = tokio::select! { + _ = self.stop_token.cancelled() => { + let _ = self.job_processor.stop_processing_loops().await; + return; + } + _ = self.notify.notified() => { + self.fetch_and_process().await + } + }; + if let Err(e) = res { + error!("Error in processing loop: {}", e); + } + } + } + + async fn fetch_and_process(&self) -> Result<(), CubeError> { + let job = self + .meta_store + .start_processing_job(self.server_name.to_string(), self.is_long_term) + .await?; + if let Some(to_process) = job { + self.run_local(to_process).await?; + // In case of job queue is in place jump to the next job immediately + self.notify.notify_one(); + } + Ok(()) + } + + fn job_timeout(&self, job: &IdRow) -> Option { + if let JobType::TableImportCSV(location) = job.get_row().job_type() { + if Table::is_stream_location(location) { + return None; + } + } + Some(Duration::from_secs(self.config_obj.import_job_timeout())) + } + + async fn run_local(&self, job: IdRow) -> Result<(), CubeError> { + let start = SystemTime::now(); + let job_id = job.get_id(); + let (mut tx, rx) = oneshot::channel::<()>(); + let meta_store = self.meta_store.clone(); + let heart_beat_timer = cube_ext::spawn(async move { + loop { + tokio::select! { + _ = tx.closed() => { + break; + } + _ = Delay::new(Duration::from_secs(30)) => { + let _ = meta_store.update_heart_beat(job_id).await; // TODO handle result + } + } + } + }); + debug!("Running job: {:?}", job); + let handle = AbortingJoinHandle::new(self.route_job(job.get_row())?); + // TODO cancel job if this worker isn't job owner anymore + let res = if let Some(duration) = self.job_timeout(&job) { + let future = timeout(duration, handle); + // TODO duplicate + tokio::select! { + _ = self.stop_token.cancelled() => { + Err(CubeError::user("shutting down".to_string())) + } + res = future => { + res.map_err(|_| CubeError::user("timed out".to_string())) + } + } + } else { + // TODO duplicate + tokio::select! { + _ = self.stop_token.cancelled() => { + Err(CubeError::user("shutting down".to_string())) + } + res = handle => { + Ok(res) + } + } + }; + + mem::drop(rx); + heart_beat_timer.await?; + if let Err(e) = res { + self.meta_store + .update_status(job_id, JobStatus::Timeout) + .await?; + error!( + "Running job {} ({:?}): {:?}", + e.message, + start.elapsed()?, + // Job can be removed by the time of fetch + self.meta_store.get_job(job_id).await.unwrap_or(job) + ); + } else if let Ok(Err(cube_err)) = res { + self.meta_store + .update_status(job_id, JobStatus::Error(cube_err.to_string())) + .await?; + error!( + "Running job join error ({:?}): {:?}", + start.elapsed()?, + // Job can be removed by the time of fetch + self.meta_store.get_job(job_id).await.unwrap_or(job) + ); + } else if let Ok(Ok(Err(cube_err))) = res { + self.meta_store + .update_status(job_id, JobStatus::Error(cube_err.to_string())) + .await?; + error!( + "Error while running job {}: {}", + job_id, + cube_err.display_with_backtrace() + ); + error!( + "Running job error ({:?}): {:?}", + start.elapsed()?, + // Job can be removed by the time of fetch + self.meta_store.get_job(job_id).await.unwrap_or(job) + ); + } else { + let job = self + .meta_store + .update_status(job_id, JobStatus::Completed) + .await?; + info!("Running job completed ({:?}): {:?}", start.elapsed()?, job); + // TODO delete jobs on reconciliation + self.meta_store.delete_job(job_id).await?; + } + Ok(()) + } + + fn route_job(&self, job: &Job) -> Result>, CubeError> { + // spawn here is required in case there's a panic in a job. If job panics worker process loop will survive it. + match job.job_type() { + JobType::WalPartitioning => { + if let RowKey::Table(TableId::WALs, wal_id) = job.row_reference() { + let chunk_store = self.chunk_store.clone(); + let wal_id = *wal_id; + Ok(cube_ext::spawn(async move { + chunk_store.partition(wal_id).await + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::Repartition => { + if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { + let chunk_store = self.chunk_store.clone(); + let partition_id = *partition_id; + Ok(cube_ext::spawn(async move { + chunk_store.repartition(partition_id).await + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::PartitionCompaction => { + if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { + let compaction_service = self.compaction_service.clone(); + let partition_id = *partition_id; + let process_rate_limiter = self.process_rate_limiter.clone(); + let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); + let metastore = self.meta_store.clone(); + let job_processor = self.job_processor.clone(); + let job_to_move = job.clone(); + Ok(cube_ext::spawn(async move { + process_rate_limiter + .wait_for_allow(TaskType::Job, timeout) + .await?; //TODO config, may be same ad orphaned timeout + + let (_, _, table, _) = + metastore.get_partition_for_compaction(partition_id).await?; + let table_id = table.get_id(); + let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; + let trace_index = TraceIndex { + table_id: Some(table_id), + trace_obj, + }; + + let data_loaded_size = DataLoadedSize::new(); + let res = job_processor.process_job(job_to_move).await; + if let Ok(job_res) = res { + process_rate_limiter + .commit_task_usage( + TaskType::Job, + job_res.data_loaded_size() as i64, + trace_index, + ) + .await; + } + Ok(()) + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::InMemoryChunksCompaction => { + if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { + let compaction_service = self.compaction_service.clone(); + let partition_id = *partition_id; + log::warn!( + "JobType::InMemoryChunksCompaction is deprecated and should not be used" + ); + Ok(cube_ext::spawn(async move { + compaction_service + .compact_in_memory_chunks(partition_id) + .await + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::NodeInMemoryChunksCompaction(_) => { + if let RowKey::Table(TableId::Tables, _) = job.row_reference() { + let compaction_service = self.compaction_service.clone(); + let node_name = self.server_name.clone(); + Ok(cube_ext::spawn(async move { + compaction_service + .compact_node_in_memory_chunks(node_name) + .await + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::MultiPartitionSplit => { + if let RowKey::Table(TableId::MultiPartitions, id) = job.row_reference() { + let compaction_service = self.compaction_service.clone(); + let id = *id; + Ok(cube_ext::spawn(async move { + compaction_service.split_multi_partition(id).await + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::FinishMultiSplit => { + if let RowKey::Table(TableId::MultiPartitions, multi_part_id) = job.row_reference() + { + let meta_store = self.meta_store.clone(); + let compaction_service = self.compaction_service.clone(); + let multi_part_id = *multi_part_id; + Ok(cube_ext::spawn(async move { + for p in meta_store.find_unsplit_partitions(multi_part_id).await? { + compaction_service + .finish_multi_split(multi_part_id, p) + .await? + } + Ok(()) + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::TableImport => { + if let RowKey::Table(TableId::Tables, table_id) = job.row_reference() { + let import_service = self.import_service.clone(); + let table_id = *table_id; + Ok(cube_ext::spawn(async move { + import_service.import_table(table_id).await + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::TableImportCSV(location) => { + if let RowKey::Table(TableId::Tables, table_id) = job.row_reference() { + let table_id = *table_id; + let import_service = self.import_service.clone(); + let location = location.to_string(); + let process_rate_limiter = self.process_rate_limiter.clone(); + let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); + let metastore = self.meta_store.clone(); + Ok(cube_ext::spawn(async move { + let is_streaming = Table::is_stream_location(&location); + let data_loaded_size = if is_streaming { + None + } else { + Some(DataLoadedSize::new()) + }; + if !is_streaming { + process_rate_limiter + .wait_for_allow(TaskType::Job, timeout) + .await?; //TODO config, may be same ad orphaned timeout + } + let res = import_service + .clone() + .import_table_part(table_id, &location, data_loaded_size.clone()) + .await; + if let Some(data_loaded) = &data_loaded_size { + let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; + let trace_index = TraceIndex { + table_id: Some(table_id), + trace_obj, + }; + process_rate_limiter + .commit_task_usage( + TaskType::Job, + data_loaded.get() as i64, + trace_index, + ) + .await; + } + res + })) + } else { + Self::fail_job_row_key(job) + } + } + JobType::RepartitionChunk => { + if let RowKey::Table(TableId::Chunks, chunk_id) = job.row_reference() { + let chunk_store = self.chunk_store.clone(); + let chunk_id = *chunk_id; + let process_rate_limiter = self.process_rate_limiter.clone(); + let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); + let metastore = self.meta_store.clone(); + Ok(cube_ext::spawn(async move { + process_rate_limiter + .wait_for_allow(TaskType::Job, timeout) + .await?; //TODO config, may be same ad orphaned timeout + let chunk = metastore.get_chunk(chunk_id).await?; + let (_, _, table, _) = metastore + .get_partition_for_compaction(chunk.get_row().get_partition_id()) + .await?; + let table_id = table.get_id(); + let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; + let trace_index = TraceIndex { + table_id: Some(table_id), + trace_obj, + }; + let data_loaded_size = DataLoadedSize::new(); + let res = chunk_store + .repartition_chunk(chunk_id, data_loaded_size.clone()) + .await; + process_rate_limiter + .commit_task_usage( + TaskType::Job, + data_loaded_size.get() as i64, + trace_index, + ) + .await; + res + })) + } else { + Self::fail_job_row_key(job) + } + } + } + } + + fn fail_job_row_key(job: &Job) -> Result>, CubeError> { + Err(CubeError::internal(format!( + "Incorrect row key for {:?}: {:?}", + job, + job.row_reference() + ))) + } +} diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/mod.rs b/rust/cubestore/cubestore/src/cluster/ingestion/mod.rs new file mode 100644 index 0000000000000..5b4e7edb2b3f1 --- /dev/null +++ b/rust/cubestore/cubestore/src/cluster/ingestion/mod.rs @@ -0,0 +1,3 @@ +pub mod worker; +pub mod job_processor; +pub mod job_runner; diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/worker.rs b/rust/cubestore/cubestore/src/cluster/ingestion/worker.rs new file mode 100644 index 0000000000000..64f8dc3fe51ad --- /dev/null +++ b/rust/cubestore/cubestore/src/cluster/ingestion/worker.rs @@ -0,0 +1,46 @@ +use crate::cluster::ingestion::job_processor::{JobIsolatedProcessor, JobProcessResult}; +#[cfg(not(target_os = "windows"))] +use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; +use crate::config::Config; +use crate::metastore::job::{Job, JobStatus, JobType}; +use crate::CubeError; +use async_trait::async_trait; +use log::{debug, error}; +use serde::{Deserialize, Serialize}; +use std::time::SystemTime; + +#[derive(Debug, Serialize, Deserialize)] +pub enum IngestionWorkerMessage { + Job(Job), +} + +#[cfg(not(target_os = "windows"))] +pub struct IngestionWorkerProcessor; + +#[cfg(not(target_os = "windows"))] +#[async_trait] +impl MessageProcessor for IngestionWorkerProcessor { + async fn process( + config: &Config, + args: IngestionWorkerMessage, + ) -> Result { + let processor = JobIsolatedProcessor::new_from_config(config).await; + println!("!!!AAAAAAAAAAA!!!!!!"); + match args { + IngestionWorkerMessage::Job(job) => { + let processor_to_move = processor.clone(); + let future = async move { + let time = SystemTime::now(); + debug!("Running job in worker started"); + let res = processor_to_move.process_separate_job(&job).await; + debug!( + "Running job in worker completed ({:?})", + time.elapsed().unwrap() + ); + res + }; + future.await + } + } + } +} diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index d936f6910c1dd..f798ff34f0a50 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -6,6 +6,8 @@ pub mod worker_pool; pub mod rate_limiter; +mod ingestion; + #[cfg(not(target_os = "windows"))] use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; @@ -69,6 +71,8 @@ use tokio::task::JoinHandle; use tokio::time::timeout; use tokio_util::sync::CancellationToken; use tracing::{instrument, Instrument}; +use ingestion::job_runner::JobRunner; +use ingestion::job_processor::JobProcessor; #[automock] #[async_trait] @@ -226,9 +230,10 @@ impl MessageProcessor Result<(SchemaRef, Vec, usize), CubeError> { + let services = config.worker_services().await; match args { WorkerMessage::Select( plan_node, @@ -298,19 +303,6 @@ fn proc_handler() { ); } -struct JobRunner { - config_obj: Arc, - meta_store: Arc, - chunk_store: Arc, - compaction_service: Arc, - import_service: Arc, - process_rate_limiter: Arc, - server_name: String, - notify: Arc, - stop_token: CancellationToken, - is_long_term: bool, -} - lazy_static! { static ref HEART_BEAT_NODE_REGEX: Regex = Regex::new(r"^node-heart-beats/(?P.*)$").unwrap(); @@ -763,364 +755,6 @@ impl JobResultListener { } } -impl JobRunner { - async fn processing_loop(&self) { - loop { - let res = tokio::select! { - _ = self.stop_token.cancelled() => { - return; - } - _ = self.notify.notified() => { - self.fetch_and_process().await - } - }; - if let Err(e) = res { - error!("Error in processing loop: {}", e); - } - } - } - - async fn fetch_and_process(&self) -> Result<(), CubeError> { - let job = self - .meta_store - .start_processing_job(self.server_name.to_string(), self.is_long_term) - .await?; - if let Some(to_process) = job { - self.run_local(to_process).await?; - // In case of job queue is in place jump to the next job immediately - self.notify.notify_one(); - } - Ok(()) - } - - fn job_timeout(&self, job: &IdRow) -> Option { - if let JobType::TableImportCSV(location) = job.get_row().job_type() { - if Table::is_stream_location(location) { - return None; - } - } - Some(Duration::from_secs(self.config_obj.import_job_timeout())) - } - - async fn run_local(&self, job: IdRow) -> Result<(), CubeError> { - let start = SystemTime::now(); - let job_id = job.get_id(); - let (mut tx, rx) = oneshot::channel::<()>(); - let meta_store = self.meta_store.clone(); - let heart_beat_timer = cube_ext::spawn(async move { - loop { - tokio::select! { - _ = tx.closed() => { - break; - } - _ = Delay::new(Duration::from_secs(30)) => { - let _ = meta_store.update_heart_beat(job_id).await; // TODO handle result - } - } - } - }); - debug!("Running job: {:?}", job); - let handle = AbortingJoinHandle::new(self.route_job(job.get_row())?); - // TODO cancel job if this worker isn't job owner anymore - let res = if let Some(duration) = self.job_timeout(&job) { - let future = timeout(duration, handle); - // TODO duplicate - tokio::select! { - _ = self.stop_token.cancelled() => { - Err(CubeError::user("shutting down".to_string())) - } - res = future => { - res.map_err(|_| CubeError::user("timed out".to_string())) - } - } - } else { - // TODO duplicate - tokio::select! { - _ = self.stop_token.cancelled() => { - Err(CubeError::user("shutting down".to_string())) - } - res = handle => { - Ok(res) - } - } - }; - - mem::drop(rx); - heart_beat_timer.await?; - if let Err(e) = res { - self.meta_store - .update_status(job_id, JobStatus::Timeout) - .await?; - error!( - "Running job {} ({:?}): {:?}", - e.message, - start.elapsed()?, - // Job can be removed by the time of fetch - self.meta_store.get_job(job_id).await.unwrap_or(job) - ); - } else if let Ok(Err(cube_err)) = res { - self.meta_store - .update_status(job_id, JobStatus::Error(cube_err.to_string())) - .await?; - error!( - "Running job join error ({:?}): {:?}", - start.elapsed()?, - // Job can be removed by the time of fetch - self.meta_store.get_job(job_id).await.unwrap_or(job) - ); - } else if let Ok(Ok(Err(cube_err))) = res { - self.meta_store - .update_status(job_id, JobStatus::Error(cube_err.to_string())) - .await?; - error!( - "Error while running job {}: {}", - job_id, - cube_err.display_with_backtrace() - ); - error!( - "Running job error ({:?}): {:?}", - start.elapsed()?, - // Job can be removed by the time of fetch - self.meta_store.get_job(job_id).await.unwrap_or(job) - ); - } else { - let job = self - .meta_store - .update_status(job_id, JobStatus::Completed) - .await?; - info!("Running job completed ({:?}): {:?}", start.elapsed()?, job); - // TODO delete jobs on reconciliation - self.meta_store.delete_job(job_id).await?; - } - Ok(()) - } - - fn route_job(&self, job: &Job) -> Result>, CubeError> { - // spawn here is required in case there's a panic in a job. If job panics worker process loop will survive it. - match job.job_type() { - JobType::WalPartitioning => { - if let RowKey::Table(TableId::WALs, wal_id) = job.row_reference() { - let chunk_store = self.chunk_store.clone(); - let wal_id = *wal_id; - Ok(cube_ext::spawn(async move { - chunk_store.partition(wal_id).await - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::Repartition => { - if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { - let chunk_store = self.chunk_store.clone(); - let partition_id = *partition_id; - Ok(cube_ext::spawn(async move { - chunk_store.repartition(partition_id).await - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::PartitionCompaction => { - if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { - let compaction_service = self.compaction_service.clone(); - let partition_id = *partition_id; - let process_rate_limiter = self.process_rate_limiter.clone(); - let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); - let metastore = self.meta_store.clone(); - Ok(cube_ext::spawn(async move { - process_rate_limiter - .wait_for_allow(TaskType::Job, timeout) - .await?; //TODO config, may be same ad orphaned timeout - - let (_, _, table, _) = - metastore.get_partition_for_compaction(partition_id).await?; - let table_id = table.get_id(); - let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; - let trace_index = TraceIndex { - table_id: Some(table_id), - trace_obj, - }; - - let data_loaded_size = DataLoadedSize::new(); - let res = compaction_service - .compact(partition_id, data_loaded_size.clone()) - .await; - process_rate_limiter - .commit_task_usage( - TaskType::Job, - data_loaded_size.get() as i64, - trace_index, - ) - .await; - res - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::InMemoryChunksCompaction => { - if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { - let compaction_service = self.compaction_service.clone(); - let partition_id = *partition_id; - log::warn!( - "JobType::InMemoryChunksCompaction is deprecated and should not be used" - ); - Ok(cube_ext::spawn(async move { - compaction_service - .compact_in_memory_chunks(partition_id) - .await - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::NodeInMemoryChunksCompaction(_) => { - if let RowKey::Table(TableId::Tables, _) = job.row_reference() { - let compaction_service = self.compaction_service.clone(); - let node_name = self.server_name.clone(); - Ok(cube_ext::spawn(async move { - compaction_service - .compact_node_in_memory_chunks(node_name) - .await - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::MultiPartitionSplit => { - if let RowKey::Table(TableId::MultiPartitions, id) = job.row_reference() { - let compaction_service = self.compaction_service.clone(); - let id = *id; - Ok(cube_ext::spawn(async move { - compaction_service.split_multi_partition(id).await - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::FinishMultiSplit => { - if let RowKey::Table(TableId::MultiPartitions, multi_part_id) = job.row_reference() - { - let meta_store = self.meta_store.clone(); - let compaction_service = self.compaction_service.clone(); - let multi_part_id = *multi_part_id; - Ok(cube_ext::spawn(async move { - for p in meta_store.find_unsplit_partitions(multi_part_id).await? { - compaction_service - .finish_multi_split(multi_part_id, p) - .await? - } - Ok(()) - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::TableImport => { - if let RowKey::Table(TableId::Tables, table_id) = job.row_reference() { - let import_service = self.import_service.clone(); - let table_id = *table_id; - Ok(cube_ext::spawn(async move { - import_service.import_table(table_id).await - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::TableImportCSV(location) => { - if let RowKey::Table(TableId::Tables, table_id) = job.row_reference() { - let table_id = *table_id; - let import_service = self.import_service.clone(); - let location = location.to_string(); - let process_rate_limiter = self.process_rate_limiter.clone(); - let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); - let metastore = self.meta_store.clone(); - Ok(cube_ext::spawn(async move { - let is_streaming = Table::is_stream_location(&location); - let data_loaded_size = if is_streaming { - None - } else { - Some(DataLoadedSize::new()) - }; - if !is_streaming { - process_rate_limiter - .wait_for_allow(TaskType::Job, timeout) - .await?; //TODO config, may be same ad orphaned timeout - } - let res = import_service - .clone() - .import_table_part(table_id, &location, data_loaded_size.clone()) - .await; - if let Some(data_loaded) = &data_loaded_size { - let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; - let trace_index = TraceIndex { - table_id: Some(table_id), - trace_obj, - }; - process_rate_limiter - .commit_task_usage( - TaskType::Job, - data_loaded.get() as i64, - trace_index, - ) - .await; - } - res - })) - } else { - Self::fail_job_row_key(job) - } - } - JobType::RepartitionChunk => { - if let RowKey::Table(TableId::Chunks, chunk_id) = job.row_reference() { - let chunk_store = self.chunk_store.clone(); - let chunk_id = *chunk_id; - let process_rate_limiter = self.process_rate_limiter.clone(); - let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); - let metastore = self.meta_store.clone(); - Ok(cube_ext::spawn(async move { - process_rate_limiter - .wait_for_allow(TaskType::Job, timeout) - .await?; //TODO config, may be same ad orphaned timeout - let chunk = metastore.get_chunk(chunk_id).await?; - let (_, _, table, _) = metastore - .get_partition_for_compaction(chunk.get_row().get_partition_id()) - .await?; - let table_id = table.get_id(); - let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; - let trace_index = TraceIndex { - table_id: Some(table_id), - trace_obj, - }; - let data_loaded_size = DataLoadedSize::new(); - let res = chunk_store - .repartition_chunk(chunk_id, data_loaded_size.clone()) - .await; - process_rate_limiter - .commit_task_usage( - TaskType::Job, - data_loaded_size.get() as i64, - trace_index, - ) - .await; - res - })) - } else { - Self::fail_job_row_key(job) - } - } - } - } - - fn fail_job_row_key(job: &Job) -> Result>, CubeError> { - Err(CubeError::internal(format!( - "Incorrect row key for {:?}: {:?}", - job, - job.row_reference() - ))) - } -} - impl ClusterImpl { pub fn new( server_name: String, @@ -1194,6 +828,18 @@ impl ClusterImpl { async move { arc.wait_processing_loops().await }, )); } + #[cfg(not(target_os = "windows"))] + let job_processor = { + let job_processor = JobProcessor::new(self.config_obj.job_runners_count(), Duration::from_secs(600)); //TODO - timeout + let job_processor_to_move = job_processor.clone(); + futures.push(cube_ext::spawn( + async move { job_processor_to_move.wait_processing_loops().await }, + )); + println!("!!!!! ---- !!!!!!"); + job_processor + + }; + for i in 0..self.config_obj.job_runners_count() + self.config_obj.long_term_job_runners_count() @@ -1215,6 +861,7 @@ impl ClusterImpl { }, stop_token: self.stop_token.clone(), is_long_term: is_long_running, + job_processor: job_processor.clone(), }; futures.push(cube_ext::spawn(async move { job_runner.processing_loop().await; diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 3e7b4099cb68f..3d065edd2eeeb 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -53,7 +53,7 @@ pub trait MessageProcessor< R: Serialize + DeserializeOwned + Sync + Send + 'static, > { - async fn process(services: &WorkerServices, args: T) -> Result; + async fn process(config: &Config, args: T) -> Result; } impl< @@ -87,6 +87,7 @@ impl< } pub async fn wait_processing_loops(&self) { + println!("FFFFFFFF"); let futures = self .workers .iter() @@ -342,7 +343,6 @@ where worker_setup(&runtime); runtime.block_on(async move { let config = get_worker_config().await; - let services = config.worker_services().await; spawn_background_processes(config.clone()); @@ -351,7 +351,7 @@ where match res { Ok(args) => { let result = - match async_try_with_catch_unwind(P::process(&services, args)).await { + match async_try_with_catch_unwind(P::process(&config, args)).await { Ok(result) => result, Err(panic) => Err(CubeError::from(panic)), }; @@ -452,6 +452,7 @@ mod tests { use crate::util::respawn; use crate::CubeError; use datafusion::cube_ext; + use crate::config::Config; #[ctor::ctor] fn test_support_init() { @@ -474,7 +475,7 @@ mod tests { #[async_trait] impl MessageProcessor for Processor { - async fn process(_services: &WorkerServices, args: Message) -> Result { + async fn process(_config: &Config, args: Message) -> Result { match args { Message::Delay(x) => { Delay::new(Duration::from_millis(x)).await; From 744451bc06170c5842f0d5ad3f06d3216cee4f12 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Thu, 31 Aug 2023 12:15:07 +0300 Subject: [PATCH 02/15] in work --- .../src/cluster/ingestion/job_processor.rs | 73 ++++++++++++------- .../src/cluster/ingestion/job_runner.rs | 13 ++-- .../cubestore/src/cluster/ingestion/mod.rs | 1 - .../cubestore/src/cluster/ingestion/worker.rs | 46 ------------ rust/cubestore/cubestore/src/cluster/mod.rs | 23 +++--- .../cubestore/src/cluster/worker_pool.rs | 26 +++++-- rust/cubestore/cubestore/src/config/mod.rs | 12 +++ rust/cubestore/cubestore/src/sql/mod.rs | 3 +- .../cubestore/src/store/compaction.rs | 9 +++ rust/cubestore/cubestore/src/store/mod.rs | 11 +++ 10 files changed, 116 insertions(+), 101 deletions(-) delete mode 100644 rust/cubestore/cubestore/src/cluster/ingestion/worker.rs diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs b/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs index 68e0d5fcdd2af..7abdd546f5d25 100644 --- a/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs +++ b/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs @@ -1,18 +1,15 @@ -#[cfg(not(target_os = "windows"))] -use crate::cluster::ingestion::worker::{IngestionWorkerMessage, IngestionWorkerProcessor}; -#[cfg(not(target_os = "windows"))] -use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; +use crate::config::injection::DIService; use crate::config::{Config, ConfigObj}; use crate::import::ImportService; -use crate::metastore::job::{Job, JobStatus, JobType}; +use crate::metastore::job::{Job, JobType}; use crate::metastore::table::Table; use crate::metastore::{MetaStore, RowKey, TableId}; use crate::queryplanner::trace_data_loaded::DataLoadedSize; use crate::store::compaction::CompactionService; use crate::CubeError; +use async_trait::async_trait; use serde::{Deserialize, Serialize}; use std::sync::Arc; -use std::time::Duration; #[derive(Clone, Serialize, Deserialize, Debug)] pub struct JobProcessResult { @@ -37,31 +34,52 @@ impl Default for JobProcessResult { } } -#[cfg(not(target_os = "windows"))] -pub struct JobProcessor { - process_pool: - Arc>, +#[async_trait] +pub trait JobProcessor: DIService + Send + Sync { + async fn wait_processing_loops(&self); + async fn stop_processing_loops(&self) -> Result<(), CubeError>; + async fn process_job(&self, job: Job) -> Result; } -#[cfg(not(target_os = "windows"))] -impl JobProcessor { - pub fn new(pool_size: usize, timeout: Duration) -> Arc { +pub struct JobProcessorImpl { + processor: Arc, +} + +impl JobProcessorImpl { + pub fn new( + config_obj: Arc, + meta_store: Arc, + compaction_service: Arc, + import_service: Arc, + ) -> Arc { Arc::new(Self { - process_pool: Arc::new(WorkerPool::new(pool_size, timeout)), + processor: JobIsolatedProcessor::new( + config_obj, + meta_store, + compaction_service, + import_service, + ), }) } - pub async fn wait_processing_loops(&self) { - self.process_pool.wait_processing_loops().await - } - pub async fn stop_processing_loops(&self) -> Result<(), CubeError> { - self.process_pool.stop_workers().await +} + +#[async_trait] +impl JobProcessor for JobProcessorImpl { + async fn wait_processing_loops(&self) {} + + async fn stop_processing_loops(&self) -> Result<(), CubeError> { + Ok(()) } - pub async fn process_job(&self, job: Job) -> Result { - println!("^^^^^^^ "); - self.process_pool.process(IngestionWorkerMessage::Job(job)).await + + async fn process_job(&self, job: Job) -> Result { + self.processor + .process_separate_job(&job) + .await } } +crate::di_service!(JobProcessorImpl, [JobProcessor]); + //TODO #[cfg(target_os = "windows")] pub struct JobProcessor { @@ -83,7 +101,9 @@ impl JobProcessor { self.process_pool.stop_workers().await } pub async fn process_job(&self, job: Job) -> Result { - self.process_pool.process(IngestionWorkerMessage::Job(job)).await + self.process_pool + .process(IngestionWorkerMessage::Job(job)) + .await } } @@ -125,9 +145,12 @@ impl JobIsolatedProcessor { let compaction_service = self.compaction_service.clone(); let partition_id = *partition_id; let data_loaded_size = DataLoadedSize::new(); - compaction_service + println!("!!!! CCCCCCCC"); + let r = compaction_service .compact(partition_id, data_loaded_size.clone()) - .await?; + .await; + println!("!!!! FFFFFF"); + r?; Ok(JobProcessResult::new(data_loaded_size.get())) } else { Self::fail_job_row_key(job) diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs b/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs index 4c642c8f20982..53a00a105b776 100644 --- a/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs +++ b/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs @@ -1,7 +1,6 @@ -use crate::config::{Config, ConfigObj}; +use crate::config::ConfigObj; use crate::metastore::{ - deactivate_table_on_corrupt_data, Chunk, IdRow, MetaStore, MetaStoreEvent, Partition, RowKey, - TableId, + IdRow, MetaStore, RowKey, TableId, }; use crate::metastore::job::{Job, JobStatus, JobType}; use crate::import::ImportService; @@ -19,8 +18,8 @@ use futures_timer::Delay; use std::time::Duration; use crate::CubeError; use std::sync::Arc; -use log::{debug, error, info, warn}; -use tokio::sync::{oneshot, watch, Notify, RwLock}; +use log::{debug, error, info}; +use tokio::sync::{oneshot, Notify}; use core::mem; use datafusion::cube_ext; use crate::cluster::ingestion::job_processor::JobProcessor; @@ -36,7 +35,7 @@ pub struct JobRunner { pub notify: Arc, pub stop_token: CancellationToken, pub is_long_term: bool, - pub job_processor: Arc, + pub job_processor: Arc, } impl JobRunner { @@ -199,7 +198,6 @@ impl JobRunner { } JobType::PartitionCompaction => { if let RowKey::Table(TableId::Partitions, partition_id) = job.row_reference() { - let compaction_service = self.compaction_service.clone(); let partition_id = *partition_id; let process_rate_limiter = self.process_rate_limiter.clone(); let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); @@ -220,7 +218,6 @@ impl JobRunner { trace_obj, }; - let data_loaded_size = DataLoadedSize::new(); let res = job_processor.process_job(job_to_move).await; if let Ok(job_res) = res { process_rate_limiter diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/mod.rs b/rust/cubestore/cubestore/src/cluster/ingestion/mod.rs index 5b4e7edb2b3f1..9cd6e93633d7f 100644 --- a/rust/cubestore/cubestore/src/cluster/ingestion/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/ingestion/mod.rs @@ -1,3 +1,2 @@ -pub mod worker; pub mod job_processor; pub mod job_runner; diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/worker.rs b/rust/cubestore/cubestore/src/cluster/ingestion/worker.rs deleted file mode 100644 index 64f8dc3fe51ad..0000000000000 --- a/rust/cubestore/cubestore/src/cluster/ingestion/worker.rs +++ /dev/null @@ -1,46 +0,0 @@ -use crate::cluster::ingestion::job_processor::{JobIsolatedProcessor, JobProcessResult}; -#[cfg(not(target_os = "windows"))] -use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; -use crate::config::Config; -use crate::metastore::job::{Job, JobStatus, JobType}; -use crate::CubeError; -use async_trait::async_trait; -use log::{debug, error}; -use serde::{Deserialize, Serialize}; -use std::time::SystemTime; - -#[derive(Debug, Serialize, Deserialize)] -pub enum IngestionWorkerMessage { - Job(Job), -} - -#[cfg(not(target_os = "windows"))] -pub struct IngestionWorkerProcessor; - -#[cfg(not(target_os = "windows"))] -#[async_trait] -impl MessageProcessor for IngestionWorkerProcessor { - async fn process( - config: &Config, - args: IngestionWorkerMessage, - ) -> Result { - let processor = JobIsolatedProcessor::new_from_config(config).await; - println!("!!!AAAAAAAAAAA!!!!!!"); - match args { - IngestionWorkerMessage::Job(job) => { - let processor_to_move = processor.clone(); - let future = async move { - let time = SystemTime::now(); - debug!("Running job in worker started"); - let res = processor_to_move.process_separate_job(&job).await; - debug!( - "Running job in worker completed ({:?})", - time.elapsed().unwrap() - ); - res - }; - future.await - } - } - } -} diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index f798ff34f0a50..3532821b8e018 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -6,7 +6,7 @@ pub mod worker_pool; pub mod rate_limiter; -mod ingestion; +pub mod ingestion; #[cfg(not(target_os = "windows"))] use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; @@ -16,13 +16,11 @@ use crate::cluster::message::NetworkMessage; use crate::cluster::rate_limiter::{ProcessRateLimiter, TaskType, TraceIndex}; use crate::cluster::transport::{ClusterTransport, MetaStoreTransport, WorkerConnection}; use crate::config::injection::{DIService, Injector}; -use crate::config::{is_router, WorkerServices}; +use crate::config::is_router; #[allow(unused_imports)] use crate::config::{Config, ConfigObj}; -use crate::import::ImportService; use crate::metastore::chunks::chunk_file_name; use crate::metastore::job::{Job, JobStatus, JobType}; -use crate::metastore::table::Table; use crate::metastore::{ deactivate_table_on_corrupt_data, Chunk, IdRow, MetaStore, MetaStoreEvent, Partition, RowKey, TableId, @@ -33,18 +31,14 @@ use crate::metastore::{ }; use crate::queryplanner::query_executor::{QueryExecutor, SerializedRecordBatchStream}; use crate::queryplanner::serialized_plan::SerializedPlan; -use crate::queryplanner::trace_data_loaded::DataLoadedSize; use crate::remotefs::RemoteFs; -use crate::store::compaction::CompactionService; use crate::store::ChunkDataStore; use crate::telemetry::tracing::TracingHelper; -use crate::util::aborting_join_handle::AbortingJoinHandle; use crate::CubeError; use arrow::datatypes::SchemaRef; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use async_trait::async_trait; -use core::mem; use datafusion::cube_ext; use datafusion::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use flatbuffers::bitflags::_core::pin::Pin; @@ -67,7 +61,6 @@ use std::time::SystemTime; use tokio::net::{TcpListener, TcpStream}; use tokio::sync::broadcast::{Receiver, Sender}; use tokio::sync::{oneshot, watch, Notify, RwLock}; -use tokio::task::JoinHandle; use tokio::time::timeout; use tokio_util::sync::CancellationToken; use tracing::{instrument, Instrument}; @@ -289,6 +282,12 @@ impl MessageProcessor String { + std::env::var("CUBESTORE_SELECT_WORKER_TITLE") + .ok() + .unwrap_or("--sel-worker".to_string()) + } } #[cfg(not(target_os = "windows"))] @@ -822,20 +821,20 @@ impl ClusterImpl { let arc = Arc::new(WorkerPool::new( self.config_obj.select_worker_pool_size(), Duration::from_secs(self.config_obj.query_timeout()), + "sel", + vec![("_CUBESTORE_SUBPROCESS_TYPE".to_string(), "SELECT_WORKER".to_string())] )); *pool = Some(arc.clone()); futures.push(cube_ext::spawn( async move { arc.wait_processing_loops().await }, )); } - #[cfg(not(target_os = "windows"))] let job_processor = { - let job_processor = JobProcessor::new(self.config_obj.job_runners_count(), Duration::from_secs(600)); //TODO - timeout + let job_processor = self.injector.upgrade().unwrap().get_service_typed::().await; //TODO - timeout let job_processor_to_move = job_processor.clone(); futures.push(cube_ext::spawn( async move { job_processor_to_move.wait_processing_loops().await }, )); - println!("!!!!! ---- !!!!!!"); job_processor }; diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 3d065edd2eeeb..853dc0aa600af 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -54,6 +54,7 @@ pub trait MessageProcessor< > { async fn process(config: &Config, args: T) -> Result; + fn process_titile() -> String; } impl< @@ -62,7 +63,7 @@ impl< P: MessageProcessor + Sync + Send + 'static, > WorkerPool { - pub fn new(num: usize, timeout: Duration) -> WorkerPool { + pub fn new(num: usize, timeout: Duration, name_prefix: &str, envs: Vec<(String, String)>) -> WorkerPool { let queue = Arc::new(unlimited::Queue::new()); let (stopped_tx, stopped_rx) = watch::channel(false); @@ -70,7 +71,8 @@ impl< for i in 1..=num { let process = Arc::new(WorkerProcess::::new( - format!("sel{}", i), + format!("{}{}", name_prefix, i), + envs.clone(), queue.clone(), timeout.clone(), stopped_rx.clone(), @@ -87,7 +89,6 @@ impl< } pub async fn wait_processing_loops(&self) { - println!("FFFFFFFF"); let futures = self .workers .iter() @@ -148,6 +149,7 @@ pub struct WorkerProcess< P: MessageProcessor + Sync + Send + 'static, > { name: String, + envs: Vec<(String, String)>, queue: Arc>>, timeout: Duration, processor: PhantomData

, @@ -163,12 +165,14 @@ impl< { fn new( name: String, + envs: Vec<(String, String)>, queue: Arc>>, timeout: Duration, stopped_rx: watch::Receiver, ) -> Self { WorkerProcess { name, + envs, queue, timeout, stopped_rx: RwLock::new(stopped_rx), @@ -300,9 +304,9 @@ impl< } ctx += &self.name; - let title = std::env::var("CUBESTORE_SELECT_WORKER_TITLE") - .ok() - .unwrap_or("--sel-worker".to_string()); + let title = P::process_titile(); + let mut envs = vec![("CUBESTORE_LOG_CONTEXT".to_string(), ctx)]; + envs.extend(self.envs.iter().cloned()); let handle = respawn( WorkerProcessArgs { @@ -311,7 +315,7 @@ impl< processor: PhantomData::

::default(), }, &[title], - &[("CUBESTORE_LOG_CONTEXT".to_string(), ctx)], + &envs, )?; Ok((args_tx, res_rx, handle)) } @@ -330,6 +334,10 @@ where R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor, { + if std::env::var("_CUBESTORE_SUBPROCESS_TYPE") == Ok("INGESTION_WORKER".to_string()) { + println!("!!! ingestion started"); + + } let (rx, tx) = (a.args, a.results); let mut tokio_builder = Builder::new_multi_thread(); tokio_builder.enable_all(); @@ -486,6 +494,10 @@ mod tests { } } } + + fn process_titile() -> String { + "--sel-worker".to_string() + } } #[test] diff --git a/rust/cubestore/cubestore/src/config/mod.rs b/rust/cubestore/cubestore/src/config/mod.rs index 60798c415c3f5..827b7cfded158 100644 --- a/rust/cubestore/cubestore/src/config/mod.rs +++ b/rust/cubestore/cubestore/src/config/mod.rs @@ -6,6 +6,7 @@ use crate::cachestore::{ CacheEvictionPolicy, CacheStore, CacheStoreSchedulerImpl, ClusterCacheStoreClient, LazyRocksCacheStore, }; +use crate::cluster::ingestion::job_processor::{JobProcessor, JobProcessorImpl}; use crate::cluster::rate_limiter::{BasicProcessRateLimiter, ProcessRateLimiter}; use crate::cluster::transport::{ ClusterTransport, ClusterTransportImpl, MetaStoreTransport, MetaStoreTransportImpl, @@ -2137,6 +2138,17 @@ impl Config { }) .await; + self.injector + .register_typed::(async move |i| { + JobProcessorImpl::new( + i.get_service_typed().await, + i.get_service_typed().await, + i.get_service_typed().await, + i.get_service_typed().await, + ) + }) + .await; + if self.config_obj.bind_address().is_some() { self.injector .register_typed::(async move |_| { diff --git a/rust/cubestore/cubestore/src/sql/mod.rs b/rust/cubestore/cubestore/src/sql/mod.rs index 6e25ebcb6beb9..32ba7dd753d9c 100644 --- a/rust/cubestore/cubestore/src/sql/mod.rs +++ b/rust/cubestore/cubestore/src/sql/mod.rs @@ -2802,7 +2802,6 @@ mod tests { ) AS `lambda` where a = 1 group by 1, 2 order by 3 desc").await.unwrap(); match &result.get_rows()[0].values()[0] { TableValue::String(s) => { - println!("!! s {}", s); assert_eq!(s, "Sort\ \n Projection, [sel__a, sel__b, sel__c]\ @@ -2838,7 +2837,6 @@ mod tests { ) AS `lambda` where a = 1 group by 1, 2 order by 3 desc").await.unwrap(); match &result.get_rows()[0].values()[0] { TableValue::String(s) => { - println!("!! s {}", s); assert_eq!(s, "Sort\ \n Projection, [sel__a, sel__b, sel__c]\ @@ -3989,6 +3987,7 @@ mod tests { Config::test("compaction").update_config(|mut config| { config.partition_split_threshold = 5; config.compaction_chunks_count_threshold = 0; + config.select_worker_pool_size = 1; config }).start_test(async move |services| { let service = services.sql_service; diff --git a/rust/cubestore/cubestore/src/store/compaction.rs b/rust/cubestore/cubestore/src/store/compaction.rs index 791ae891a09bb..1e0ae86b4c1ed 100644 --- a/rust/cubestore/cubestore/src/store/compaction.rs +++ b/rust/cubestore/cubestore/src/store/compaction.rs @@ -426,11 +426,13 @@ impl CompactionService for CompactionServiceImpl { partition_id: u64, data_loaded_size: Arc, ) -> Result<(), CubeError> { + println!("11111"); let (partition, index, table, multi_part) = self .meta_store .get_partition_for_compaction(partition_id) .await?; + println!("2222"); if !partition.get_row().is_active() && !multi_part.is_some() { log::trace!( "Cannot compact inactive partition: {:?}", @@ -438,6 +440,7 @@ impl CompactionService for CompactionServiceImpl { ); return Ok(()); } + println!("3333"); if let Some(mp) = &multi_part { if mp.get_row().prepared_for_split() { log::debug!( @@ -447,6 +450,7 @@ impl CompactionService for CompactionServiceImpl { return Ok(()); } } + println!("44444"); let mut all_pending_chunks = self .meta_store .get_chunks_by_partition(partition_id, false) @@ -468,10 +472,12 @@ impl CompactionService for CompactionServiceImpl { .map(|c| c.clone()) .collect::>(); + println!("5555"); if chunks.is_empty() { return Ok(()); } + println!("6666"); let partition_id = partition.get_id(); let mut data = Vec::new(); @@ -506,6 +512,8 @@ impl CompactionService for CompactionServiceImpl { } } + println!("7777777"); + data_loaded_size.add(chunks_total_size); let chunks = chunks_to_use; @@ -532,6 +540,7 @@ impl CompactionService for CompactionServiceImpl { } }; + println!("8888888"); let mut total_rows = chunks_row_count; if new_chunk.is_none() { total_rows += partition.get_row().main_table_row_count(); diff --git a/rust/cubestore/cubestore/src/store/mod.rs b/rust/cubestore/cubestore/src/store/mod.rs index 86fbc54a68ff3..9ee7c5c6b11fc 100644 --- a/rust/cubestore/cubestore/src/store/mod.rs +++ b/rust/cubestore/cubestore/src/store/mod.rs @@ -566,7 +566,9 @@ impl ChunkDataStore for ChunkStore { partition: IdRow, index: IdRow, ) -> Result, CubeError> { + println!("f1111"); if chunk.get_row().in_memory() { + println!("f2222"); let node_name = self.cluster.node_name_by_partition(&partition); let server_name = self.cluster.server_name(); if node_name != server_name { @@ -580,9 +582,12 @@ impl ChunkDataStore for ChunkStore { arrow_schema(&index.get_row()), )))]) } else { + println!("f33333"); let (local_file, index) = self.download_chunk(chunk, partition, index).await?; + println!("f4444"); Ok(cube_ext::spawn_blocking(move || -> Result<_, CubeError> { let parquet = ParquetTableStore::new(index, ROW_GROUP_SIZE); + println!("f5555"); Ok(parquet.read_columns(&local_file)?) }) .await??) @@ -726,16 +731,20 @@ impl ChunkStore { partition: IdRow, index: IdRow, ) -> Result<(String, Index), CubeError> { + println!("d11111"); if !chunk.get_row().uploaded() { return Err(CubeError::internal(format!( "Trying to get not uploaded chunk: {:?}", chunk ))); } + println!("d22222"); let file_size = chunk.get_row().file_size(); let chunk_id = chunk.get_id(); let remote_path = ChunkStore::chunk_file_name(chunk); + println!("d22233"); let result = self.remote_fs.download_file(&remote_path, file_size).await; + println!("d33333"); deactivate_table_on_corrupt_data( self.meta_store.clone(), @@ -745,6 +754,8 @@ impl ChunkStore { ) .await; + println!("d4444"); + Ok(( self.remote_fs.local_file(&remote_path).await?, index.into_row(), From aaf73c71aa5f13a0a8b8ac50932cb32eda7441eb Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Mon, 4 Sep 2023 17:49:45 +0300 Subject: [PATCH 03/15] update --- .../src/cluster/ingestion/job_processor.rs | 67 +++---- .../src/cluster/ingestion/job_runner.rs | 183 +++++++++--------- rust/cubestore/cubestore/src/cluster/mod.rs | 77 +++++++- .../cubestore/src/cluster/worker_pool.rs | 121 ++++++------ .../cubestore/src/store/compaction.rs | 9 - rust/cubestore/cubestore/src/store/mod.rs | 11 -- 6 files changed, 249 insertions(+), 219 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs b/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs index 7abdd546f5d25..70e52c6a605e8 100644 --- a/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs +++ b/rust/cubestore/cubestore/src/cluster/ingestion/job_processor.rs @@ -1,11 +1,12 @@ use crate::config::injection::DIService; -use crate::config::{Config, ConfigObj}; +use crate::config::Config; use crate::import::ImportService; use crate::metastore::job::{Job, JobType}; use crate::metastore::table::Table; use crate::metastore::{MetaStore, RowKey, TableId}; use crate::queryplanner::trace_data_loaded::DataLoadedSize; use crate::store::compaction::CompactionService; +use crate::store::ChunkDataStore; use crate::CubeError; use async_trait::async_trait; use serde::{Deserialize, Serialize}; @@ -47,15 +48,15 @@ pub struct JobProcessorImpl { impl JobProcessorImpl { pub fn new( - config_obj: Arc, meta_store: Arc, + chunk_store: Arc, compaction_service: Arc, import_service: Arc, ) -> Arc { Arc::new(Self { processor: JobIsolatedProcessor::new( - config_obj, meta_store, + chunk_store, compaction_service, import_service, ), @@ -72,58 +73,29 @@ impl JobProcessor for JobProcessorImpl { } async fn process_job(&self, job: Job) -> Result { - self.processor - .process_separate_job(&job) - .await + self.processor.process_separate_job(&job).await } } crate::di_service!(JobProcessorImpl, [JobProcessor]); -//TODO -#[cfg(target_os = "windows")] -pub struct JobProcessor { - process_pool: - Arc>, -} - -#[cfg(target_os = "windows")] -impl JobProcessor { - pub fn new(pool_size: usize, timeout: Duration) -> Arc { - Arc::new(Self { - process_pool: Arc::new(WorkerPool::new(pool_size, timeout)), - }) - } - pub async fn wait_processing_loops(&self) { - self.process_pool.wait_processing_loops().await - } - pub async fn stop_processing_loops(&self) -> Result<(), CubeError> { - self.process_pool.stop_workers().await - } - pub async fn process_job(&self, job: Job) -> Result { - self.process_pool - .process(IngestionWorkerMessage::Job(job)) - .await - } -} - pub struct JobIsolatedProcessor { - config_obj: Arc, meta_store: Arc, + chunk_store: Arc, compaction_service: Arc, import_service: Arc, } impl JobIsolatedProcessor { pub fn new( - config_obj: Arc, meta_store: Arc, + chunk_store: Arc, compaction_service: Arc, import_service: Arc, ) -> Arc { Arc::new(Self { - config_obj, meta_store, + chunk_store, compaction_service, import_service, }) @@ -131,7 +103,7 @@ impl JobIsolatedProcessor { pub async fn new_from_config(config: &Config) -> Arc { Self::new( - config.config_obj(), + config.injector().get_service_typed().await, config.injector().get_service_typed().await, config.injector().get_service_typed().await, config.injector().get_service_typed().await, @@ -145,11 +117,9 @@ impl JobIsolatedProcessor { let compaction_service = self.compaction_service.clone(); let partition_id = *partition_id; let data_loaded_size = DataLoadedSize::new(); - println!("!!!! CCCCCCCC"); let r = compaction_service .compact(partition_id, data_loaded_size.clone()) .await; - println!("!!!! FFFFFF"); r?; Ok(JobProcessResult::new(data_loaded_size.get())) } else { @@ -215,6 +185,25 @@ impl JobIsolatedProcessor { Self::fail_job_row_key(job) } } + JobType::RepartitionChunk => { + if let RowKey::Table(TableId::Chunks, chunk_id) = job.row_reference() { + let chunk_id = *chunk_id; + let chunk = self.meta_store.get_chunk(chunk_id).await?; + if chunk.get_row().in_memory() { + return Err(CubeError::internal( + "In-memory chunk cannot be repartitioned in separate process" + .to_string(), + )); + } + let data_loaded_size = DataLoadedSize::new(); + self.chunk_store + .repartition_chunk(chunk_id, data_loaded_size.clone()) + .await?; + Ok(JobProcessResult::new(data_loaded_size.get())) + } else { + Self::fail_job_row_key(job) + } + } _ => Err(CubeError::internal(format!( "Job {:?} cannot be processed in separate process", job.job_type() diff --git a/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs b/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs index 53a00a105b776..5fb207686ebc1 100644 --- a/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs +++ b/rust/cubestore/cubestore/src/cluster/ingestion/job_runner.rs @@ -1,28 +1,26 @@ +use crate::cluster::ingestion::job_processor::JobProcessor; +use crate::cluster::rate_limiter::{ProcessRateLimiter, TaskType, TraceIndex}; use crate::config::ConfigObj; -use crate::metastore::{ - IdRow, MetaStore, RowKey, TableId, -}; -use crate::metastore::job::{Job, JobStatus, JobType}; use crate::import::ImportService; -use crate::store::ChunkDataStore; -use crate::store::compaction::CompactionService; -use tokio_util::sync::CancellationToken; -use crate::queryplanner::trace_data_loaded::DataLoadedSize; +use crate::metastore::job::{Job, JobStatus, JobType}; use crate::metastore::table::Table; -use tokio::time::timeout; -use tokio::task::JoinHandle; +use crate::metastore::{IdRow, MetaStore, RowKey, TableId}; +use crate::queryplanner::trace_data_loaded::DataLoadedSize; +use crate::store::compaction::CompactionService; +use crate::store::ChunkDataStore; use crate::util::aborting_join_handle::AbortingJoinHandle; -use std::time::SystemTime; -use crate::cluster::rate_limiter::{ProcessRateLimiter, TaskType, TraceIndex}; -use futures_timer::Delay; -use std::time::Duration; use crate::CubeError; -use std::sync::Arc; -use log::{debug, error, info}; -use tokio::sync::{oneshot, Notify}; use core::mem; use datafusion::cube_ext; -use crate::cluster::ingestion::job_processor::JobProcessor; +use futures_timer::Delay; +use log::{debug, error, info}; +use std::sync::Arc; +use std::time::Duration; +use std::time::SystemTime; +use tokio::sync::{oneshot, Notify}; +use tokio::task::JoinHandle; +use tokio::time::timeout; +use tokio_util::sync::CancellationToken; pub struct JobRunner { pub config_obj: Arc, @@ -218,17 +216,19 @@ impl JobRunner { trace_obj, }; - let res = job_processor.process_job(job_to_move).await; - if let Ok(job_res) = res { - process_rate_limiter - .commit_task_usage( - TaskType::Job, - job_res.data_loaded_size() as i64, - trace_index, - ) - .await; + match job_processor.process_job(job_to_move).await { + Ok(job_res) => { + process_rate_limiter + .commit_task_usage( + TaskType::Job, + job_res.data_loaded_size() as i64, + trace_index, + ) + .await; + Ok(()) + } + Err(e) => Err(e), } - Ok(()) })) } else { Self::fail_job_row_key(job) @@ -264,40 +264,33 @@ impl JobRunner { } } JobType::MultiPartitionSplit => { - if let RowKey::Table(TableId::MultiPartitions, id) = job.row_reference() { - let compaction_service = self.compaction_service.clone(); - let id = *id; + if let RowKey::Table(TableId::MultiPartitions, _) = job.row_reference() { + let job_to_move = job.clone(); + let job_processor = self.job_processor.clone(); Ok(cube_ext::spawn(async move { - compaction_service.split_multi_partition(id).await + job_processor.process_job(job_to_move).await.map(|_| ()) })) } else { Self::fail_job_row_key(job) } } JobType::FinishMultiSplit => { - if let RowKey::Table(TableId::MultiPartitions, multi_part_id) = job.row_reference() - { - let meta_store = self.meta_store.clone(); - let compaction_service = self.compaction_service.clone(); - let multi_part_id = *multi_part_id; + if let RowKey::Table(TableId::MultiPartitions, _) = job.row_reference() { + let job_to_move = job.clone(); + let job_processor = self.job_processor.clone(); Ok(cube_ext::spawn(async move { - for p in meta_store.find_unsplit_partitions(multi_part_id).await? { - compaction_service - .finish_multi_split(multi_part_id, p) - .await? - } - Ok(()) + job_processor.process_job(job_to_move).await.map(|_| ()) })) } else { Self::fail_job_row_key(job) } } JobType::TableImport => { - if let RowKey::Table(TableId::Tables, table_id) = job.row_reference() { - let import_service = self.import_service.clone(); - let table_id = *table_id; + if let RowKey::Table(TableId::Tables, _) = job.row_reference() { + let job_to_move = job.clone(); + let job_processor = self.job_processor.clone(); Ok(cube_ext::spawn(async move { - import_service.import_table(table_id).await + job_processor.process_job(job_to_move).await.map(|_| ()) })) } else { Self::fail_job_row_key(job) @@ -311,6 +304,8 @@ impl JobRunner { let process_rate_limiter = self.process_rate_limiter.clone(); let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); let metastore = self.meta_store.clone(); + let job_to_move = job.clone(); + let job_processor = self.job_processor.clone(); Ok(cube_ext::spawn(async move { let is_streaming = Table::is_stream_location(&location); let data_loaded_size = if is_streaming { @@ -322,26 +317,31 @@ impl JobRunner { process_rate_limiter .wait_for_allow(TaskType::Job, timeout) .await?; //TODO config, may be same ad orphaned timeout + match job_processor.process_job(job_to_move).await { + Ok(job_res) => { + let trace_obj = + metastore.get_trace_obj_by_table_id(table_id).await?; + let trace_index = TraceIndex { + table_id: Some(table_id), + trace_obj, + }; + process_rate_limiter + .commit_task_usage( + TaskType::Job, + job_res.data_loaded_size() as i64, + trace_index, + ) + .await; + Ok(()) + } + Err(e) => Err(e), + } + } else { + import_service + .clone() + .import_table_part(table_id, &location, data_loaded_size.clone()) + .await } - let res = import_service - .clone() - .import_table_part(table_id, &location, data_loaded_size.clone()) - .await; - if let Some(data_loaded) = &data_loaded_size { - let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; - let trace_index = TraceIndex { - table_id: Some(table_id), - trace_obj, - }; - process_rate_limiter - .commit_task_usage( - TaskType::Job, - data_loaded.get() as i64, - trace_index, - ) - .await; - } - res })) } else { Self::fail_job_row_key(job) @@ -354,32 +354,41 @@ impl JobRunner { let process_rate_limiter = self.process_rate_limiter.clone(); let timeout = Some(Duration::from_secs(self.config_obj.import_job_timeout())); let metastore = self.meta_store.clone(); + let job_to_move = job.clone(); + let job_processor = self.job_processor.clone(); Ok(cube_ext::spawn(async move { process_rate_limiter .wait_for_allow(TaskType::Job, timeout) .await?; //TODO config, may be same ad orphaned timeout let chunk = metastore.get_chunk(chunk_id).await?; - let (_, _, table, _) = metastore - .get_partition_for_compaction(chunk.get_row().get_partition_id()) - .await?; - let table_id = table.get_id(); - let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; - let trace_index = TraceIndex { - table_id: Some(table_id), - trace_obj, - }; - let data_loaded_size = DataLoadedSize::new(); - let res = chunk_store - .repartition_chunk(chunk_id, data_loaded_size.clone()) - .await; - process_rate_limiter - .commit_task_usage( - TaskType::Job, - data_loaded_size.get() as i64, - trace_index, - ) - .await; - res + if !chunk.get_row().in_memory() { + let (_, _, table, _) = metastore + .get_partition_for_compaction(chunk.get_row().get_partition_id()) + .await?; + let table_id = table.get_id(); + let trace_obj = metastore.get_trace_obj_by_table_id(table_id).await?; + let trace_index = TraceIndex { + table_id: Some(table_id), + trace_obj, + }; + match job_processor.process_job(job_to_move).await { + Ok(job_res) => { + process_rate_limiter + .commit_task_usage( + TaskType::Job, + job_res.data_loaded_size() as i64, + trace_index, + ) + .await; + Ok(()) + } + Err(e) => Err(e), + } + } else { + chunk_store + .repartition_chunk(chunk_id, DataLoadedSize::new()) + .await + } })) } else { Self::fail_job_row_key(job) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index 3532821b8e018..dc4c134bd8265 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -43,9 +43,12 @@ use datafusion::cube_ext; use datafusion::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use flatbuffers::bitflags::_core::pin::Pin; use futures::future::join_all; +use futures::future::BoxFuture; use futures::task::{Context, Poll}; use futures::{Future, Stream}; use futures_timer::Delay; +use ingestion::job_processor::JobProcessor; +use ingestion::job_runner::JobRunner; use itertools::Itertools; use log::{debug, error, info, warn}; use mockall::automock; @@ -64,8 +67,6 @@ use tokio::sync::{oneshot, watch, Notify, RwLock}; use tokio::time::timeout; use tokio_util::sync::CancellationToken; use tracing::{instrument, Instrument}; -use ingestion::job_runner::JobRunner; -use ingestion::job_processor::JobProcessor; #[automock] #[async_trait] @@ -222,8 +223,30 @@ pub struct WorkerProcessor; impl MessageProcessor, usize)> for WorkerProcessor { + type Config = Config; + + async fn configure() -> Result { + let custom_fn = SELECT_WORKER_CONFIGURE_FN.read().unwrap().clone(); + let config = if let Some(func) = custom_fn.as_ref() { + func().await + } else { + let config = Config::default(); + config.configure_injector().await; + config + }; + Ok(config) + } + + fn spawn_background_processes(config: Self::Config) -> Result<(), CubeError> { + let custom_fn = SELECT_WORKER_SPAWN_BACKGROUND_FN.read().unwrap(); + if let Some(func) = custom_fn.as_ref() { + func(config); + } + Ok(()) + } + async fn process( - config: &Config, + config: &Self::Config, args: WorkerMessage, ) -> Result<(SchemaRef, Vec, usize), CubeError> { let services = config.worker_services().await; @@ -302,6 +325,34 @@ fn proc_handler() { ); } +lazy_static! { + static ref SELECT_WORKER_CONFIGURE_FN: std::sync::RwLock BoxFuture<'static, Config> + Send + Sync>>> = + std::sync::RwLock::new(None); +} + +lazy_static! { + static ref SELECT_WORKER_SPAWN_BACKGROUND_FN: std::sync::RwLock>> = + std::sync::RwLock::new(None); +} + +pub fn register_select_worker_configure_fn(f: fn() -> BoxFuture<'static, Config>) { + let mut func = SELECT_WORKER_CONFIGURE_FN.write().unwrap(); + assert!( + func.is_none(), + "select worker configure function already registered" + ); + *func = Some(Arc::new(f)); +} + +pub fn register_select_worker_spawn_background_fn(f: fn(Config)) { + let mut func = SELECT_WORKER_SPAWN_BACKGROUND_FN.write().unwrap(); + assert!( + func.is_none(), + "select worker spawn background function already registered" + ); + *func = Some(Box::new(f)); +} + lazy_static! { static ref HEART_BEAT_NODE_REGEX: Regex = Regex::new(r"^node-heart-beats/(?P.*)$").unwrap(); @@ -822,7 +873,10 @@ impl ClusterImpl { self.config_obj.select_worker_pool_size(), Duration::from_secs(self.config_obj.query_timeout()), "sel", - vec![("_CUBESTORE_SUBPROCESS_TYPE".to_string(), "SELECT_WORKER".to_string())] + vec![( + "_CUBESTORE_SUBPROCESS_TYPE".to_string(), + "SELECT_WORKER".to_string(), + )], )); *pool = Some(arc.clone()); futures.push(cube_ext::spawn( @@ -830,16 +884,19 @@ impl ClusterImpl { )); } let job_processor = { - let job_processor = self.injector.upgrade().unwrap().get_service_typed::().await; //TODO - timeout + let job_processor = self + .injector + .upgrade() + .unwrap() + .get_service_typed::() + .await; //TODO - timeout let job_processor_to_move = job_processor.clone(); - futures.push(cube_ext::spawn( - async move { job_processor_to_move.wait_processing_loops().await }, - )); + futures.push(cube_ext::spawn(async move { + job_processor_to_move.wait_processing_loops().await + })); job_processor - }; - for i in 0..self.config_obj.job_runners_count() + self.config_obj.long_term_job_runners_count() { diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 853dc0aa600af..5ec5a90f4ee59 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -8,7 +8,6 @@ use std::time::Duration; use async_trait::async_trait; use deadqueue::unlimited; use futures::future::join_all; -use futures::future::BoxFuture; use ipc_channel::ipc; use ipc_channel::ipc::{IpcReceiver, IpcSender}; use log::error; @@ -53,7 +52,14 @@ pub trait MessageProcessor< R: Serialize + DeserializeOwned + Sync + Send + 'static, > { - async fn process(config: &Config, args: T) -> Result; + type Config: Sync + Send + Clone + 'static; + + async fn configure() -> Result; + + fn spawn_background_processes(config: Self::Config) -> Result<(), CubeError>; + + async fn process(config: &Self::Config, args: T) -> Result; + fn process_titile() -> String; } @@ -63,7 +69,12 @@ impl< P: MessageProcessor + Sync + Send + 'static, > WorkerPool { - pub fn new(num: usize, timeout: Duration, name_prefix: &str, envs: Vec<(String, String)>) -> WorkerPool { + pub fn new( + num: usize, + timeout: Duration, + name_prefix: &str, + envs: Vec<(String, String)>, + ) -> WorkerPool { let queue = Arc::new(unlimited::Queue::new()); let (stopped_tx, stopped_rx) = watch::channel(false); @@ -71,7 +82,7 @@ impl< for i in 1..=num { let process = Arc::new(WorkerProcess::::new( - format!("{}{}", name_prefix, i), + format!("{}{}", name_prefix, i), envs.clone(), queue.clone(), timeout.clone(), @@ -334,10 +345,6 @@ where R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor, { - if std::env::var("_CUBESTORE_SUBPROCESS_TYPE") == Ok("INGESTION_WORKER".to_string()) { - println!("!!! ingestion started"); - - } let (rx, tx) = (a.args, a.results); let mut tokio_builder = Builder::new_multi_thread(); tokio_builder.enable_all(); @@ -350,19 +357,35 @@ where let runtime = tokio_builder.build().unwrap(); worker_setup(&runtime); runtime.block_on(async move { - let config = get_worker_config().await; + let config = match P::configure().await { + Err(e) => { + error!( + "Error during {} worker configure: {}", + P::process_titile(), + e + ); + return 1; + } + Ok(config) => config, + }; - spawn_background_processes(config.clone()); + if let Err(e) = P::spawn_background_processes(config.clone()) { + error!( + "Error during {} worker background processes spawn: {}", + P::process_titile(), + e + ); + } loop { let res = rx.recv(); match res { Ok(args) => { - let result = - match async_try_with_catch_unwind(P::process(&config, args)).await { - Ok(result) => result, - Err(panic) => Err(CubeError::from(panic)), - }; + let result = match async_try_with_catch_unwind(P::process(&config, args)).await + { + Ok(result) => result, + Err(panic) => Err(CubeError::from(panic)), + }; let send_res = tx.send(result); if let Err(e) = send_res { error!("Worker message send error: {:?}", e); @@ -385,63 +408,17 @@ fn worker_setup(runtime: &Runtime) { } } -async fn get_worker_config() -> Config { - let custom_fn = SELECT_WORKER_CONFIGURE_FN.read().unwrap(); - if let Some(func) = custom_fn.as_ref() { - func().await - } else { - let config = Config::default(); - config.configure_injector().await; - config - } -} - -fn spawn_background_processes(config: Config) { - let custom_fn = SELECT_WORKER_SPAWN_BACKGROUND_FN.read().unwrap(); - if let Some(func) = custom_fn.as_ref() { - func(config); - } -} - lazy_static! { static ref SELECT_WORKER_SETUP: std::sync::RwLock>> = std::sync::RwLock::new(None); } -lazy_static! { - static ref SELECT_WORKER_CONFIGURE_FN: std::sync::RwLock BoxFuture<'static, Config> + Send + Sync>>> = - std::sync::RwLock::new(None); -} - -lazy_static! { - static ref SELECT_WORKER_SPAWN_BACKGROUND_FN: std::sync::RwLock>> = - std::sync::RwLock::new(None); -} - pub fn register_select_worker_setup(f: fn(&Runtime)) { let mut setup = SELECT_WORKER_SETUP.write().unwrap(); assert!(setup.is_none(), "select worker setup already registered"); *setup = Some(Box::new(f)); } -pub fn register_select_worker_configure_fn(f: fn() -> BoxFuture<'static, Config>) { - let mut func = SELECT_WORKER_CONFIGURE_FN.write().unwrap(); - assert!( - func.is_none(), - "select worker configure function already registered" - ); - *func = Some(Box::new(f)); -} - -pub fn register_select_worker_spawn_background_fn(f: fn(Config)) { - let mut func = SELECT_WORKER_SPAWN_BACKGROUND_FN.write().unwrap(); - assert!( - func.is_none(), - "select worker spawn background function already registered" - ); - *func = Some(Box::new(f)); -} - #[cfg(test)] mod tests { use std::sync::Arc; @@ -455,12 +432,11 @@ mod tests { use tokio::runtime::Builder; use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; - use crate::config::WorkerServices; + use crate::config::Config; use crate::queryplanner::serialized_plan::SerializedLogicalPlan; use crate::util::respawn; use crate::CubeError; use datafusion::cube_ext; - use crate::config::Config; #[ctor::ctor] fn test_support_init() { @@ -483,6 +459,17 @@ mod tests { #[async_trait] impl MessageProcessor for Processor { + type Config = Config; + + async fn configure() -> Result { + let config = Config::default(); + config.configure_injector().await; + Ok(config) + } + + fn spawn_background_processes(_config: Self::Config) -> Result<(), CubeError> { + Ok(()) + } async fn process(_config: &Config, args: Message) -> Result { match args { Message::Delay(x) => { @@ -508,6 +495,8 @@ mod tests { let pool = Arc::new(WorkerPool::::new( 4, Duration::from_millis(1000), + "test", + Vec::new(), )); let pool_to_move = pool.clone(); cube_ext::spawn(async move { pool_to_move.wait_processing_loops().await }); @@ -527,6 +516,8 @@ mod tests { let pool = Arc::new(WorkerPool::::new( 4, Duration::from_millis(1000), + "test", + Vec::new(), )); let pool_to_move = pool.clone(); cube_ext::spawn(async move { pool_to_move.wait_processing_loops().await }); @@ -550,6 +541,8 @@ mod tests { let pool = Arc::new(WorkerPool::::new( 4, Duration::from_millis(450), + "test", + Vec::new(), )); let pool_to_move = pool.clone(); cube_ext::spawn(async move { pool_to_move.wait_processing_loops().await }); @@ -577,6 +570,8 @@ mod tests { let pool = Arc::new(WorkerPool::::new( 4, Duration::from_millis(2000), + "test", + Vec::new(), )); let pool_to_move = pool.clone(); cube_ext::spawn(async move { pool_to_move.wait_processing_loops().await }); diff --git a/rust/cubestore/cubestore/src/store/compaction.rs b/rust/cubestore/cubestore/src/store/compaction.rs index 1e0ae86b4c1ed..791ae891a09bb 100644 --- a/rust/cubestore/cubestore/src/store/compaction.rs +++ b/rust/cubestore/cubestore/src/store/compaction.rs @@ -426,13 +426,11 @@ impl CompactionService for CompactionServiceImpl { partition_id: u64, data_loaded_size: Arc, ) -> Result<(), CubeError> { - println!("11111"); let (partition, index, table, multi_part) = self .meta_store .get_partition_for_compaction(partition_id) .await?; - println!("2222"); if !partition.get_row().is_active() && !multi_part.is_some() { log::trace!( "Cannot compact inactive partition: {:?}", @@ -440,7 +438,6 @@ impl CompactionService for CompactionServiceImpl { ); return Ok(()); } - println!("3333"); if let Some(mp) = &multi_part { if mp.get_row().prepared_for_split() { log::debug!( @@ -450,7 +447,6 @@ impl CompactionService for CompactionServiceImpl { return Ok(()); } } - println!("44444"); let mut all_pending_chunks = self .meta_store .get_chunks_by_partition(partition_id, false) @@ -472,12 +468,10 @@ impl CompactionService for CompactionServiceImpl { .map(|c| c.clone()) .collect::>(); - println!("5555"); if chunks.is_empty() { return Ok(()); } - println!("6666"); let partition_id = partition.get_id(); let mut data = Vec::new(); @@ -512,8 +506,6 @@ impl CompactionService for CompactionServiceImpl { } } - println!("7777777"); - data_loaded_size.add(chunks_total_size); let chunks = chunks_to_use; @@ -540,7 +532,6 @@ impl CompactionService for CompactionServiceImpl { } }; - println!("8888888"); let mut total_rows = chunks_row_count; if new_chunk.is_none() { total_rows += partition.get_row().main_table_row_count(); diff --git a/rust/cubestore/cubestore/src/store/mod.rs b/rust/cubestore/cubestore/src/store/mod.rs index 9ee7c5c6b11fc..86fbc54a68ff3 100644 --- a/rust/cubestore/cubestore/src/store/mod.rs +++ b/rust/cubestore/cubestore/src/store/mod.rs @@ -566,9 +566,7 @@ impl ChunkDataStore for ChunkStore { partition: IdRow, index: IdRow, ) -> Result, CubeError> { - println!("f1111"); if chunk.get_row().in_memory() { - println!("f2222"); let node_name = self.cluster.node_name_by_partition(&partition); let server_name = self.cluster.server_name(); if node_name != server_name { @@ -582,12 +580,9 @@ impl ChunkDataStore for ChunkStore { arrow_schema(&index.get_row()), )))]) } else { - println!("f33333"); let (local_file, index) = self.download_chunk(chunk, partition, index).await?; - println!("f4444"); Ok(cube_ext::spawn_blocking(move || -> Result<_, CubeError> { let parquet = ParquetTableStore::new(index, ROW_GROUP_SIZE); - println!("f5555"); Ok(parquet.read_columns(&local_file)?) }) .await??) @@ -731,20 +726,16 @@ impl ChunkStore { partition: IdRow, index: IdRow, ) -> Result<(String, Index), CubeError> { - println!("d11111"); if !chunk.get_row().uploaded() { return Err(CubeError::internal(format!( "Trying to get not uploaded chunk: {:?}", chunk ))); } - println!("d22222"); let file_size = chunk.get_row().file_size(); let chunk_id = chunk.get_id(); let remote_path = ChunkStore::chunk_file_name(chunk); - println!("d22233"); let result = self.remote_fs.download_file(&remote_path, file_size).await; - println!("d33333"); deactivate_table_on_corrupt_data( self.meta_store.clone(), @@ -754,8 +745,6 @@ impl ChunkStore { ) .await; - println!("d4444"); - Ok(( self.remote_fs.local_file(&remote_path).await?, index.into_row(), From 2adc274d587ee02977c07fe4960d8ff015fa2900 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Sun, 17 Sep 2023 17:23:51 +0300 Subject: [PATCH 04/15] in work --- rust/cubestore/cubestore/src/cluster/mod.rs | 9 ++ .../cubestore/src/cluster/worker_pool.rs | 125 ++++++++++++++-- .../cubestore/src/cluster/worker_services.rs | 141 ++++++++++++++++++ .../src/util/cancellation_token_guard.rs | 17 +++ rust/cubestore/cubestore/src/util/mod.rs | 1 + 5 files changed, 279 insertions(+), 14 deletions(-) create mode 100644 rust/cubestore/cubestore/src/cluster/worker_services.rs create mode 100644 rust/cubestore/cubestore/src/util/cancellation_token_guard.rs diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index dc4c134bd8265..d658f608a2cce 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -4,12 +4,17 @@ pub mod transport; #[cfg(not(target_os = "windows"))] pub mod worker_pool; +#[cfg(not(target_os = "windows"))] +pub mod worker_services; + pub mod rate_limiter; pub mod ingestion; #[cfg(not(target_os = "windows"))] use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; +#[cfg(not(target_os = "windows"))] +use crate::cluster::worker_services::{DefaultServicesServerDef, DefaultServicesServerProcessor}; use crate::ack_error; use crate::cluster::message::NetworkMessage; @@ -191,6 +196,8 @@ pub struct ClusterImpl { WorkerMessage, (SchemaRef, Vec, usize), WorkerProcessor, + DefaultServicesServerDef, + DefaultServicesServerProcessor, >, >, >, @@ -321,6 +328,7 @@ fn proc_handler() { WorkerMessage, (SchemaRef, Vec, usize), WorkerProcessor, + DefaultServicesServerDef, >, ); } @@ -870,6 +878,7 @@ impl ClusterImpl { { let mut pool = self.select_process_pool.write().await; let arc = Arc::new(WorkerPool::new( + DefaultServicesServerProcessor::new(), self.config_obj.select_worker_pool_size(), Duration::from_secs(self.config_obj.query_timeout()), "sel", diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 5ec5a90f4ee59..0768b1c741b30 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -15,7 +15,7 @@ use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; use tokio::runtime::{Builder, Runtime}; use tokio::sync::oneshot::Sender; -use tokio::sync::{oneshot, watch, Notify, RwLock}; +use tokio::sync::{oneshot, watch, Notify, RwLock, Mutex}; use tracing::{instrument, Instrument}; use tracing_futures::WithSubscriber; @@ -24,16 +24,22 @@ use crate::util::respawn::respawn; use crate::CubeError; use datafusion::cube_ext; use datafusion::cube_ext::catch_unwind::async_try_with_catch_unwind; +use crate::cluster::worker_services::{ServicesServerDef, ServicesServerProcessor, ServicesServer}; pub struct WorkerPool< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, + S: ServicesServerDef + Debug + Sync + Send + 'static, + SP: ServicesServerProcessor + Sync + Send + 'static, + > { queue: Arc>>, stopped_tx: watch::Sender, - workers: Vec>>, + workers: Vec>>, processor: PhantomData

, + services_def: PhantomData, + services_processor: Arc } pub struct Message< @@ -67,21 +73,25 @@ impl< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, - > WorkerPool + S: ServicesServerDef + Sync + Debug + Send + 'static, + SP: ServicesServerProcessor + Sync + Send + 'static, + > WorkerPool { pub fn new( + services_processor: Arc, num: usize, timeout: Duration, name_prefix: &str, envs: Vec<(String, String)>, - ) -> WorkerPool { + ) -> Self { let queue = Arc::new(unlimited::Queue::new()); let (stopped_tx, stopped_rx) = watch::channel(false); let mut workers = Vec::new(); for i in 1..=num { - let process = Arc::new(WorkerProcess::::new( + let process = Arc::new(WorkerProcess::::new( + services_processor.clone(), format!("{}{}", name_prefix, i), envs.clone(), queue.clone(), @@ -96,6 +106,8 @@ impl< queue, workers, processor: PhantomData, + services_def: PhantomData, + services_processor, } } @@ -158,6 +170,8 @@ pub struct WorkerProcess< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, + S: ServicesServerDef + Sync + Debug + Send + 'static, + SP: ServicesServerProcessor + Sync + Send + 'static, > { name: String, envs: Vec<(String, String)>, @@ -166,15 +180,21 @@ pub struct WorkerProcess< processor: PhantomData

, stopped_rx: RwLock>, finished_notify: Arc, + services_def: PhantomData, + services_processor: Arc, + services_server: Mutex>>, } impl< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, - > WorkerProcess + S: ServicesServerDef + Debug + Sync + Send + 'static, + SP: ServicesServerProcessor + Sync + Send + 'static, + > WorkerProcess { fn new( + services_processor: Arc, name: String, envs: Vec<(String, String)>, queue: Arc>>, @@ -182,19 +202,22 @@ impl< stopped_rx: watch::Receiver, ) -> Self { WorkerProcess { + services_processor, name, envs, queue, timeout, stopped_rx: RwLock::new(stopped_rx), finished_notify: Arc::new(Notify::new()), + services_server: Mutex::new(None), processor: PhantomData, + services_def: PhantomData, } } async fn processing_loop(&self) { loop { - let process = self.spawn_process(); + let process = self.spawn_process().await; match process { Ok((mut args_tx, mut res_rx, handle)) => { @@ -301,9 +324,15 @@ impl< Ok((res??, args_tx, res_rx)) } - fn spawn_process( + async fn spawn_process( &self, ) -> Result<(IpcSender, IpcReceiver>, Child), CubeError> { + { + if let Some(services_server) = self.services_server.lock().await.as_ref() { + services_server.stop(); + } + } + let (args_tx, args_rx) = ipc::channel()?; let (res_tx, res_rx) = ipc::channel()?; @@ -319,31 +348,43 @@ impl< let mut envs = vec![("CUBESTORE_LOG_CONTEXT".to_string(), ctx)]; envs.extend(self.envs.iter().cloned()); + let (service_request_tx, service_request_rx) = ipc::channel()?; + let (service_response_tx, service_response_rx) = ipc::channel()?; + let handle = respawn( WorkerProcessArgs { args: args_rx, results: res_tx, processor: PhantomData::

::default(), + services_def: PhantomData::::default(), + services_sender: service_request_tx, + services_reciever: service_response_rx, }, &[title], &envs, )?; + + //*self.services_server.lock().await = Some(ServicesServer::start(service_request_rx, service_response_tx, self.services_processor.clone())); Ok((args_tx, res_rx, handle)) } } #[derive(Serialize, Deserialize)] -pub struct WorkerProcessArgs { +pub struct WorkerProcessArgs { args: IpcReceiver, results: IpcSender>, processor: PhantomData

, + services_def: PhantomData, + services_sender: IpcSender, + services_reciever: IpcReceiver, } -pub fn worker_main(a: WorkerProcessArgs) -> i32 +pub fn worker_main(a: WorkerProcessArgs) -> i32 where T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor, + S: ServicesServerDef { let (rx, tx) = (a.args, a.results); let mut tokio_builder = Builder::new_multi_thread(); @@ -438,6 +479,10 @@ mod tests { use crate::CubeError; use datafusion::cube_ext; + use crate::cluster::worker_services::{ DefaultServicesServerDef, ServicesServerDef, DefaultServicesServerProcessor, ServicesServerProcessor}; + + type TestPool = WorkerPool::; + #[ctor::ctor] fn test_support_init() { respawn::replace_cmd_args_in_tests(); @@ -492,7 +537,8 @@ mod tests { let runtime = Builder::new_current_thread().enable_all().build().unwrap(); runtime.block_on(async move { - let pool = Arc::new(WorkerPool::::new( + let pool = Arc::new(TestPool::new( + DefaultServicesServerProcessor::new(), 4, Duration::from_millis(1000), "test", @@ -513,7 +559,8 @@ mod tests { let runtime = Builder::new_current_thread().enable_all().build().unwrap(); runtime.block_on(async move { - let pool = Arc::new(WorkerPool::::new( + let pool = Arc::new(TestPool::new( + DefaultServicesServerProcessor::new(), 4, Duration::from_millis(1000), "test", @@ -538,7 +585,8 @@ mod tests { let runtime = Builder::new_current_thread().enable_all().build().unwrap(); runtime.block_on(async move { - let pool = Arc::new(WorkerPool::::new( + let pool = Arc::new(TestPool::new( + DefaultServicesServerProcessor::new(), 4, Duration::from_millis(450), "test", @@ -567,7 +615,8 @@ mod tests { let runtime = Builder::new_current_thread().enable_all().build().unwrap(); runtime.block_on(async move { - let pool = Arc::new(WorkerPool::::new( + let pool = Arc::new(TestPool::new( + DefaultServicesServerProcessor::new(), 4, Duration::from_millis(2000), "test", @@ -597,4 +646,52 @@ mod tests { bincode::deserialize::(bytes.as_slice())?; Ok(()) } + + type TestServicePool = WorkerPool::; + + #[derive(Debug)] + struct TestServicesServerDef; + + impl ServicesServerDef for TestServicesServerDef { + type Request = i64; + type Response = bool; + } + + pub struct TestServicesServerProcessor; + + impl TestServicesServerProcessor { + pub fn new() -> Arc { + Arc::new(Self {}) + } + } + + #[async_trait] + impl ServicesServerProcessor for TestServicesServerProcessor { + async fn process(&self, request: i64) -> bool { + request % 2 == 0 + } + } + + + #[test] + fn test_services_basic() { + let runtime = Builder::new_current_thread().enable_all().build().unwrap(); + + runtime.block_on(async move { + let pool = Arc::new(TestServicePool::new( + TestServicesServerProcessor::new(), + 4, + Duration::from_millis(1000), + "test", + Vec::new(), + )); + let pool_to_move = pool.clone(); + cube_ext::spawn(async move { pool_to_move.wait_processing_loops().await }); + assert_eq!( + pool.process(Message::Delay(100)).await.unwrap(), + Response::Foo(100) + ); + pool.stop_workers().await.unwrap(); + }); + } } diff --git a/rust/cubestore/cubestore/src/cluster/worker_services.rs b/rust/cubestore/cubestore/src/cluster/worker_services.rs new file mode 100644 index 0000000000000..885992059e1c9 --- /dev/null +++ b/rust/cubestore/cubestore/src/cluster/worker_services.rs @@ -0,0 +1,141 @@ +use crate::util::cancellation_token_guard::CancellationGuard; +use async_trait::async_trait; +use datafusion::cube_ext; +use ipc_channel::ipc::{IpcReceiver, IpcSender}; +use serde::de::DeserializeOwned; +use serde::{Deserialize, Serialize}; +use std::fmt::Debug; +use std::marker::PhantomData; +use std::sync::Arc; +use tokio::task::JoinHandle; +use tokio_util::sync::CancellationToken; + +pub trait ServicesServerDef { + type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; +} + +#[derive(Debug, Serialize, Deserialize)] +pub struct DefaultServicesServerDef; + +impl ServicesServerDef for DefaultServicesServerDef { + type Request = (); + type Response = (); +} + +#[async_trait] +pub trait ServicesServerProcessor { + type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + async fn process(&self, request: S::Request) -> S::Response; +} + +pub struct DefaultServicesServerProcessor; + +impl DefaultServicesServerProcessor { + pub fn new() -> Arc { + Arc::new(Self {}) + } +} + +#[async_trait] +impl ServicesServerProcessor for DefaultServicesServerProcessor { + type Request = (); + type Response = (); + async fn process(&self, _request: ()) -> () { + () + } +} + +#[derive(Serialize, Deserialize, Debug)] +pub struct RequestMessage { + pub message_id: u64, + pub payload: S::Request, +} + +#[derive(Serialize, Deserialize, Debug)] +pub struct ResponseMessage { + pub message_id: u64, + pub payload: S::Response, +} + +pub struct ServicesServer< + S: ServicesServerDef + Debug + Send + Sync + 'static, + P: ServicesServerProcessor + Send + Sync + 'static, +> { + join_handle: JoinHandle<()>, + server_def: PhantomData, + processor: PhantomData

, +} + +impl< + S: ServicesServerDef + Debug + Send + Sync + 'static, + P: ServicesServerProcessor + Send + Sync + 'static, + > ServicesServer +{ + pub fn start( + reciever: IpcReceiver>, + sender: IpcSender>, + processor: Arc

, + ) -> Self { + let join_handle = Self::processing_loop(reciever, sender, processor); + Self { + join_handle, + server_def: PhantomData, + processor: PhantomData, + } + } + + pub fn stop(&self) { + self.join_handle.abort(); + } + + fn processing_loop( + reciever: IpcReceiver>, + sender: IpcSender>, + processor: Arc

, + ) -> JoinHandle<()> { + cube_ext::spawn_blocking(move || loop { + println!("##########"); + let req = reciever.recv(); + println!("req: {:?}", req); + println!("111111111"); + + let RequestMessage { + message_id, + payload, + } = match req { + Ok(message) => message, + Err(e) => { + log::error!("Error while reading ipc service request: {:?}", e); + break; + } + }; + + let processor_to_move = processor.clone(); + let sender_to_move = sender.clone(); + + cube_ext::spawn(async move { + let res = processor_to_move.process(payload).await; + match sender_to_move.send(ResponseMessage { + message_id, + payload: res, + }) { + Ok(_) => {} + Err(e) => { + log::error!("Error while sending IPC response: {:?}", e); + } + } + }); + }) + } +} + +impl< + S: ServicesServerDef + Debug + Send + Sync + 'static, + P: ServicesServerProcessor + Send + Sync + 'static, +> Drop for ServicesServer { + fn drop(&mut self) { + self.stop(); + } +} diff --git a/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs b/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs new file mode 100644 index 0000000000000..8f47c57479346 --- /dev/null +++ b/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs @@ -0,0 +1,17 @@ +use tokio_util::sync::CancellationToken; +pub struct CancellationGuard<'a> { + token: &'a CancellationToken +} + +impl<'a> CancellationGuard<'a> { + + pub fn new(token: &'a CancellationToken) -> Self { + Self { token } + } +} + +impl<'a> Drop for CancellationGuard<'a> { + fn drop(&mut self) { + self.token.cancel() + } +} diff --git a/rust/cubestore/cubestore/src/util/mod.rs b/rust/cubestore/cubestore/src/util/mod.rs index 0f17d0efe0eae..e48af1c6d8251 100644 --- a/rust/cubestore/cubestore/src/util/mod.rs +++ b/rust/cubestore/cubestore/src/util/mod.rs @@ -13,6 +13,7 @@ pub mod metrics; pub mod respawn; pub mod strings; pub mod time_span; +pub mod cancellation_token_guard; pub use malloc_trim_loop::spawn_malloc_trim_loop; From cd820a471d51a9f78bc1ebf39c7e92be262c7dc7 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Sun, 17 Sep 2023 22:57:13 +0300 Subject: [PATCH 05/15] in work --- rust/cubestore/cubestore/src/cluster/mod.rs | 7 +- .../cubestore/src/cluster/worker_pool.rs | 84 ++++++++------- .../cubestore/src/cluster/worker_services.rs | 101 ++++++++++++------ 3 files changed, 117 insertions(+), 75 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index d658f608a2cce..0478549d075d8 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -14,7 +14,7 @@ pub mod ingestion; #[cfg(not(target_os = "windows"))] use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; #[cfg(not(target_os = "windows"))] -use crate::cluster::worker_services::{DefaultServicesServerDef, DefaultServicesServerProcessor}; +use crate::cluster::worker_services::{DefaultWorkerServicesDef, DefaultServicesServerProcessor}; use crate::ack_error; use crate::cluster::message::NetworkMessage; @@ -196,8 +196,7 @@ pub struct ClusterImpl { WorkerMessage, (SchemaRef, Vec, usize), WorkerProcessor, - DefaultServicesServerDef, - DefaultServicesServerProcessor, + DefaultWorkerServicesDef, >, >, >, @@ -328,7 +327,7 @@ fn proc_handler() { WorkerMessage, (SchemaRef, Vec, usize), WorkerProcessor, - DefaultServicesServerDef, + DefaultWorkerServicesDef, >, ); } diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 0768b1c741b30..7778e8fad525a 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -24,22 +24,20 @@ use crate::util::respawn::respawn; use crate::CubeError; use datafusion::cube_ext; use datafusion::cube_ext::catch_unwind::async_try_with_catch_unwind; -use crate::cluster::worker_services::{ServicesServerDef, ServicesServerProcessor, ServicesServer}; +use crate::cluster::worker_services::{ServicesServerDef, ServicesServerProcessor, ServicesServer, WorkerServicesDef, WorkerProcessing}; pub struct WorkerPool< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, - S: ServicesServerDef + Debug + Sync + Send + 'static, - SP: ServicesServerProcessor + Sync + Send + 'static, + S: WorkerServicesDef + Sync + Send + 'static, > { queue: Arc>>, stopped_tx: watch::Sender, - workers: Vec>>, + workers: Vec>>, processor: PhantomData

, - services_def: PhantomData, - services_processor: Arc + services_processor: Arc } pub struct Message< @@ -73,12 +71,11 @@ impl< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, - S: ServicesServerDef + Sync + Debug + Send + 'static, - SP: ServicesServerProcessor + Sync + Send + 'static, - > WorkerPool + S: WorkerServicesDef + Sync + Send + 'static, + > WorkerPool { pub fn new( - services_processor: Arc, + services_processor: Arc, num: usize, timeout: Duration, name_prefix: &str, @@ -90,7 +87,7 @@ impl< let mut workers = Vec::new(); for i in 1..=num { - let process = Arc::new(WorkerProcess::::new( + let process = Arc::new(WorkerProcess::::new( services_processor.clone(), format!("{}{}", name_prefix, i), envs.clone(), @@ -106,7 +103,6 @@ impl< queue, workers, processor: PhantomData, - services_def: PhantomData, services_processor, } } @@ -170,8 +166,7 @@ pub struct WorkerProcess< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, - S: ServicesServerDef + Sync + Debug + Send + 'static, - SP: ServicesServerProcessor + Sync + Send + 'static, + S: WorkerServicesDef + Sync + Send + 'static, > { name: String, envs: Vec<(String, String)>, @@ -180,21 +175,19 @@ pub struct WorkerProcess< processor: PhantomData

, stopped_rx: RwLock>, finished_notify: Arc, - services_def: PhantomData, - services_processor: Arc, - services_server: Mutex>>, + services_processor: Arc, + services_server: Mutex>, } impl< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor + Sync + Send + 'static, - S: ServicesServerDef + Debug + Sync + Send + 'static, - SP: ServicesServerProcessor + Sync + Send + 'static, - > WorkerProcess + S: WorkerServicesDef + Sync + Send + 'static, + > WorkerProcess { fn new( - services_processor: Arc, + services_processor: Arc, name: String, envs: Vec<(String, String)>, queue: Arc>>, @@ -211,7 +204,6 @@ impl< finished_notify: Arc::new(Notify::new()), services_server: Mutex::new(None), processor: PhantomData, - services_def: PhantomData, } } @@ -356,7 +348,7 @@ impl< args: args_rx, results: res_tx, processor: PhantomData::

::default(), - services_def: PhantomData::::default(), + worker_services_def: PhantomData::::default(), services_sender: service_request_tx, services_reciever: service_response_rx, }, @@ -364,19 +356,19 @@ impl< &envs, )?; - //*self.services_server.lock().await = Some(ServicesServer::start(service_request_rx, service_response_tx, self.services_processor.clone())); + *self.services_server.lock().await = Some(ServicesServer::start(service_request_rx, service_response_tx, self.services_processor.clone())); Ok((args_tx, res_rx, handle)) } } #[derive(Serialize, Deserialize)] -pub struct WorkerProcessArgs { +pub struct WorkerProcessArgs { args: IpcReceiver, results: IpcSender>, processor: PhantomData

, - services_def: PhantomData, - services_sender: IpcSender, - services_reciever: IpcReceiver, + worker_services_def: PhantomData, + services_sender: IpcSender<>::IpcRequest>, + services_reciever: IpcReceiver<>::IpcResponse>, } pub fn worker_main(a: WorkerProcessArgs) -> i32 @@ -384,7 +376,7 @@ where T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, R: Serialize + DeserializeOwned + Sync + Send + 'static, P: MessageProcessor, - S: ServicesServerDef + S: WorkerServicesDef + Sync + Send + 'static, { let (rx, tx) = (a.args, a.results); let mut tokio_builder = Builder::new_multi_thread(); @@ -397,6 +389,7 @@ where tokio_builder.thread_stack_size(stack_size); let runtime = tokio_builder.build().unwrap(); worker_setup(&runtime); + println!("!!!!!"); runtime.block_on(async move { let config = match P::configure().await { Err(e) => { @@ -479,14 +472,14 @@ mod tests { use crate::CubeError; use datafusion::cube_ext; - use crate::cluster::worker_services::{ DefaultServicesServerDef, ServicesServerDef, DefaultServicesServerProcessor, ServicesServerProcessor}; + use crate::cluster::worker_services::{ DefaultServicesServerProcessor, ServicesServerProcessor, DefaultWorkerServicesDef, WorkerServicesDef, ServicesServerImpl}; - type TestPool = WorkerPool::; + type TestPool = WorkerPool::; #[ctor::ctor] fn test_support_init() { respawn::replace_cmd_args_in_tests(); - respawn::register_handler(worker_main::) + respawn::register_handler(worker_main::) } #[derive(Serialize, Deserialize, Eq, PartialEq, Debug)] @@ -647,16 +640,9 @@ mod tests { Ok(()) } - type TestServicePool = WorkerPool::; + type TestServicePool = WorkerPool::; #[derive(Debug)] - struct TestServicesServerDef; - - impl ServicesServerDef for TestServicesServerDef { - type Request = i64; - type Response = bool; - } - pub struct TestServicesServerProcessor; impl TestServicesServerProcessor { @@ -665,8 +651,23 @@ mod tests { } } + pub struct TestWorkerServicesDef; + + impl WorkerServicesDef for TestWorkerServicesDef { + type Processor = TestServicesServerProcessor; + type Server = ServicesServerImpl; + } + + #[ctor::ctor] + fn test_services_support_init() { + respawn::replace_cmd_args_in_tests(); + respawn::register_handler(worker_main::) + } + #[async_trait] - impl ServicesServerProcessor for TestServicesServerProcessor { + impl ServicesServerProcessor for TestServicesServerProcessor { + type Request = i64; + type Response = bool; async fn process(&self, request: i64) -> bool { request % 2 == 0 } @@ -691,6 +692,7 @@ mod tests { pool.process(Message::Delay(100)).await.unwrap(), Response::Foo(100) ); + println!("GGGGG"); pool.stop_workers().await.unwrap(); }); } diff --git a/rust/cubestore/cubestore/src/cluster/worker_services.rs b/rust/cubestore/cubestore/src/cluster/worker_services.rs index 885992059e1c9..272d72cbf8850 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_services.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_services.rs @@ -1,3 +1,4 @@ +use crate::CubeError; use crate::util::cancellation_token_guard::CancellationGuard; use async_trait::async_trait; use datafusion::cube_ext; @@ -10,26 +11,51 @@ use std::sync::Arc; use tokio::task::JoinHandle; use tokio_util::sync::CancellationToken; -pub trait ServicesServerDef { + +#[async_trait] +pub trait WorkerProcessing { + type Config: Sync + Send + Clone + 'static; type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type ServicesServer: ServicesServerDef; + + async fn configure() -> Result; + + fn spawn_background_processes(config: Self::Config) -> Result<(), CubeError>; + + async fn process(config: &Self::Config, args: Self::Request) -> Result; + + fn process_titile() -> String; } -#[derive(Debug, Serialize, Deserialize)] -pub struct DefaultServicesServerDef; -impl ServicesServerDef for DefaultServicesServerDef { - type Request = (); - type Response = (); +pub trait ServicesServerDef { + type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; } #[async_trait] -pub trait ServicesServerProcessor { +pub trait ServicesServerProcessor { type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; - async fn process(&self, request: S::Request) -> S::Response; + async fn process(&self, request: Self::Request) -> Self::Response; } +pub trait WorkerServicesDef { + type Processor: ServicesServerProcessor + Send + Sync + 'static; + type Server: ServicesServer + Send + Sync + 'static; +} + +#[derive(Debug)] +pub struct DefaultWorkerServicesDef; + +impl WorkerServicesDef for DefaultWorkerServicesDef { + type Processor = DefaultServicesServerProcessor; + type Server = ServicesServerImpl; +} + + +#[derive(Debug)] pub struct DefaultServicesServerProcessor; impl DefaultServicesServerProcessor { @@ -39,7 +65,7 @@ impl DefaultServicesServerProcessor { } #[async_trait] -impl ServicesServerProcessor for DefaultServicesServerProcessor { +impl ServicesServerProcessor for DefaultServicesServerProcessor { type Request = (); type Response = (); async fn process(&self, _request: ()) -> () { @@ -48,51 +74,67 @@ impl ServicesServerProcessor for DefaultServicesServer } #[derive(Serialize, Deserialize, Debug)] -pub struct RequestMessage { +pub struct RequestMessage { pub message_id: u64, pub payload: S::Request, } #[derive(Serialize, Deserialize, Debug)] -pub struct ResponseMessage { +pub struct ResponseMessage { pub message_id: u64, pub payload: S::Response, } -pub struct ServicesServer< - S: ServicesServerDef + Debug + Send + Sync + 'static, - P: ServicesServerProcessor + Send + Sync + 'static, +pub trait ServicesServer { + type IpcRequest: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type IpcResponse: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + + fn start( + reciever: IpcReceiver, + sender: IpcSender, + processor: Arc

, + ) -> Self; + + fn stop(&self); + +} + +pub struct ServicesServerImpl< + P: ServicesServerProcessor + Debug + Send + Sync + 'static, > { join_handle: JoinHandle<()>, - server_def: PhantomData, processor: PhantomData

, } -impl< - S: ServicesServerDef + Debug + Send + Sync + 'static, - P: ServicesServerProcessor + Send + Sync + 'static, - > ServicesServer -{ - pub fn start( - reciever: IpcReceiver>, - sender: IpcSender>, +impl ServicesServer

for ServicesServerImpl

{ + type IpcRequest = RequestMessage

; + type IpcResponse = ResponseMessage

; + + fn start( + reciever: IpcReceiver, + sender: IpcSender, processor: Arc

, ) -> Self { let join_handle = Self::processing_loop(reciever, sender, processor); Self { join_handle, - server_def: PhantomData, processor: PhantomData, } } - pub fn stop(&self) { + fn stop(&self) { self.join_handle.abort(); } +} + +impl< + P: ServicesServerProcessor + Debug + Send + Sync + 'static, + > ServicesServerImpl

+{ fn processing_loop( - reciever: IpcReceiver>, - sender: IpcSender>, + reciever: IpcReceiver>, + sender: IpcSender>, processor: Arc

, ) -> JoinHandle<()> { cube_ext::spawn_blocking(move || loop { @@ -132,9 +174,8 @@ impl< } impl< - S: ServicesServerDef + Debug + Send + Sync + 'static, - P: ServicesServerProcessor + Send + Sync + 'static, -> Drop for ServicesServer { + P: ServicesServerProcessor + Debug + Send + Sync + 'static, +> Drop for ServicesServerImpl

{ fn drop(&mut self) { self.stop(); } From 35e61514e9ce5993a12fb2cfb8b5bf4bbc9f77a3 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Mon, 18 Sep 2023 04:39:29 +0300 Subject: [PATCH 06/15] in work --- rust/cubestore/cubestore/src/cluster/mod.rs | 41 +-- .../cubestore/src/cluster/worker_pool.rs | 265 +++++++++++------- .../cubestore/src/cluster/worker_services.rs | 232 +++++++++++++-- .../src/util/cancellation_token_guard.rs | 3 +- rust/cubestore/cubestore/src/util/mod.rs | 2 +- 5 files changed, 378 insertions(+), 165 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index 0478549d075d8..47aa3b3df610f 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -12,9 +12,11 @@ pub mod rate_limiter; pub mod ingestion; #[cfg(not(target_os = "windows"))] -use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; +use crate::cluster::worker_pool::{worker_main, WorkerPool}; #[cfg(not(target_os = "windows"))] -use crate::cluster::worker_services::{DefaultWorkerServicesDef, DefaultServicesServerProcessor}; +use crate::cluster::worker_services::{ + DefaultServicesServerProcessor, DefaultWorkerServicesDef, WorkerProcessing, +}; use crate::ack_error; use crate::cluster::message::NetworkMessage; @@ -73,6 +75,8 @@ use tokio::time::timeout; use tokio_util::sync::CancellationToken; use tracing::{instrument, Instrument}; +use self::worker_services::WorkerServicesDef; + #[automock] #[async_trait] pub trait Cluster: DIService + Send + Sync { @@ -189,18 +193,7 @@ pub struct ClusterImpl { long_running_job_notify: Arc, meta_store_sender: Sender, #[cfg(not(target_os = "windows"))] - select_process_pool: RwLock< - Option< - Arc< - WorkerPool< - WorkerMessage, - (SchemaRef, Vec, usize), - WorkerProcessor, - DefaultWorkerServicesDef, - >, - >, - >, - >, + select_process_pool: RwLock>>>, config_obj: Arc, query_executor: Arc, stop_token: CancellationToken, @@ -226,12 +219,15 @@ pub struct WorkerProcessor; #[cfg(not(target_os = "windows"))] #[async_trait] -impl MessageProcessor, usize)> - for WorkerProcessor -{ +impl WorkerProcessing for WorkerProcessor { + type Request = WorkerMessage; + type Response = (SchemaRef, Vec, usize); type Config = Config; + type Services = DefaultWorkerServicesDef; - async fn configure() -> Result { + async fn configure( + _services_client: Arc<::Client>, + ) -> Result { let custom_fn = SELECT_WORKER_CONFIGURE_FN.read().unwrap().clone(); let config = if let Some(func) = custom_fn.as_ref() { func().await @@ -322,14 +318,7 @@ impl MessageProcessor, usize), - WorkerProcessor, - DefaultWorkerServicesDef, - >, - ); + crate::util::respawn::register_handler(worker_main::); } lazy_static! { diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 7778e8fad525a..50869d7e3cc66 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -15,29 +15,24 @@ use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; use tokio::runtime::{Builder, Runtime}; use tokio::sync::oneshot::Sender; -use tokio::sync::{oneshot, watch, Notify, RwLock, Mutex}; +use tokio::sync::{oneshot, watch, Mutex, Notify, RwLock}; use tracing::{instrument, Instrument}; use tracing_futures::WithSubscriber; use crate::config::{env_parse, Config, WorkerServices}; +use crate::cluster::worker_services::{ + ServicesClient, ServicesServer, ServicesServerProcessor, WorkerProcessing, WorkerServicesDef, +}; use crate::util::respawn::respawn; use crate::CubeError; use datafusion::cube_ext; use datafusion::cube_ext::catch_unwind::async_try_with_catch_unwind; -use crate::cluster::worker_services::{ServicesServerDef, ServicesServerProcessor, ServicesServer, WorkerServicesDef, WorkerProcessing}; -pub struct WorkerPool< - T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, - R: Serialize + DeserializeOwned + Sync + Send + 'static, - P: MessageProcessor + Sync + Send + 'static, - S: WorkerServicesDef + Sync + Send + 'static, - -> { - queue: Arc>>, +pub struct WorkerPool { + queue: Arc>>, stopped_tx: watch::Sender, - workers: Vec>>, - processor: PhantomData

, - services_processor: Arc + workers: Vec>>, + processor: PhantomData, } pub struct Message< @@ -50,32 +45,9 @@ pub struct Message< dispatcher: tracing::dispatcher::Dispatch, } -#[async_trait] -pub trait MessageProcessor< - T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, - R: Serialize + DeserializeOwned + Sync + Send + 'static, -> -{ - type Config: Sync + Send + Clone + 'static; - - async fn configure() -> Result; - - fn spawn_background_processes(config: Self::Config) -> Result<(), CubeError>; - - async fn process(config: &Self::Config, args: T) -> Result; - - fn process_titile() -> String; -} - -impl< - T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, - R: Serialize + DeserializeOwned + Sync + Send + 'static, - P: MessageProcessor + Sync + Send + 'static, - S: WorkerServicesDef + Sync + Send + 'static, - > WorkerPool -{ +impl WorkerPool { pub fn new( - services_processor: Arc, + services_processor: Arc<::Processor>, num: usize, timeout: Duration, name_prefix: &str, @@ -87,7 +59,7 @@ impl< let mut workers = Vec::new(); for i in 1..=num { - let process = Arc::new(WorkerProcess::::new( + let process = Arc::new(WorkerProcess::::new( services_processor.clone(), format!("{}{}", name_prefix, i), envs.clone(), @@ -103,7 +75,6 @@ impl< queue, workers, processor: PhantomData, - services_processor, } } @@ -116,7 +87,7 @@ impl< join_all(futures).await; } - pub async fn process(&self, message: T) -> Result { + pub async fn process(&self, message: T::Request) -> Result { let (tx, rx) = oneshot::channel(); self.queue.push(Message { message, @@ -162,35 +133,24 @@ impl Drop for ProcessHandleGuard { } } -pub struct WorkerProcess< - T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, - R: Serialize + DeserializeOwned + Sync + Send + 'static, - P: MessageProcessor + Sync + Send + 'static, - S: WorkerServicesDef + Sync + Send + 'static, -> { +pub struct WorkerProcess { name: String, envs: Vec<(String, String)>, - queue: Arc>>, + queue: Arc>>, timeout: Duration, - processor: PhantomData

, + processor: PhantomData, stopped_rx: RwLock>, finished_notify: Arc, - services_processor: Arc, - services_server: Mutex>, + services_processor: Arc<::Processor>, + services_server: Mutex::Server>>, } -impl< - T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, - R: Serialize + DeserializeOwned + Sync + Send + 'static, - P: MessageProcessor + Sync + Send + 'static, - S: WorkerServicesDef + Sync + Send + 'static, - > WorkerProcess -{ +impl WorkerProcess { fn new( - services_processor: Arc, + services_processor: Arc<::Processor>, name: String, envs: Vec<(String, String)>, - queue: Arc>>, + queue: Arc>>, timeout: Duration, stopped_rx: watch::Receiver, ) -> Self { @@ -307,10 +267,17 @@ impl< #[instrument(level = "trace", skip(self, message, args_tx, res_rx))] async fn process_message( &self, - message: T, - args_tx: IpcSender, - res_rx: IpcReceiver>, - ) -> Result<(R, IpcSender, IpcReceiver>), CubeError> { + message: T::Request, + args_tx: IpcSender, + res_rx: IpcReceiver>, + ) -> Result< + ( + T::Response, + IpcSender, + IpcReceiver>, + ), + CubeError, + > { args_tx.send(message)?; let (res, res_rx) = cube_ext::spawn_blocking(move || (res_rx.recv(), res_rx)).await?; Ok((res??, args_tx, res_rx)) @@ -318,7 +285,14 @@ impl< async fn spawn_process( &self, - ) -> Result<(IpcSender, IpcReceiver>, Child), CubeError> { + ) -> Result< + ( + IpcSender, + IpcReceiver>, + Child, + ), + CubeError, + > { { if let Some(services_server) = self.services_server.lock().await.as_ref() { services_server.stop(); @@ -336,7 +310,7 @@ impl< } ctx += &self.name; - let title = P::process_titile(); + let title = T::process_titile(); let mut envs = vec![("CUBESTORE_LOG_CONTEXT".to_string(), ctx)]; envs.extend(self.envs.iter().cloned()); @@ -347,8 +321,7 @@ impl< WorkerProcessArgs { args: args_rx, results: res_tx, - processor: PhantomData::

::default(), - worker_services_def: PhantomData::::default(), + processor: PhantomData::::default(), services_sender: service_request_tx, services_reciever: service_response_rx, }, @@ -356,29 +329,38 @@ impl< &envs, )?; - *self.services_server.lock().await = Some(ServicesServer::start(service_request_rx, service_response_tx, self.services_processor.clone())); + *self.services_server.lock().await = Some(ServicesServer::start( + service_request_rx, + service_response_tx, + self.services_processor.clone(), + )); Ok((args_tx, res_rx, handle)) } } #[derive(Serialize, Deserialize)] -pub struct WorkerProcessArgs { - args: IpcReceiver, - results: IpcSender>, - processor: PhantomData

, - worker_services_def: PhantomData, - services_sender: IpcSender<>::IpcRequest>, - services_reciever: IpcReceiver<>::IpcResponse>, +pub struct WorkerProcessArgs { + args: IpcReceiver, + results: IpcSender>, + processor: PhantomData, + services_sender: IpcSender< + <::Server as ServicesServer< + ::Processor, + >>::IpcRequest, + >, + services_reciever: IpcReceiver< + <::Server as ServicesServer< + ::Processor, + >>::IpcResponse, + >, } -pub fn worker_main(a: WorkerProcessArgs) -> i32 +pub fn worker_main(a: WorkerProcessArgs) -> i32 where - T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, - R: Serialize + DeserializeOwned + Sync + Send + 'static, - P: MessageProcessor, - S: WorkerServicesDef + Sync + Send + 'static, + T: WorkerProcessing + Sync + Send + 'static, { - let (rx, tx) = (a.args, a.results); + let (rx, tx, services_sender, services_reciever) = + (a.args, a.results, a.services_sender, a.services_reciever); let mut tokio_builder = Builder::new_multi_thread(); tokio_builder.enable_all(); tokio_builder.thread_name("cubestore-worker"); @@ -389,13 +371,14 @@ where tokio_builder.thread_stack_size(stack_size); let runtime = tokio_builder.build().unwrap(); worker_setup(&runtime); - println!("!!!!!"); runtime.block_on(async move { - let config = match P::configure().await { + let services_client = + ::Client::connect(services_sender, services_reciever); + let config = match T::configure(services_client).await { Err(e) => { error!( "Error during {} worker configure: {}", - P::process_titile(), + T::process_titile(), e ); return 1; @@ -403,10 +386,10 @@ where Ok(config) => config, }; - if let Err(e) = P::spawn_background_processes(config.clone()) { + if let Err(e) = T::spawn_background_processes(config.clone()) { error!( "Error during {} worker background processes spawn: {}", - P::process_titile(), + T::process_titile(), e ); } @@ -415,7 +398,7 @@ where let res = rx.recv(); match res { Ok(args) => { - let result = match async_try_with_catch_unwind(P::process(&config, args)).await + let result = match async_try_with_catch_unwind(T::process(&config, args)).await { Ok(result) => result, Err(panic) => Err(CubeError::from(panic)), @@ -465,21 +448,25 @@ mod tests { use serde::{Deserialize, Serialize}; use tokio::runtime::Builder; - use crate::cluster::worker_pool::{worker_main, MessageProcessor, WorkerPool}; + use crate::cluster::worker_pool::{worker_main, WorkerPool}; use crate::config::Config; use crate::queryplanner::serialized_plan::SerializedLogicalPlan; use crate::util::respawn; use crate::CubeError; use datafusion::cube_ext; - use crate::cluster::worker_services::{ DefaultServicesServerProcessor, ServicesServerProcessor, DefaultWorkerServicesDef, WorkerServicesDef, ServicesServerImpl}; + use crate::cluster::worker_services::{ + DefaultServicesServerProcessor, DefaultWorkerServicesDef, ServicesClient, + ServicesClientImpl, ServicesServerImpl, ServicesServerProcessor, WorkerProcessing, + WorkerServicesDef, + }; - type TestPool = WorkerPool::; + type TestPool = WorkerPool; #[ctor::ctor] fn test_support_init() { respawn::replace_cmd_args_in_tests(); - respawn::register_handler(worker_main::) + respawn::register_handler(worker_main::) } #[derive(Serialize, Deserialize, Eq, PartialEq, Debug)] @@ -496,10 +483,15 @@ mod tests { pub struct Processor; #[async_trait] - impl MessageProcessor for Processor { + impl WorkerProcessing for Processor { type Config = Config; + type Request = Message; + type Response = Response; + type Services = DefaultWorkerServicesDef; - async fn configure() -> Result { + async fn configure( + _services_client: Arc<::Client>, + ) -> Result { let config = Config::default(); config.configure_injector().await; Ok(config) @@ -640,11 +632,20 @@ mod tests { Ok(()) } - type TestServicePool = WorkerPool::; + type TestServicePool = WorkerPool; #[derive(Debug)] pub struct TestServicesServerProcessor; + #[async_trait] + impl ServicesServerProcessor for TestServicesServerProcessor { + type Request = i64; + type Response = bool; + async fn process(&self, request: i64) -> bool { + request % 2 == 0 + } + } + impl TestServicesServerProcessor { pub fn new() -> Arc { Arc::new(Self {}) @@ -656,23 +657,68 @@ mod tests { impl WorkerServicesDef for TestWorkerServicesDef { type Processor = TestServicesServerProcessor; type Server = ServicesServerImpl; + type Client = ServicesClientImpl; } - #[ctor::ctor] - fn test_services_support_init() { - respawn::replace_cmd_args_in_tests(); - respawn::register_handler(worker_main::) + #[derive(Clone)] + pub struct TestConfig { + pub services_client: Arc<::Client>, + } + + #[derive(Serialize, Deserialize, Eq, PartialEq, Debug)] + pub struct TestServReq { + pub v: i64, } + #[derive(Serialize, Deserialize, Eq, PartialEq, Debug)] + pub enum TestServRes { + Even(i64), + Odd(i64), + } + + pub struct ServProcessor; + #[async_trait] - impl ServicesServerProcessor for TestServicesServerProcessor { - type Request = i64; - type Response = bool; - async fn process(&self, request: i64) -> bool { - request % 2 == 0 + impl WorkerProcessing for ServProcessor { + type Config = TestConfig; + type Request = TestServReq; + type Response = TestServRes; + type Services = TestWorkerServicesDef; + + async fn configure( + services_client: Arc<::Client>, + ) -> Result { + let config = TestConfig { services_client }; + Ok(config) + } + + fn spawn_background_processes(_config: Self::Config) -> Result<(), CubeError> { + Ok(()) + } + async fn process( + config: &Self::Config, + args: TestServReq, + ) -> Result { + let r = config.services_client.send(args.v.clone()).await.unwrap(); + + let res = if r { + TestServRes::Even(args.v) + } else { + TestServRes::Odd(args.v) + }; + Ok(res) + } + + fn process_titile() -> String { + "--sel-worker".to_string() } } + #[ctor::ctor] + fn test_services_support_init() { + respawn::replace_cmd_args_in_tests(); + respawn::register_handler(worker_main::) + } #[test] fn test_services_basic() { @@ -681,7 +727,7 @@ mod tests { runtime.block_on(async move { let pool = Arc::new(TestServicePool::new( TestServicesServerProcessor::new(), - 4, + 1, Duration::from_millis(1000), "test", Vec::new(), @@ -689,10 +735,13 @@ mod tests { let pool_to_move = pool.clone(); cube_ext::spawn(async move { pool_to_move.wait_processing_loops().await }); assert_eq!( - pool.process(Message::Delay(100)).await.unwrap(), - Response::Foo(100) + pool.process(TestServReq { v: 10 }).await.unwrap(), + TestServRes::Even(10) + ); + assert_eq!( + pool.process(TestServReq { v: 11 }).await.unwrap(), + TestServRes::Odd(11) ); - println!("GGGGG"); pool.stop_workers().await.unwrap(); }); } diff --git a/rust/cubestore/cubestore/src/cluster/worker_services.rs b/rust/cubestore/cubestore/src/cluster/worker_services.rs index 272d72cbf8850..5a6b40c5e1f75 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_services.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_services.rs @@ -1,39 +1,40 @@ -use crate::CubeError; use crate::util::cancellation_token_guard::CancellationGuard; +use crate::CubeError; use async_trait::async_trait; use datafusion::cube_ext; +use deadqueue::unlimited; use ipc_channel::ipc::{IpcReceiver, IpcSender}; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; use std::fmt::Debug; use std::marker::PhantomData; use std::sync::Arc; +use std::time::Duration; +use tokio::sync::{broadcast, oneshot, Notify, RwLock}; use tokio::task::JoinHandle; use tokio_util::sync::CancellationToken; - #[async_trait] pub trait WorkerProcessing { type Config: Sync + Send + Clone + 'static; type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; - type ServicesServer: ServicesServerDef; + type Services: WorkerServicesDef; - async fn configure() -> Result; + async fn configure( + services_client: Arc<::Client>, + ) -> Result; fn spawn_background_processes(config: Self::Config) -> Result<(), CubeError>; - async fn process(config: &Self::Config, args: Self::Request) -> Result; + async fn process( + config: &Self::Config, + args: Self::Request, + ) -> Result; fn process_titile() -> String; } - -pub trait ServicesServerDef { - type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; - type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; -} - #[async_trait] pub trait ServicesServerProcessor { type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; @@ -44,6 +45,7 @@ pub trait ServicesServerProcessor { pub trait WorkerServicesDef { type Processor: ServicesServerProcessor + Send + Sync + 'static; type Server: ServicesServer + Send + Sync + 'static; + type Client: ServicesClient + Send + Sync + 'static; } #[derive(Debug)] @@ -52,9 +54,9 @@ pub struct DefaultWorkerServicesDef; impl WorkerServicesDef for DefaultWorkerServicesDef { type Processor = DefaultServicesServerProcessor; type Server = ServicesServerImpl; + type Client = ServicesClientImpl; } - #[derive(Debug)] pub struct DefaultServicesServerProcessor; @@ -96,17 +98,16 @@ pub trait ServicesServer { ) -> Self; fn stop(&self); - } -pub struct ServicesServerImpl< - P: ServicesServerProcessor + Debug + Send + Sync + 'static, -> { +pub struct ServicesServerImpl { join_handle: JoinHandle<()>, processor: PhantomData

, } -impl ServicesServer

for ServicesServerImpl

{ +impl ServicesServer

+ for ServicesServerImpl

+{ type IpcRequest = RequestMessage

; type IpcResponse = ResponseMessage

; @@ -127,21 +128,14 @@ impl ServicesServer< } } -impl< - P: ServicesServerProcessor + Debug + Send + Sync + 'static, - > ServicesServerImpl

-{ - +impl ServicesServerImpl

{ fn processing_loop( reciever: IpcReceiver>, sender: IpcSender>, processor: Arc

, ) -> JoinHandle<()> { cube_ext::spawn_blocking(move || loop { - println!("##########"); let req = reciever.recv(); - println!("req: {:?}", req); - println!("111111111"); let RequestMessage { message_id, @@ -173,9 +167,191 @@ impl< } } -impl< - P: ServicesServerProcessor + Debug + Send + Sync + 'static, -> Drop for ServicesServerImpl

{ +impl Drop for ServicesServerImpl

{ + fn drop(&mut self) { + self.stop(); + } +} + +#[async_trait] +pub trait ServicesClient< + P: ServicesServerProcessor + Send + Sync + 'static, + S: ServicesServer

+ Send + Sync + 'static, +> +{ + fn connect( + sender: IpcSender, + reciever: IpcReceiver, + ) -> Arc; + async fn send(&self, request: P::Request) -> Result; + fn stop(&self); +} + +struct ServicesClientMessage< + T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, + R: Serialize + DeserializeOwned + Sync + Send + 'static, +> { + message: T, + result_sender: oneshot::Sender>, +} + +pub struct ServicesClientImpl { + queue: Arc>>, + handle: JoinHandle<()>, + processor: PhantomData

, +} + +#[async_trait] +impl + ServicesClient> for ServicesClientImpl

+{ + fn connect( + sender: IpcSender>, + reciever: IpcReceiver>, + ) -> Arc { + let queue = Arc::new(unlimited::Queue::new()); + let handle = Self::processing_loop(sender, reciever, queue.clone()); + Arc::new(Self { + processor: PhantomData, + handle, + queue, + }) + } + async fn send(&self, request: P::Request) -> Result { + let (tx, rx) = oneshot::channel(); + self.queue.push(ServicesClientMessage { + message: request, + result_sender: tx, + }); + rx.await? + } + fn stop(&self) { + self.handle.abort(); + } +} + +impl ServicesClientImpl

{ + fn processing_loop( + sender: IpcSender>, + reciever: IpcReceiver>, + queue: Arc>>, + ) -> JoinHandle<()> { + let (message_broadcast_tx, _) = broadcast::channel(10000); + + let message_broadcast_tx_to_move = message_broadcast_tx.clone(); + + let recieve_loop = cube_ext::spawn_blocking(move || loop { + let res = reciever.recv(); + match res { + Ok(ResponseMessage { + message_id, + payload, + }) => { + if let Err(e) = message_broadcast_tx_to_move + .send((message_id, Arc::new(RwLock::new(Some(payload))))) + { + log::error!( + "Worker broadcasting processed message id {}: {}", + message_id, + e + ); + } + } + Err(e) => { + log::error!("Error while reading ipc service response: {:?}", e); + break; + } + } + }); + + cube_ext::spawn(async move { + let mut id_counter = 0; + loop { + let ServicesClientMessage { + message, + result_sender, + } = queue.pop().await; + + let message_id = id_counter; + id_counter += 1; + + let ipc_message = RequestMessage { + message_id, + payload: message, + }; + + if let Err(e) = sender.send(ipc_message) { + log::error!("Error while sending ipc service request: {:?}", e); + break; + } + + let mut broadcast_rx = message_broadcast_tx.subscribe(); + + cube_ext::spawn(async move { + loop { + let broadcast_message = tokio::select! { + _ = tokio::time::sleep(Duration::from_secs(5)) => { //TODO! config + Err(CubeError::internal(format!( + "Worker service timeout for message id: {}", + message_id + ))) + } + msg = broadcast_rx.recv() => { + Ok(msg) + } + }; + + let res = match broadcast_message { + Ok(r) => match r { + Ok((id, res)) => { + if id == message_id { + let mut option = res.write().await; + if let Some(res) = option.take() { + Some(Ok(res)) + } else { + Some(Err(CubeError::internal(format!( + "Worker service result consumed by another listener for message id {}", + message_id + )))) + } + } else { + None + } + } + Err(e) => Some(Err(CubeError::internal(format!( + "Worker service processing error for message id {}: {}", + message_id, e + )))), + }, + Err(e) => { + log::error!("Worker service read from broadcast error for message id {}: {}", message_id, e); + Some(Err(CubeError::internal(format!( + "Worker service read from broadcast error for message id {}: {}", + e, + message_id + )))) + } + }; + + if let Some(res) = res { + if let Err(_) = result_sender.send(res) { + log::error!( + "Worker service send result error for message id {}", + message_id + ); + } + break; + } + } + }); + } + }); + + recieve_loop + } +} + +impl Drop for ServicesClientImpl

{ fn drop(&mut self) { self.stop(); } diff --git a/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs b/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs index 8f47c57479346..2efc350048d0a 100644 --- a/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs +++ b/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs @@ -1,10 +1,9 @@ use tokio_util::sync::CancellationToken; pub struct CancellationGuard<'a> { - token: &'a CancellationToken + token: &'a CancellationToken, } impl<'a> CancellationGuard<'a> { - pub fn new(token: &'a CancellationToken) -> Self { Self { token } } diff --git a/rust/cubestore/cubestore/src/util/mod.rs b/rust/cubestore/cubestore/src/util/mod.rs index e48af1c6d8251..f0afd64eeb118 100644 --- a/rust/cubestore/cubestore/src/util/mod.rs +++ b/rust/cubestore/cubestore/src/util/mod.rs @@ -1,5 +1,6 @@ pub mod aborting_join_handle; pub mod batch_memory; +pub mod cancellation_token_guard; pub mod decimal; pub mod error; pub mod int96; @@ -13,7 +14,6 @@ pub mod metrics; pub mod respawn; pub mod strings; pub mod time_span; -pub mod cancellation_token_guard; pub use malloc_trim_loop::spawn_malloc_trim_loop; From f3389d04953fd5c7077a993ec46f81aaabbc2233 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Mon, 18 Sep 2023 17:34:40 +0300 Subject: [PATCH 07/15] change interface of remote_fs --- rust/cubestore/cubestore/src/cluster/mod.rs | 8 +- .../cubestore/src/cluster/worker_pool.rs | 3 +- rust/cubestore/cubestore/src/config/mod.rs | 8 +- rust/cubestore/cubestore/src/import/mod.rs | 4 +- rust/cubestore/cubestore/src/metastore/mod.rs | 8 +- .../cubestore/src/metastore/rocks_fs.rs | 42 ++--- .../cubestore/src/remotefs/cleanup.rs | 18 +- rust/cubestore/cubestore/src/remotefs/gcs.rs | 57 ++++-- .../cubestore/cubestore/src/remotefs/minio.rs | 57 ++++-- rust/cubestore/cubestore/src/remotefs/mod.rs | 171 +++++++++++------- .../cubestore/cubestore/src/remotefs/queue.rs | 121 ++++++++----- rust/cubestore/cubestore/src/remotefs/s3.rs | 55 ++++-- rust/cubestore/cubestore/src/scheduler/mod.rs | 4 +- rust/cubestore/cubestore/src/sql/mod.rs | 57 ++++-- .../cubestore/src/store/compaction.rs | 22 +-- rust/cubestore/cubestore/src/store/mod.rs | 14 +- 16 files changed, 395 insertions(+), 254 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index 47aa3b3df610f..64abaf2df21bb 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -606,7 +606,7 @@ impl Cluster for ClusterImpl { NetworkMessage::WarmupDownload(remote_path, expected_file_size) => { let res = self .remote_fs - .download_file(&remote_path, expected_file_size) + .download_file(remote_path, expected_file_size) .await; NetworkMessage::WarmupDownloadResult(res.map(|_| ())) } @@ -1300,7 +1300,7 @@ impl ClusterImpl { async move { let res = self .remote_fs - .download_file(remote, file_size.clone()) + .download_file(remote.clone(), file_size.clone()) .await; deactivate_table_on_corrupt_data( meta_store, @@ -1562,7 +1562,7 @@ impl ClusterImpl { // so they are not errors most of the time. ack_error!( self.remote_fs - .download_file(&file, p.get_row().file_size()) + .download_file(file, p.get_row().file_size()) .await ); } @@ -1577,7 +1577,7 @@ impl ClusterImpl { let result = self .remote_fs .download_file( - &chunk_file_name(c.get_id(), c.get_row().suffix()), + chunk_file_name(c.get_id(), c.get_row().suffix()), c.get_row().file_size(), ) .await; diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 50869d7e3cc66..15af62b23f3e8 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -5,7 +5,6 @@ use std::process::{Child, ExitStatus}; use std::sync::Arc; use std::time::Duration; -use async_trait::async_trait; use deadqueue::unlimited; use futures::future::join_all; use ipc_channel::ipc; @@ -21,7 +20,7 @@ use tracing_futures::WithSubscriber; use crate::config::{env_parse, Config, WorkerServices}; use crate::cluster::worker_services::{ - ServicesClient, ServicesServer, ServicesServerProcessor, WorkerProcessing, WorkerServicesDef, + ServicesClient, ServicesServer, WorkerProcessing, WorkerServicesDef, }; use crate::util::respawn::respawn; use crate::CubeError; diff --git a/rust/cubestore/cubestore/src/config/mod.rs b/rust/cubestore/cubestore/src/config/mod.rs index 827b7cfded158..ce60a54cc927f 100644 --- a/rust/cubestore/cubestore/src/config/mod.rs +++ b/rust/cubestore/cubestore/src/config/mod.rs @@ -1652,10 +1652,10 @@ impl Config { let remote_fs = self.remote_fs().await.unwrap(); let _ = fs::remove_dir_all(store_path.clone()); if clean_remote { - let remote_files = remote_fs.list("").await.unwrap(); + let remote_files = remote_fs.list("".to_string()).await.unwrap(); for file in remote_files { debug!("Cleaning {}", file); - let _ = remote_fs.delete_file(file.as_str()).await.unwrap(); + let _ = remote_fs.delete_file(file).await.unwrap(); } } { @@ -1679,9 +1679,9 @@ impl Config { let _ = DB::destroy(&Options::default(), self.cache_store_path()); let _ = fs::remove_dir_all(store_path.clone()); if clean_remote { - let remote_files = remote_fs.list("").await.unwrap(); + let remote_files = remote_fs.list("".to_string()).await.unwrap(); for file in remote_files { - let _ = remote_fs.delete_file(file.as_str()).await; + let _ = remote_fs.delete_file(file).await; } } } diff --git a/rust/cubestore/cubestore/src/import/mod.rs b/rust/cubestore/cubestore/src/import/mod.rs index 0f8c8376563f0..399fc690665fe 100644 --- a/rust/cubestore/cubestore/src/import/mod.rs +++ b/rust/cubestore/cubestore/src/import/mod.rs @@ -646,7 +646,7 @@ impl ImportServiceImpl { async fn download_temp_file(&self, location: &str) -> Result { let to_download = LocationHelper::temp_uploads_path(location); // TODO check file size - let local_file = self.remote_fs.download_file(&to_download, None).await?; + let local_file = self.remote_fs.download_file(to_download, None).await?; Ok(File::open(local_file.clone()) .await .map_err(|e| CubeError::internal(format!("Open temp_file {}: {}", local_file, e)))?) @@ -656,7 +656,7 @@ impl ImportServiceImpl { // TODO There also should be a process which collects orphaned uploads due to failed imports if location.starts_with("temp://") { self.remote_fs - .delete_file(&LocationHelper::temp_uploads_path(location)) + .delete_file(LocationHelper::temp_uploads_path(location)) .await?; } Ok(()) diff --git a/rust/cubestore/cubestore/src/metastore/mod.rs b/rust/cubestore/cubestore/src/metastore/mod.rs index 19b1b28dc7b0d..b901a51e91a1f 100644 --- a/rust/cubestore/cubestore/src/metastore/mod.rs +++ b/rust/cubestore/cubestore/src/metastore/mod.rs @@ -5944,7 +5944,7 @@ mod tests { .await .unwrap(); rocks_meta_store.run_upload().await.unwrap(); - let uploaded = remote_fs.list("metastore-").await.unwrap(); + let uploaded = remote_fs.list("metastore-".to_string()).await.unwrap(); assert!(uploaded.is_empty()); rocks_meta_store.upload_check_point().await.unwrap(); @@ -5957,7 +5957,7 @@ mod tests { rocks_meta_store.run_upload().await.unwrap(); - let uploaded = remote_fs.list("metastore-").await.unwrap(); + let uploaded = remote_fs.list("metastore-".to_string()).await.unwrap(); let logs_uploaded = uploaded .into_iter() @@ -5968,7 +5968,7 @@ mod tests { rocks_meta_store.run_upload().await.unwrap(); - let uploaded = remote_fs.list("metastore-").await.unwrap(); + let uploaded = remote_fs.list("metastore-".to_string()).await.unwrap(); let logs_uploaded = uploaded .into_iter() @@ -5985,7 +5985,7 @@ mod tests { rocks_meta_store.run_upload().await.unwrap(); - let uploaded = remote_fs.list("metastore-").await.unwrap(); + let uploaded = remote_fs.list("metastore-".to_string()).await.unwrap(); let logs_uploaded = uploaded .into_iter() diff --git a/rust/cubestore/cubestore/src/metastore/rocks_fs.rs b/rust/cubestore/cubestore/src/metastore/rocks_fs.rs index de6b721ceab49..0dd8d34cbe962 100644 --- a/rust/cubestore/cubestore/src/metastore/rocks_fs.rs +++ b/rust/cubestore/cubestore/src/metastore/rocks_fs.rs @@ -92,7 +92,7 @@ impl BaseRocksStoreFs { } pub async fn make_local_metastore_dir(&self) -> Result { - let meta_store_path = self.remote_fs.local_file(&self.name).await?; + let meta_store_path = self.remote_fs.local_file(self.name.to_string()).await?; fs::create_dir_all(meta_store_path.to_string()).await?; Ok(meta_store_path) } @@ -119,9 +119,9 @@ impl BaseRocksStoreFs { .map(|f| { let remote_fs = self.remote_fs.clone(); return async move { - let local = remote_fs.local_file(&f).await?; + let local = remote_fs.local_file(f.clone()).await?; // TODO persist file size - Ok::<_, CubeError>((f.clone(), remote_fs.upload_file(&local, &f).await?)) + Ok::<_, CubeError>((f.clone(), remote_fs.upload_file(local, f.clone()).await?)) }; }) .collect::>(), @@ -133,7 +133,7 @@ impl BaseRocksStoreFs { Ok(upload_results) } pub async fn delete_old_snapshots(&self) -> Result, CubeError> { - let existing_metastore_files = self.remote_fs.list(&format!("{}-", self.name)).await?; + let existing_metastore_files = self.remote_fs.list(format!("{}-", self.name)).await?; let candidates = existing_metastore_files .iter() .filter_map(|existing| { @@ -190,7 +190,7 @@ impl BaseRocksStoreFs { for v in join_all( to_delete .iter() - .map(|f| self.remote_fs.delete_file(&f)) + .map(|f| self.remote_fs.delete_file(f.to_string())) .collect::>(), ) .await @@ -208,7 +208,7 @@ impl BaseRocksStoreFs { pub async fn is_remote_metadata_exists(&self) -> Result { let res = self .remote_fs - .list(&format!("{}-current", self.name)) + .list(format!("{}-current", self.name)) .await? .len() > 0; @@ -218,7 +218,7 @@ impl BaseRocksStoreFs { pub async fn load_current_snapshot_id(&self) -> Result, CubeError> { if self .remote_fs - .list(&format!("{}-current", self.name)) + .list(format!("{}-current", self.name)) .await? .len() == 0 @@ -230,13 +230,13 @@ impl BaseRocksStoreFs { let current_metastore_file = self .remote_fs - .local_file(&format!("{}-current", self.name)) + .local_file(format!("{}-current", self.name)) .await?; if fs::metadata(current_metastore_file.as_str()).await.is_ok() { fs::remove_file(current_metastore_file.as_str()).await?; } self.remote_fs - .download_file(&format!("{}-current", self.name), None) + .download_file(format!("{}-current", self.name), None) .await?; self.parse_local_current_snapshot_id().await } @@ -244,7 +244,7 @@ impl BaseRocksStoreFs { pub async fn parse_local_current_snapshot_id(&self) -> Result, CubeError> { let current_metastore_file = self .remote_fs - .local_file(&format!("{}-current", self.name)) + .local_file(format!("{}-current", self.name)) .await?; let re = Regex::new(&format!(r"^{}-(\d+)", &self.name)).unwrap(); @@ -268,7 +268,7 @@ impl BaseRocksStoreFs { pub async fn files_to_load(&self, snapshot: u128) -> Result, CubeError> { let res = self .remote_fs - .list_with_metadata(&format!("{}-{}", self.name, snapshot)) + .list_with_metadata(format!("{}-{}", self.name, snapshot)) .await? .into_iter() .map(|f| (f.remote_path, f.file_size)) @@ -307,8 +307,8 @@ impl MetaStoreFs for BaseRocksStoreFs { let meta_store_path = self.make_local_metastore_dir().await?; for (file, _) in to_load.iter() { // TODO check file size - self.remote_fs.download_file(file, None).await?; - let local = self.remote_fs.local_file(file).await?; + self.remote_fs.download_file(file.clone(), None).await?; + let local = self.remote_fs.local_file(file.clone()).await?; let path = Path::new(&local); fs::copy( path, @@ -326,7 +326,7 @@ impl MetaStoreFs for BaseRocksStoreFs { .await; } } else { - trace!("Can't find {}-current in {:?}", self.name, self.remote_fs); + //TODO FIX IT (Debug for ext service) trace!("Can't find {}-current in {:?}", self.name, self.remote_fs); } info!("Creating {} from scratch in {}", self.name, path); } else { @@ -348,10 +348,10 @@ impl MetaStoreFs for BaseRocksStoreFs { serializer: &WriteBatchContainer, ) -> Result { let log_name = format!("{}/{}.flex", dir, seq_number); - let file_name = self.remote_fs.local_file(&log_name).await?; + let file_name = self.remote_fs.local_file(log_name.clone()).await?; serializer.write_to_file(&file_name).await?; // TODO persist file size - self.remote_fs.upload_file(&file_name, &log_name).await + self.remote_fs.upload_file(file_name, log_name).await } async fn upload_checkpoint( @@ -375,7 +375,7 @@ impl MetaStoreFs for BaseRocksStoreFs { ) -> Result<(), CubeError> { let logs_to_batch = self .remote_fs - .list(&format!("{}-{}-logs", self.name, snapshot)) + .list(format!("{}-{}-logs", self.name, snapshot)) .await?; let mut logs_to_batch_to_seq = logs_to_batch .into_iter() @@ -393,7 +393,7 @@ impl MetaStoreFs for BaseRocksStoreFs { logs_to_batch_to_seq.sort_unstable_by_key(|(_, seq)| *seq); for (log_file, _) in logs_to_batch_to_seq.iter() { - let path_to_log = self.remote_fs.local_file(log_file).await?; + let path_to_log = self.remote_fs.local_file(log_file.clone()).await?; let batch = WriteBatchContainer::read_from_file(&path_to_log).await; if let Ok(batch) = batch { let db = rocks_store.db.clone(); @@ -413,7 +413,7 @@ impl MetaStoreFs for BaseRocksStoreFs { let remote_fs = self.remote_fs(); let re = Regex::new(&*format!(r"^{}-(\d+)/", self.get_name())).unwrap(); - let stores = remote_fs.list(&format!("{}-", self.get_name())).await?; + let stores = remote_fs.list(format!("{}-", self.get_name())).await?; let mut snapshots = BTreeSet::new(); for store in stores.iter() { let parse_result = re @@ -450,8 +450,8 @@ impl MetaStoreFs for BaseRocksStoreFs { self.remote_fs .upload_file( - file_path.keep()?.to_str().unwrap(), - &format!("{}-current", self.name), + file_path.keep()?.to_str().unwrap().to_string(), + format!("{}-current", self.name), ) .await?; Ok(()) diff --git a/rust/cubestore/cubestore/src/remotefs/cleanup.rs b/rust/cubestore/cubestore/src/remotefs/cleanup.rs index e136c5a315e99..4b3d5746c46a5 100644 --- a/rust/cubestore/cubestore/src/remotefs/cleanup.rs +++ b/rust/cubestore/cubestore/src/remotefs/cleanup.rs @@ -81,7 +81,7 @@ impl RemoteFsCleanup { files_to_remove.remove(&f); } for f in files_to_remove.iter() { - if let Err(e) = self.remote_fs.delete_file(&f).await { + if let Err(e) = self.remote_fs.delete_file(f.clone()).await { log::error!("Error while deleting {} in remote fs: {}", f, e); } } @@ -89,7 +89,7 @@ impl RemoteFsCleanup { files_to_remove.clear(); - let res_remote_files = remote_fs.list_with_metadata("").await; + let res_remote_files = remote_fs.list_with_metadata("".to_string()).await; let remote_files = match res_remote_files { Err(e) => { log::error!("could not get the list of remote files: {}", e); @@ -141,7 +141,7 @@ impl RemoteFsCleanup { async fn cleanup_local_files_loop(&self) { let token = self.stopped_token.child_token(); let remote_fs = self.remote_fs.clone(); - let local_dir = remote_fs.local_path().await; + let local_dir = remote_fs.local_path().await.unwrap(); let cleanup_interval = Duration::from_secs(self.config.local_files_cleanup_interval_secs()); let cleanup_local_files_delay = Duration::from_secs(self.config.local_files_cleanup_delay_secs()); @@ -331,10 +331,10 @@ mod test { let files = remove_root_paritition(meta_store.get_all_filenames().await.unwrap()); assert_eq!(files.len(), 2); for f in files.iter() { - let path = remote_fs.local_file(&f).await.unwrap(); + let path = remote_fs.local_file(f.clone()).await.unwrap(); assert!(Path::new(&path).exists()); } - let path = remote_fs.local_file("metastore").await.unwrap(); + let path = remote_fs.local_file("metastore".to_string()).await.unwrap(); assert!(Path::new(&path).exists()); meta_store @@ -347,18 +347,18 @@ mod test { 1 ); for f in files.iter() { - let path = remote_fs.local_file(&f).await.unwrap(); + let path = remote_fs.local_file(f.clone()).await.unwrap(); assert!(Path::new(&path).exists()); } Delay::new(Duration::from_millis(4000)).await; // TODO logger init conflict - let path = remote_fs.local_file(&files[0]).await.unwrap(); + let path = remote_fs.local_file(files[0].clone()).await.unwrap(); assert!(!Path::new(&path).exists()); - let path = remote_fs.local_file(&files[1]).await.unwrap(); + let path = remote_fs.local_file(files[1].clone()).await.unwrap(); assert!(Path::new(&path).exists()); - let path = remote_fs.local_file("metastore").await.unwrap(); + let path = remote_fs.local_file("metastore".to_string()).await.unwrap(); assert!(Path::new(&path).exists()); let _ = service.exec_query("SELECT * FROM test.tst").await.unwrap(); diff --git a/rust/cubestore/cubestore/src/remotefs/gcs.rs b/rust/cubestore/cubestore/src/remotefs/gcs.rs index cf497f8f30116..c631e061de97c 100644 --- a/rust/cubestore/cubestore/src/remotefs/gcs.rs +++ b/rust/cubestore/cubestore/src/remotefs/gcs.rs @@ -1,6 +1,6 @@ use crate::app_metrics; use crate::di_service; -use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs}; +use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs, CommonRemoteFsUtils}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -119,10 +119,27 @@ di_service!(GCSRemoteFs, [RemoteFs]); #[async_trait] impl RemoteFs for GCSRemoteFs { + + async fn temp_upload_path(&self, remote_path: String) -> Result { + CommonRemoteFsUtils::temp_upload_path(self, remote_path).await + } + + async fn uploads_dir(&self) -> Result { + CommonRemoteFsUtils::uploads_dir(self).await + } + + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError> { + CommonRemoteFsUtils::check_upload_file(self, remote_path, expected_size).await + } + async fn upload_file( &self, - temp_upload_path: &str, - remote_path: &str, + temp_upload_path: String, + remote_path: String, ) -> Result { app_metrics::REMOTE_FS_OPERATION_CORE.add_with_tags( 1, @@ -133,23 +150,23 @@ impl RemoteFs for GCSRemoteFs { ); let time = SystemTime::now(); debug!("Uploading {}", remote_path); - let file = File::open(temp_upload_path).await?; + let file = File::open(temp_upload_path.clone()).await?; let stream = FramedRead::new(file, BytesCodec::new()); let stream = stream.map(|r| r.map(|b| b.to_vec())); Object::create_streamed( self.bucket.as_str(), stream, None, - self.gcs_path(remote_path).as_str(), + self.gcs_path(&remote_path).as_str(), "application/octet-stream", ) .await?; - let size = fs::metadata(temp_upload_path).await?.len(); - self.check_upload_file(remote_path, size).await?; + let size = fs::metadata(temp_upload_path.clone()).await?.len(); + self.check_upload_file(remote_path.clone(), size).await?; - let local_path = self.dir.as_path().join(remote_path); - if Path::new(temp_upload_path) != local_path { + let local_path = self.dir.as_path().join(&remote_path); + if Path::new(&temp_upload_path) != local_path { fs::create_dir_all(local_path.parent().unwrap()) .await .map_err(|e| { @@ -167,10 +184,10 @@ impl RemoteFs for GCSRemoteFs { async fn download_file( &self, - remote_path: &str, + remote_path: String, _expected_file_size: Option, ) -> Result { - let mut local_file = self.dir.as_path().join(remote_path); + let mut local_file = self.dir.as_path().join(&remote_path); let local_dir = local_file.parent().unwrap(); let downloads_dirs = local_dir.join("downloads"); @@ -192,7 +209,7 @@ impl RemoteFs for GCSRemoteFs { let mut writer = BufWriter::new(tokio::fs::File::from_std(temp_file)); let mut stream = Object::download_streamed( self.bucket.as_str(), - self.gcs_path(remote_path).as_str(), + self.gcs_path(&remote_path).as_str(), ) .await?; @@ -220,7 +237,7 @@ impl RemoteFs for GCSRemoteFs { Ok(local_file.into_os_string().into_string().unwrap()) } - async fn delete_file(&self, remote_path: &str) -> Result<(), CubeError> { + async fn delete_file(&self, remote_path: String) -> Result<(), CubeError> { app_metrics::REMOTE_FS_OPERATION_CORE.add_with_tags( 1, Some(&vec![ @@ -230,7 +247,7 @@ impl RemoteFs for GCSRemoteFs { ); let time = SystemTime::now(); debug!("Deleting {}", remote_path); - Object::delete(self.bucket.as_str(), self.gcs_path(remote_path).as_str()).await?; + Object::delete(self.bucket.as_str(), self.gcs_path(&remote_path).as_str()).await?; info!("Deleting {} ({:?})", remote_path, time.elapsed()?); let _guard = acquire_lock("delete file", self.delete_mut.lock()).await?; @@ -244,7 +261,7 @@ impl RemoteFs for GCSRemoteFs { Ok(()) } - async fn list(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list(&self, remote_prefix: String) -> Result, CubeError> { Ok(self .list_with_metadata(remote_prefix) .await? @@ -253,8 +270,8 @@ impl RemoteFs for GCSRemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: &str) -> Result, CubeError> { - let prefix = self.gcs_path(remote_prefix); + async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + let prefix = self.gcs_path(&remote_prefix); let list = Object::list_prefix(self.bucket.as_str(), prefix.as_str()).await?; let leading_slash = Regex::new(format!("^{}", self.gcs_path("")).as_str()).unwrap(); let result = list @@ -291,11 +308,11 @@ impl RemoteFs for GCSRemoteFs { Ok(result) } - async fn local_path(&self) -> String { - self.dir.to_str().unwrap().to_owned() + async fn local_path(&self) -> Result { + Ok(self.dir.to_str().unwrap().to_owned()) } - async fn local_file(&self, remote_path: &str) -> Result { + async fn local_file(&self, remote_path: String) -> Result { let buf = self.dir.join(remote_path); fs::create_dir_all(buf.parent().unwrap()).await?; Ok(buf.to_str().unwrap().to_string()) diff --git a/rust/cubestore/cubestore/src/remotefs/minio.rs b/rust/cubestore/cubestore/src/remotefs/minio.rs index 0565ccf8da6b4..5f008a322589a 100644 --- a/rust/cubestore/cubestore/src/remotefs/minio.rs +++ b/rust/cubestore/cubestore/src/remotefs/minio.rs @@ -1,5 +1,5 @@ use crate::di_service; -use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs}; +use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs, CommonRemoteFsUtils}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -144,15 +144,32 @@ di_service!(MINIORemoteFs, [RemoteFs]); #[async_trait] impl RemoteFs for MINIORemoteFs { + + async fn temp_upload_path(&self, remote_path: String) -> Result { + CommonRemoteFsUtils::temp_upload_path(self, remote_path).await + } + + async fn uploads_dir(&self) -> Result { + CommonRemoteFsUtils::uploads_dir(self).await + } + + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError> { + CommonRemoteFsUtils::check_upload_file(self, remote_path, expected_size).await + } + async fn upload_file( &self, - temp_upload_path: &str, - remote_path: &str, + temp_upload_path: String, + remote_path: String, ) -> Result { { let time = SystemTime::now(); debug!("Uploading {}", remote_path); - let path = self.s3_path(remote_path); + let path = self.s3_path(&remote_path); info!("path {}", remote_path); let bucket = self.bucket.read().unwrap().clone(); let temp_upload_path_copy = temp_upload_path.to_string(); @@ -170,11 +187,11 @@ impl RemoteFs for MINIORemoteFs { } } - let size = fs::metadata(temp_upload_path).await?.len(); - self.check_upload_file(remote_path, size).await?; + let size = fs::metadata(&temp_upload_path).await?.len(); + self.check_upload_file(remote_path.clone(), size).await?; - let local_path = self.dir.as_path().join(remote_path); - if Path::new(temp_upload_path) != local_path { + let local_path = self.dir.as_path().join(&remote_path); + if Path::new(&temp_upload_path) != local_path { fs::create_dir_all(local_path.parent().unwrap()) .await .map_err(|e| { @@ -191,10 +208,10 @@ impl RemoteFs for MINIORemoteFs { async fn download_file( &self, - remote_path: &str, + remote_path: String, _expected_file_size: Option, ) -> Result { - let local_file = self.dir.as_path().join(remote_path); + let local_file = self.dir.as_path().join(&remote_path); let local_dir = local_file.parent().unwrap(); let downloads_dir = local_dir.join("downloads"); @@ -204,7 +221,7 @@ impl RemoteFs for MINIORemoteFs { if !local_file.exists() { let time = SystemTime::now(); debug!("Downloading {}", remote_path); - let path = self.s3_path(remote_path); + let path = self.s3_path(&remote_path); info!("path {}", remote_path); let bucket = self.bucket.read().unwrap().clone(); let status_code = cube_ext::spawn_blocking(move || -> Result { @@ -230,11 +247,11 @@ impl RemoteFs for MINIORemoteFs { Ok(local_file_str) } - async fn delete_file(&self, remote_path: &str) -> Result<(), CubeError> { + async fn delete_file(&self, remote_path: String) -> Result<(), CubeError> { let time = SystemTime::now(); debug!("Deleting {}", remote_path); info!("remote_path {}", remote_path); - let path = self.s3_path(remote_path); + let path = self.s3_path(&remote_path); info!("path {}", remote_path); let bucket = self.bucket.read().unwrap().clone(); let (_, status_code) = @@ -248,7 +265,7 @@ impl RemoteFs for MINIORemoteFs { } let _guard = acquire_lock("delete file", self.delete_mut.lock()).await?; - let local = self.dir.as_path().join(remote_path); + let local = self.dir.as_path().join(&remote_path); if fs::metadata(local.clone()).await.is_ok() { fs::remove_file(local.clone()).await?; LocalDirRemoteFs::remove_empty_paths(self.dir.as_path().to_path_buf(), local.clone()) @@ -258,7 +275,7 @@ impl RemoteFs for MINIORemoteFs { Ok(()) } - async fn list(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list(&self, remote_prefix: String) -> Result, CubeError> { Ok(self .list_with_metadata(remote_prefix) .await? @@ -267,8 +284,8 @@ impl RemoteFs for MINIORemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: &str) -> Result, CubeError> { - let path = self.s3_path(remote_prefix); + async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + let path = self.s3_path(&remote_prefix); let bucket = self.bucket.read().unwrap().clone(); let list = cube_ext::spawn_blocking(move || bucket.list_blocking(path, None)).await??; let leading_slash = Regex::new(format!("^{}", self.s3_path("")).as_str()).unwrap(); @@ -290,11 +307,11 @@ impl RemoteFs for MINIORemoteFs { Ok(result) } - async fn local_path(&self) -> String { - self.dir.to_str().unwrap().to_owned() + async fn local_path(&self) -> Result { + Ok(self.dir.to_str().unwrap().to_owned()) } - async fn local_file(&self, remote_path: &str) -> Result { + async fn local_file(&self, remote_path: String) -> Result { let buf = self.dir.join(remote_path); fs::create_dir_all(buf.parent().unwrap()).await?; Ok(buf.to_str().unwrap().to_string()) diff --git a/rust/cubestore/cubestore/src/remotefs/mod.rs b/rust/cubestore/cubestore/src/remotefs/mod.rs index 693b045644244..7eb2c1064dd0e 100644 --- a/rust/cubestore/cubestore/src/remotefs/mod.rs +++ b/rust/cubestore/cubestore/src/remotefs/mod.rs @@ -20,8 +20,9 @@ use std::sync::Arc; use tempfile::{NamedTempFile, PathPersistError}; use tokio::fs; use tokio::sync::{Mutex, RwLock}; +use serde::{Deserialize, Serialize}; -#[derive(Debug, Clone)] +#[derive(Debug, Clone, Serialize, Deserialize)] pub struct RemoteFile { pub remote_path: String, pub updated: DateTime, @@ -38,20 +39,64 @@ impl RemoteFile { } } -#[async_trait] +#[cuberpc::service] pub trait RemoteFs: DIService + Send + Sync + Debug { /// Use this path to prepare files for upload. Writing into `local_path()` directly can result /// in files being deleted by the background cleanup process, see `QueueRemoteFs::cleanup_loop`. - async fn temp_upload_path(&self, remote_path: &str) -> Result { + async fn temp_upload_path(&self, remote_path: String) -> Result; + + /// Convention is to use this directory for creating files to be uploaded later. + async fn uploads_dir(&self) -> Result; + + /// Check existance and size of uploaded file. Raise error if file doesn't exists or has wrong + /// size + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError>; + + /// In addition to uploading this file to the remote filesystem, this function moves the file + /// from `temp_upload_path` to `self.local_path(remote_path)` on the local file system. + async fn upload_file( + &self, + temp_upload_path: String, + remote_path: String, + ) -> Result; + + async fn download_file( + &self, + remote_path: String, + expected_file_size: Option, + ) -> Result; + + async fn delete_file(&self, remote_path: String) -> Result<(), CubeError>; + + async fn list(&self, remote_prefix: String) -> Result, CubeError>; + + async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError>; + + async fn local_path(&self) -> Result; + + async fn local_file(&self, remote_path: String) -> Result; +} + +pub struct CommonRemoteFsUtils; + +impl CommonRemoteFsUtils { + /// + /// Use this path to prepare files for upload. Writing into `local_path()` directly can result + /// in files being deleted by the background cleanup process, see `QueueRemoteFs::cleanup_loop`. + pub async fn temp_upload_path(remote_fs: &dyn RemoteFs, remote_path: String) -> Result { // Putting files into a subdirectory prevents cleanups from removing them. - self.local_file(&format!("uploads/{}", remote_path)).await + remote_fs.local_file(format!("uploads/{}", remote_path)).await } /// Convention is to use this directory for creating files to be uploaded later. - async fn uploads_dir(&self) -> Result { + pub async fn uploads_dir(remote_fs: &dyn RemoteFs) -> Result { // Call to `temp_upload_path` ensures we created the uploads dir. - let file_in_dir = self - .temp_upload_path("never_created_remote_fs_file") + let file_in_dir = remote_fs + .temp_upload_path("never_created_remote_fs_file".to_string()) .await?; Ok(Path::new(&file_in_dir) .parent() @@ -62,12 +107,12 @@ pub trait RemoteFs: DIService + Send + Sync + Debug { } /// Check existance and size of uploaded file. Raise error if file doesn't exists or has wrong /// size - async fn check_upload_file( - &self, - remote_path: &str, + pub async fn check_upload_file( + remote_fs: &dyn RemoteFs, + remote_path: String, expected_size: u64, ) -> Result<(), CubeError> { - match self.list_with_metadata(&remote_path).await { + match remote_fs.list_with_metadata(remote_path.clone()).await { Ok(list) => { let list_res = list.iter().next().ok_or(CubeError::internal( format!("File {} can't be listed after upload. Either there's Cube Store cluster misconfiguration, or storage can't provide the required consistency.", remote_path), @@ -91,30 +136,6 @@ pub trait RemoteFs: DIService + Send + Sync + Debug { Err(e) => Err(e), } } - - /// In addition to uploading this file to the remote filesystem, this function moves the file - /// from `temp_upload_path` to `self.local_path(remote_path)` on the local file system. - async fn upload_file( - &self, - temp_upload_path: &str, - remote_path: &str, - ) -> Result; - - async fn download_file( - &self, - remote_path: &str, - expected_file_size: Option, - ) -> Result; - - async fn delete_file(&self, remote_path: &str) -> Result<(), CubeError>; - - async fn list(&self, remote_prefix: &str) -> Result, CubeError>; - - async fn list_with_metadata(&self, remote_prefix: &str) -> Result, CubeError>; - - async fn local_path(&self) -> String; - - async fn local_file(&self, remote_path: &str) -> Result; } pub fn ensure_temp_file_is_dropped(path: String) { @@ -158,19 +179,37 @@ impl LocalDirRemoteFs { } di_service!(LocalDirRemoteFs, [RemoteFs]); +di_service!(RemoteFsRpcClient, [RemoteFs]); #[async_trait] impl RemoteFs for LocalDirRemoteFs { + + async fn temp_upload_path(&self, remote_path: String) -> Result { + CommonRemoteFsUtils::temp_upload_path(self, remote_path).await + } + + async fn uploads_dir(&self) -> Result { + CommonRemoteFsUtils::uploads_dir(self).await + } + + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError> { + CommonRemoteFsUtils::check_upload_file(self, remote_path, expected_size).await + } + async fn upload_file( &self, - temp_upload_path: &str, - remote_path: &str, + temp_upload_path: String, + remote_path: String, ) -> Result { let mut has_remote = false; if let Some(remote_dir) = self.remote_dir.write().await.as_ref() { has_remote = true; debug!("Uploading {}", remote_path); - let dest = remote_dir.as_path().join(remote_path); + let dest = remote_dir.as_path().join(&remote_path); fs::create_dir_all(dest.parent().unwrap()) .await .map_err(|e| { @@ -193,12 +232,12 @@ impl RemoteFs for LocalDirRemoteFs { } if has_remote { let size = fs::metadata(&temp_upload_path).await?.len(); - self.check_upload_file(&remote_path, size).await?; + self.check_upload_file(remote_path.clone(), size).await?; } - let local_path = self.dir.as_path().join(remote_path); + let local_path = self.dir.as_path().join(&remote_path); - if Path::new(temp_upload_path) != local_path { + if Path::new(&temp_upload_path) != local_path { fs::create_dir_all(local_path.parent().unwrap()) .await .map_err(|e| { @@ -224,10 +263,10 @@ impl RemoteFs for LocalDirRemoteFs { async fn download_file( &self, - remote_path: &str, + remote_path: String, _expected_file_size: Option, ) -> Result { - let mut local_file = self.dir.as_path().join(remote_path); + let mut local_file = self.dir.as_path().join(&remote_path); let local_dir = local_file.parent().unwrap(); let downloads_dir = local_dir.join("downloads"); fs::create_dir_all(&downloads_dir).await?; @@ -238,7 +277,7 @@ impl RemoteFs for LocalDirRemoteFs { cube_ext::spawn_blocking(move || NamedTempFile::new_in(downloads_dir)) .await?? .into_temp_path(); - fs::copy(remote_dir.as_path().join(remote_path), &temp_path) + fs::copy(remote_dir.as_path().join(&remote_path), &temp_path) .await .map_err(|e| { CubeError::internal(format!( @@ -262,11 +301,11 @@ impl RemoteFs for LocalDirRemoteFs { Ok(local_file.into_os_string().into_string().unwrap()) } - async fn delete_file(&self, remote_path: &str) -> Result<(), CubeError> { + async fn delete_file(&self, remote_path: String) -> Result<(), CubeError> { debug!("Deleting {}", remote_path); { if let Some(remote_dir) = self.remote_dir.write().await.as_ref() { - let remote = remote_dir.as_path().join(remote_path); + let remote = remote_dir.as_path().join(&remote_path); if fs::metadata(remote.clone()).await.is_ok() { fs::remove_file(remote.clone()).await?; Self::remove_empty_paths(remote_dir.clone(), remote.clone()).await?; @@ -275,7 +314,7 @@ impl RemoteFs for LocalDirRemoteFs { } let _local_guard = acquire_lock("delete file", self.dir_delete_mut.lock()).await?; - let local = self.dir.as_path().join(remote_path); + let local = self.dir.as_path().join(&remote_path); if fs::metadata(local.clone()).await.is_ok() { fs::remove_file(local.clone()).await?; LocalDirRemoteFs::remove_empty_paths(self.dir.as_path().to_path_buf(), local.clone()) @@ -285,7 +324,7 @@ impl RemoteFs for LocalDirRemoteFs { Ok(()) } - async fn list(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list(&self, remote_prefix: String) -> Result, CubeError> { Ok(self .list_with_metadata(remote_prefix) .await? @@ -294,7 +333,7 @@ impl RemoteFs for LocalDirRemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { let remote_dir = self.remote_dir.read().await.as_ref().cloned(); let result = Self::list_recursive( remote_dir.clone().unwrap_or(self.dir.clone()), @@ -305,11 +344,11 @@ impl RemoteFs for LocalDirRemoteFs { Ok(result) } - async fn local_path(&self) -> String { - self.dir.to_str().unwrap().to_owned() + async fn local_path(&self) -> Result { + Ok(self.dir.to_str().unwrap().to_owned()) } - async fn local_file(&self, remote_path: &str) -> Result { + async fn local_file(&self, remote_path: String) -> Result { let buf = self.dir.join(remote_path); fs::create_dir_all(buf.parent().unwrap()).await?; Ok(buf.to_str().unwrap().to_string()) @@ -439,11 +478,11 @@ mod tests { remote_fs: &Arc, remote_file: &str, ) -> Result { - let temp_upload_path = remote_fs.temp_upload_path(remote_file).await?; + let temp_upload_path = remote_fs.temp_upload_path(remote_file.to_string()).await?; let mut file = fs::File::create(&temp_upload_path).unwrap(); file.write_all(b"test").unwrap(); remote_fs - .upload_file(&temp_upload_path, remote_file) + .upload_file(temp_upload_path.clone(), remote_file.to_string()) .await?; Ok(temp_upload_path) @@ -455,9 +494,9 @@ mod tests { name_maker: NameMaker, download_test: bool, ) { - assert_eq!(remote_fs.local_path().await, local_dir.to_str().unwrap()); + assert_eq!(remote_fs.local_path().await.unwrap(), local_dir.to_str().unwrap()); - let local_file = remote_fs.local_file("test.tst").await.ok().unwrap(); + let local_file = remote_fs.local_file("test.tst".to_string()).await.ok().unwrap(); assert_eq!(local_file, local_dir.join("test.tst").to_str().unwrap()); let local_file_path = Path::new("test_dir") @@ -466,7 +505,7 @@ mod tests { .unwrap() .to_owned(); let local_file = remote_fs - .local_file(local_file_path.as_ref()) + .local_file(local_file_path.to_string()) .await .unwrap(); @@ -490,7 +529,7 @@ mod tests { assert!(local_dir.join(filename).is_file()); } - let mut remote_list = remote_fs.list(&name_maker.name("test-")).await.unwrap(); + let mut remote_list = remote_fs.list(name_maker.name("test-")).await.unwrap(); remote_list.sort(); remote_list .iter() @@ -499,7 +538,7 @@ mod tests { assert_eq!(list_name, origin_name); }); - let mut remote_list = remote_fs.list(&name_maker.name("subdir/")).await.unwrap(); + let mut remote_list = remote_fs.list(name_maker.name("subdir/")).await.unwrap(); remote_list.sort(); remote_list .iter() @@ -509,7 +548,7 @@ mod tests { }); let mut remote_list = remote_fs - .list_with_metadata(&name_maker.name("test")) + .list_with_metadata(name_maker.name("test")) .await .unwrap(); @@ -530,19 +569,19 @@ mod tests { for filename in root_files.iter().chain(subdir_files.iter()) { assert!(!local_dir.join(filename).is_file()); - remote_fs.download_file(filename, None).await.unwrap(); + remote_fs.download_file(filename.clone(), None).await.unwrap(); assert!(local_dir.join(filename).is_file()); } } for filename in root_files.iter().chain(subdir_files.iter()) { - assert!(local_dir.join(filename).is_file()); - assert_eq!(&remote_fs.list(filename).await.unwrap()[0], filename); + assert!(local_dir.join(&filename).is_file()); + assert_eq!(&remote_fs.list(filename.clone()).await.unwrap()[0], filename); - remote_fs.delete_file(filename).await.unwrap(); + remote_fs.delete_file(filename.clone()).await.unwrap(); - assert!(!local_dir.join(filename).is_file()); - assert!(&remote_fs.list(filename).await.unwrap().is_empty()); + assert!(!local_dir.join(&filename).is_file()); + assert!(&remote_fs.list(filename.clone()).await.unwrap().is_empty()); } } diff --git a/rust/cubestore/cubestore/src/remotefs/queue.rs b/rust/cubestore/cubestore/src/remotefs/queue.rs index 066456417420a..ed667569edc36 100644 --- a/rust/cubestore/cubestore/src/remotefs/queue.rs +++ b/rust/cubestore/cubestore/src/remotefs/queue.rs @@ -1,6 +1,6 @@ use crate::config::ConfigObj; use crate::di_service; -use crate::remotefs::{RemoteFile, RemoteFs}; +use crate::remotefs::{RemoteFile, RemoteFs, CommonRemoteFsUtils}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -32,9 +32,10 @@ pub struct QueueRemoteFs { impl Debug for QueueRemoteFs { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { - f.debug_struct("QueueRemoteFs") - .field("remote_fs", &self.remote_fs) - .finish() + f.debug_struct("QueueRemoteFs").finish() + //TODO FIX IT + /* .field("remote_fs", &self.remote_fs) + .finish() */ } } @@ -148,7 +149,7 @@ impl QueueRemoteFs { { let res = self .remote_fs - .upload_file(&temp_upload_path, &remote_path) + .upload_file(temp_upload_path, remote_path.clone()) .await; self.result_sender .send(RemoteFsOpResult::Upload(remote_path, res))?; @@ -157,7 +158,7 @@ impl QueueRemoteFs { RemoteFsOp::Delete(file) => { self.result_sender.send(RemoteFsOpResult::Delete( file.to_string(), - self.remote_fs.delete_file(file.as_str()).await, + self.remote_fs.delete_file(file.to_string()).await, ))?; } x => panic!("Unexpected operation: {:?}", x), @@ -170,7 +171,7 @@ impl QueueRemoteFs { RemoteFsOp::Download(file, expected_file_size) => { let result = self .remote_fs - .download_file(file.as_str(), expected_file_size) + .download_file(file.clone(), expected_file_size) .await; let mut downloading = acquire_lock("download loop downloading", self.downloading.write()).await?; @@ -186,10 +187,27 @@ impl QueueRemoteFs { #[async_trait] impl RemoteFs for QueueRemoteFs { + + async fn temp_upload_path(&self, remote_path: String) -> Result { + CommonRemoteFsUtils::temp_upload_path(self, remote_path).await + } + + async fn uploads_dir(&self) -> Result { + CommonRemoteFsUtils::uploads_dir(self).await + } + + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError> { + CommonRemoteFsUtils::check_upload_file(self, remote_path, expected_size).await + } + async fn upload_file( &self, - local_upload_path: &str, - remote_path: &str, + local_upload_path: String, + remote_path: String, ) -> Result { if !self.config.upload_to_remote() { log::info!("Skipping upload {}", remote_path); @@ -203,7 +221,7 @@ impl RemoteFs for QueueRemoteFs { loop { let res = receiver.recv().await?; if let RemoteFsOpResult::Upload(file, result) = res { - if &file == remote_path { + if file == remote_path { return result; } } @@ -212,15 +230,15 @@ impl RemoteFs for QueueRemoteFs { async fn download_file( &self, - remote_path: &str, + remote_path: String, expected_file_size: Option, ) -> Result { // We might be lucky and the file has already been downloaded. - if let Ok(local_path) = self.local_file(remote_path).await { + if let Ok(local_path) = self.local_file(remote_path.clone()).await { let metadata = tokio::fs::metadata(&local_path).await; if metadata.is_ok() { if let Err(e) = QueueRemoteFs::check_file_size( - remote_path, + &remote_path, expected_file_size, &local_path, metadata.unwrap(), @@ -236,7 +254,7 @@ impl RemoteFs for QueueRemoteFs { { let mut downloading = acquire_lock("download file downloading", self.downloading.write()).await?; - if !downloading.contains(remote_path) { + if !downloading.contains(&remote_path) { self.download_queue.push(RemoteFsOp::Download( remote_path.to_string(), expected_file_size, @@ -247,10 +265,10 @@ impl RemoteFs for QueueRemoteFs { loop { let res = receiver.recv().await?; if let RemoteFsOpResult::Download(file, result) = res { - if &file == remote_path { + if file == remote_path { match result { Ok(f) => { - let local_path = self.local_file(remote_path).await?; + let local_path = self.local_file(remote_path.clone()).await?; let metadata = tokio::fs::metadata(&local_path).await.map_err(|e| { CubeError::internal(format!( "Error while listing local file for consistency check {}: {}", @@ -258,7 +276,7 @@ impl RemoteFs for QueueRemoteFs { )) })?; if let Err(e) = QueueRemoteFs::check_file_size( - remote_path, + &remote_path, expected_file_size, &local_path, metadata, @@ -271,7 +289,7 @@ impl RemoteFs for QueueRemoteFs { } Err(err) => { //Check if file doesn't exists in remoteFs - if self.remote_fs.list(&file).await?.is_empty() { + if self.remote_fs.list(file.clone()).await?.is_empty() { return Err(CubeError::corrupt_data(format!( "File {} doesn't exist in remote file system", file @@ -285,7 +303,7 @@ impl RemoteFs for QueueRemoteFs { } } - async fn delete_file(&self, remote_path: &str) -> Result<(), CubeError> { + async fn delete_file(&self, remote_path: String) -> Result<(), CubeError> { if !self.config.upload_to_remote() { log::info!("Skipping delete {}", remote_path); return Ok(()); @@ -296,26 +314,26 @@ impl RemoteFs for QueueRemoteFs { loop { let res = receiver.recv().await?; if let RemoteFsOpResult::Delete(file, result) = res { - if &file == remote_path { + if file == remote_path { return result; } } } } - async fn list(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list(&self, remote_prefix: String) -> Result, CubeError> { self.remote_fs.list(remote_prefix).await } - async fn list_with_metadata(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { self.remote_fs.list_with_metadata(remote_prefix).await } - async fn local_path(&self) -> String { + async fn local_path(&self) -> Result { self.remote_fs.local_path().await } - async fn local_file(&self, remote_path: &str) -> Result { + async fn local_file(&self, remote_path: String) -> Result { self.remote_fs.local_file(remote_path).await } } @@ -370,14 +388,31 @@ mod test { #[async_trait] impl RemoteFs for MockFs { + + async fn temp_upload_path(&self, remote_path: String) -> Result { + CommonRemoteFsUtils::temp_upload_path(self, remote_path).await + } + + async fn uploads_dir(&self) -> Result { + CommonRemoteFsUtils::uploads_dir(self).await + } + + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError> { + CommonRemoteFsUtils::check_upload_file(self, remote_path, expected_size).await + } + async fn upload_file( &self, - local_upload_path: &str, - remote_path: &str, + local_upload_path: String, + remote_path: String, ) -> Result { let res = self .base_fs - .upload_file(local_upload_path, remote_path) + .upload_file(local_upload_path, remote_path.clone()) .await; if let Ok(size) = res { self.check_upload_file(remote_path, size).await? @@ -387,7 +422,7 @@ mod test { async fn download_file( &self, - remote_path: &str, + remote_path: String, expected_file_size: Option, ) -> Result { let res = self @@ -400,17 +435,17 @@ mod test { res } - async fn delete_file(&self, _remote_path: &str) -> Result<(), CubeError> { + async fn delete_file(&self, _remote_path: String) -> Result<(), CubeError> { Ok(()) } - async fn list(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list(&self, remote_prefix: String) -> Result, CubeError> { self.base_fs.list(remote_prefix).await } async fn list_with_metadata( &self, - remote_prefix: &str, + remote_prefix: String, ) -> Result, CubeError> { let mut res = self .base_fs @@ -429,11 +464,11 @@ mod test { Ok(res) } - async fn local_path(&self) -> String { + async fn local_path(&self) -> Result { self.base_fs.local_path().await } - async fn local_file(&self, remote_path: &str) -> Result { + async fn local_file(&self, remote_path: String) -> Result { self.base_fs.local_file(remote_path).await } } @@ -478,7 +513,7 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); let res = queue_fs - .upload_file(path.to_str().unwrap(), "temp-upload/foo.csv") + .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) .await; queue_fs.stop_processing_loops().unwrap(); r.await.unwrap().unwrap(); @@ -501,7 +536,7 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); let res = queue_fs - .upload_file(path.to_str().unwrap(), "temp-upload/foo.csv") + .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) .await; queue_fs.stop_processing_loops().unwrap(); r.await.unwrap().unwrap(); @@ -524,7 +559,7 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); let res = queue_fs - .upload_file(path.to_str().unwrap(), "temp-upload/foo.csv") + .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) .await; queue_fs.stop_processing_loops().unwrap(); r.await.unwrap().unwrap(); @@ -542,7 +577,7 @@ mod test { config.injector().get_service("original_remote_fs").await, ); let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); - let res = queue_fs.download_file("temp-upload/foo.csv", None).await; + let res = queue_fs.download_file("temp-upload/foo.csv".to_string(), None).await; match res { Ok(_) => assert!(false), Err(e) => assert!(e.is_corrupt_data()), @@ -564,13 +599,13 @@ mod test { ); let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); queue_fs - .upload_file(path.to_str().unwrap(), "temp-upload/foo.csv") + .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) .await .unwrap(); - std::fs::remove_file(queue_fs.local_file("temp-upload/foo.csv").await.unwrap()).unwrap(); + std::fs::remove_file(queue_fs.local_file("temp-upload/foo.csv".to_string()).await.unwrap()).unwrap(); - let res = queue_fs.download_file("temp-upload/foo.csv", Some(1)).await; + let res = queue_fs.download_file("temp-upload/foo.csv".to_string(), Some(1)).await; match res { Ok(_) => assert!(false), @@ -597,12 +632,12 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); queue_fs - .upload_file(path.to_str().unwrap(), "temp-upload/foo.csv") + .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) .await .unwrap(); - std::fs::remove_file(queue_fs.local_file("temp-upload/foo.csv").await.unwrap()).unwrap(); + std::fs::remove_file(queue_fs.local_file("temp-upload/foo.csv".to_string()).await.unwrap()).unwrap(); - let res = queue_fs.download_file("temp-upload/foo.csv", None).await; + let res = queue_fs.download_file("temp-upload/foo.csv".to_string(), None).await; match res { Ok(_) => assert!(false), diff --git a/rust/cubestore/cubestore/src/remotefs/s3.rs b/rust/cubestore/cubestore/src/remotefs/s3.rs index 43cfff7f0df2e..142116ec93bd1 100644 --- a/rust/cubestore/cubestore/src/remotefs/s3.rs +++ b/rust/cubestore/cubestore/src/remotefs/s3.rs @@ -1,6 +1,6 @@ use crate::app_metrics; use crate::di_service; -use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs}; +use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs, CommonRemoteFsUtils}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -133,10 +133,27 @@ di_service!(S3RemoteFs, [RemoteFs]); #[async_trait] impl RemoteFs for S3RemoteFs { + + async fn temp_upload_path(&self, remote_path: String) -> Result { + CommonRemoteFsUtils::temp_upload_path(self, remote_path).await + } + + async fn uploads_dir(&self) -> Result { + CommonRemoteFsUtils::uploads_dir(self).await + } + + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError> { + CommonRemoteFsUtils::check_upload_file(self, remote_path, expected_size).await + } + async fn upload_file( &self, - temp_upload_path: &str, - remote_path: &str, + temp_upload_path: String, + remote_path: String, ) -> Result { { app_metrics::REMOTE_FS_OPERATION_CORE.add_with_tags( @@ -149,7 +166,7 @@ impl RemoteFs for S3RemoteFs { let time = SystemTime::now(); debug!("Uploading {}", remote_path); - let path = self.s3_path(remote_path); + let path = self.s3_path(&remote_path); let bucket = self.bucket.read().unwrap().clone(); let temp_upload_path_copy = temp_upload_path.to_string(); let status_code = cube_ext::spawn_blocking(move || { @@ -165,11 +182,11 @@ impl RemoteFs for S3RemoteFs { ))); } } - let size = fs::metadata(temp_upload_path).await?.len(); - self.check_upload_file(remote_path, size).await?; + let size = fs::metadata(&temp_upload_path).await?.len(); + self.check_upload_file(remote_path.clone(), size).await?; - let local_path = self.dir.as_path().join(remote_path); - if Path::new(temp_upload_path) != local_path { + let local_path = self.dir.as_path().join(&remote_path); + if Path::new(&temp_upload_path) != local_path { fs::create_dir_all(local_path.parent().unwrap()) .await .map_err(|e| { @@ -186,10 +203,10 @@ impl RemoteFs for S3RemoteFs { async fn download_file( &self, - remote_path: &str, + remote_path: String, _expected_file_size: Option, ) -> Result { - let local_file = self.dir.as_path().join(remote_path); + let local_file = self.dir.as_path().join(&remote_path); let local_dir = local_file.parent().unwrap(); let downloads_dir = local_dir.join("downloads"); @@ -206,7 +223,7 @@ impl RemoteFs for S3RemoteFs { ); let time = SystemTime::now(); debug!("Downloading {}", remote_path); - let path = self.s3_path(remote_path); + let path = self.s3_path(&remote_path); let bucket = self.bucket.read().unwrap().clone(); let status_code = cube_ext::spawn_blocking(move || -> Result { let (mut temp_file, temp_path) = @@ -231,7 +248,7 @@ impl RemoteFs for S3RemoteFs { Ok(local_file_str) } - async fn delete_file(&self, remote_path: &str) -> Result<(), CubeError> { + async fn delete_file(&self, remote_path: String) -> Result<(), CubeError> { app_metrics::REMOTE_FS_OPERATION_CORE.add_with_tags( 1, Some(&vec![ @@ -241,7 +258,7 @@ impl RemoteFs for S3RemoteFs { ); let time = SystemTime::now(); debug!("Deleting {}", remote_path); - let path = self.s3_path(remote_path); + let path = self.s3_path(&remote_path); let bucket = self.bucket.read().unwrap().clone(); let (_, status_code) = cube_ext::spawn_blocking(move || bucket.delete_object_blocking(path)).await??; @@ -264,7 +281,7 @@ impl RemoteFs for S3RemoteFs { Ok(()) } - async fn list(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list(&self, remote_prefix: String) -> Result, CubeError> { Ok(self .list_with_metadata(remote_prefix) .await? @@ -273,8 +290,8 @@ impl RemoteFs for S3RemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: &str) -> Result, CubeError> { - let path = self.s3_path(remote_prefix); + async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + let path = self.s3_path(&remote_prefix); let bucket = self.bucket.read().unwrap().clone(); let list = cube_ext::spawn_blocking(move || bucket.list_blocking(path, None)).await??; let pages_count = list.len(); @@ -304,11 +321,11 @@ impl RemoteFs for S3RemoteFs { Ok(result) } - async fn local_path(&self) -> String { - self.dir.to_str().unwrap().to_owned() + async fn local_path(&self) -> Result { + Ok(self.dir.to_str().unwrap().to_owned()) } - async fn local_file(&self, remote_path: &str) -> Result { + async fn local_file(&self, remote_path: String) -> Result { let buf = self.dir.join(remote_path); fs::create_dir_all(buf.parent().unwrap()).await?; Ok(buf.to_str().unwrap().to_string()) diff --git a/rust/cubestore/cubestore/src/scheduler/mod.rs b/rust/cubestore/cubestore/src/scheduler/mod.rs index 8817909c46fbf..f6a2306557e2e 100644 --- a/rust/cubestore/cubestore/src/scheduler/mod.rs +++ b/rust/cubestore/cubestore/src/scheduler/mod.rs @@ -156,7 +156,7 @@ impl SchedulerImpl { match task { GCTask::RemoveRemoteFile(remote_path) => { log::trace!("Removing deactivated data file: {}", remote_path); - if let Err(e) = self.remote_fs.delete_file(&remote_path).await { + if let Err(e) = self.remote_fs.delete_file(remote_path.clone()).await { log::error!( "Could not remove deactivated data file({}): {}", remote_path, @@ -747,7 +747,7 @@ impl SchedulerImpl { if let MetaStoreEvent::Delete(TableId::WALs, row_id) = event { let file = self .remote_fs - .local_file(WALStore::wal_remote_path(row_id).as_str()) + .local_file(WALStore::wal_remote_path(row_id)) .await?; tokio::fs::remove_file(file).await?; } diff --git a/rust/cubestore/cubestore/src/sql/mod.rs b/rust/cubestore/cubestore/src/sql/mod.rs index 32ba7dd753d9c..b2711d05f43ff 100644 --- a/rust/cubestore/cubestore/src/sql/mod.rs +++ b/rust/cubestore/cubestore/src/sql/mod.rs @@ -623,7 +623,7 @@ impl SqlServiceImpl { ) .await?; - let mut dump_dir = PathBuf::from(&self.remote_fs.local_path().await); + let mut dump_dir = PathBuf::from(&self.remote_fs.local_path().await?); dump_dir.push("dumps"); tokio::fs::create_dir_all(&dump_dir).await?; @@ -640,7 +640,7 @@ impl SqlServiceImpl { log::debug!("Dumping data files to {:?}", data_dir); // TODO: download in parallel. for (_, f, size, _) in p.all_required_files() { - let f = self.remote_fs.download_file(&f, size).await?; + let f = self.remote_fs.download_file(f, size).await?; let name = Path::new(&f).file_name().ok_or_else(|| { CubeError::internal(format!("Could not get filename of '{}'", f)) })?; @@ -1397,7 +1397,7 @@ impl SqlService for SqlServiceImpl { ); let mut mocked_names = HashMap::new(); for (_, f, _, _) in worker_plan.files_to_download() { - let name = self.remote_fs.local_file(&f).await?; + let name = self.remote_fs.local_file(f.clone()).await?; mocked_names.insert(f, name); } let chunk_ids_to_batches = worker_plan @@ -1442,8 +1442,8 @@ impl SqlService for SqlServiceImpl { // TODO persist file size self.remote_fs .upload_file( - file_path.to_string_lossy().as_ref(), - &format!("temp-uploads/{}", name), + file_path.to_string_lossy().to_string(), + format!("temp-uploads/{}", name), ) .await?; Ok(()) @@ -2308,49 +2308,66 @@ mod tests { let _ = fs::remove_dir_all(remote_store_path.clone()); } - #[derive(Debug)] + //#[derive(Debug)] pub struct FailingRemoteFs(Arc); crate::di_service!(FailingRemoteFs, [RemoteFs]); + use crate::remotefs::CommonRemoteFsUtils; #[async_trait::async_trait] impl RemoteFs for FailingRemoteFs { + + async fn temp_upload_path(&self, remote_path: String) -> Result { + CommonRemoteFsUtils::temp_upload_path(self, remote_path).await + } + + async fn uploads_dir(&self) -> Result { + CommonRemoteFsUtils::uploads_dir(self).await + } + + async fn check_upload_file( + &self, + remote_path: String, + expected_size: u64, + ) -> Result<(), CubeError> { + CommonRemoteFsUtils::check_upload_file(self, remote_path, expected_size).await + } async fn upload_file( &self, - _temp_upload_path: &str, - _remote_path: &str, + _temp_upload_path: String, + _remote_path: String, ) -> Result { Err(CubeError::internal("Not allowed".to_string())) } async fn download_file( &self, - remote_path: &str, + remote_path: String, expected_file_size: Option, ) -> Result { self.0.download_file(remote_path, expected_file_size).await } - async fn delete_file(&self, remote_path: &str) -> Result<(), CubeError> { + async fn delete_file(&self, remote_path: String) -> Result<(), CubeError> { self.0.delete_file(remote_path).await } - async fn list(&self, remote_prefix: &str) -> Result, CubeError> { + async fn list(&self, remote_prefix: String) -> Result, CubeError> { self.0.list(remote_prefix).await } async fn list_with_metadata( &self, - remote_prefix: &str, + remote_prefix: String, ) -> Result, CubeError> { self.0.list_with_metadata(remote_prefix).await } - async fn local_path(&self) -> String { + async fn local_path(&self) -> Result { self.0.local_path().await } - async fn local_file(&self, remote_path: &str) -> Result { + async fn local_file(&self, remote_path: String) -> Result { self.0.local_file(remote_path).await } } @@ -2382,7 +2399,7 @@ mod tests { let remote_fs = services.injector.get_service_typed::().await; - let temp_upload = remote_fs.temp_upload_path("").await.unwrap(); + let temp_upload = remote_fs.temp_upload_path("".to_string()).await.unwrap(); let res = fs::read_dir(temp_upload.clone()).unwrap(); assert!(res.into_iter().next().is_none(), "Expected empty uploads directory but found: {:?}", fs::read_dir(temp_upload).unwrap().into_iter().map(|e| e.unwrap().path().to_string_lossy().to_string()).collect::>()); }) @@ -2969,8 +2986,8 @@ mod tests { let remote_fs = services.injector.get_service_typed::().await; remote_fs .upload_file( - path.to_str().unwrap(), - &chunk.get_row().get_full_name(chunk.get_id()), + path.to_str().unwrap().to_string(), + chunk.get_row().get_full_name(chunk.get_id()), ) .await .unwrap(); @@ -3479,7 +3496,7 @@ mod tests { let remote_fs = services.injector.get_service_typed::().await; let files = remote_fs - .list("") + .list("".to_string()) .await .unwrap() .into_iter() @@ -4074,7 +4091,7 @@ mod tests { file.shutdown().await.unwrap(); let remote_fs = services.injector.get_service_typed::().await; - remote_fs.upload_file(path_2.to_str().unwrap(), "temp-uploads/foo-3.csv.gz").await.unwrap(); + remote_fs.upload_file(path_2.to_str().unwrap().to_string(), "temp-uploads/foo-3.csv.gz".to_string()).await.unwrap(); vec!["temp://foo-3.csv.gz".to_string()] }; @@ -4274,7 +4291,7 @@ mod tests { .injector .get_service_typed::() .await - .upload_file(path_2.to_str().unwrap(), "temp-uploads/orders.csv.gz") + .upload_file(path_2.to_str().unwrap().to_string(), "temp-uploads/orders.csv.gz".to_string()) .await .unwrap(); diff --git a/rust/cubestore/cubestore/src/store/compaction.rs b/rust/cubestore/cubestore/src/store/compaction.rs index 791ae891a09bb..469cab2726f0b 100644 --- a/rust/cubestore/cubestore/src/store/compaction.rs +++ b/rust/cubestore/cubestore/src/store/compaction.rs @@ -580,7 +580,7 @@ impl CompactionService for CompactionServiceImpl { let old_partition_local = if let Some(f) = old_partition_remote { let result = self .remote_fs - .download_file(&f, partition.get_row().file_size()) + .download_file(f, partition.get_row().file_size()) .await; deactivate_table_on_corrupt_data(self.meta_store.clone(), &result, &partition, None) .await; @@ -591,11 +591,11 @@ impl CompactionService for CompactionServiceImpl { let mut new_local_files = Vec::new(); if let Some(c) = &new_chunk { let remote = ChunkStore::chunk_remote_path(c.get_id(), c.get_row().suffix()); - new_local_files.push(self.remote_fs.temp_upload_path(&remote).await?); + new_local_files.push(self.remote_fs.temp_upload_path(remote).await?); } else { for p in new_partitions.iter() { let new_remote_path = partition_file_name(p.get_id(), p.get_row().suffix()); - new_local_files.push(self.remote_fs.temp_upload_path(&new_remote_path).await?); + new_local_files.push(self.remote_fs.temp_upload_path(new_remote_path).await?); } } @@ -677,7 +677,7 @@ impl CompactionService for CompactionServiceImpl { let remote = ChunkStore::chunk_remote_path(c.get_id(), c.get_row().suffix()); let file_size = self .remote_fs - .upload_file(&new_local_files[0], &remote) + .upload_file(new_local_files[0].clone(), remote.clone()) .await?; let chunk_ids = chunks.iter().map(|c| c.get_id()).collect_vec(); // In memory chunks shouldn't ever get here. Otherwise replay handle should be defined. @@ -690,7 +690,7 @@ impl CompactionService for CompactionServiceImpl { "Cancelled compaction of {}. It runs concurrently with multi-split", partition_id ); - self.remote_fs.delete_file(&remote).await?; + self.remote_fs.delete_file(remote).await?; } return Ok(()); } @@ -706,7 +706,7 @@ impl CompactionService for CompactionServiceImpl { let new_remote_path = partition_file_name(p.get_id(), p.get_row().suffix()); let file_size = self .remote_fs - .upload_file(&new_local_files[i], new_remote_path.as_str()) + .upload_file(new_local_files[i].clone(), new_remote_path.to_string()) .await?; filtered_partitions.push((p, file_size)); } @@ -1066,7 +1066,7 @@ async fn download_files( let (f, size) = take(f); let fs = fs.clone(); tasks.push(cube_ext::spawn( - async move { fs.download_file(&f, size).await }, + async move { fs.download_file(f, size).await }, )) } remote_files.clear(); @@ -1937,7 +1937,7 @@ mod tests { .get_full_name(partition.get_id()) .unwrap(); let local = remote_fs - .download_file(&remote, partition.get_row().file_size()) + .download_file(remote.clone(), partition.get_row().file_size()) .await .unwrap(); let reader = Arc::new( @@ -2089,14 +2089,14 @@ impl MultiSplit { let mut in_files = Vec::new(); collect_remote_files(&p, &mut in_files); for (f, _) in &mut in_files { - *f = self.fs.local_file(f).await?; + *f = self.fs.local_file(f.clone()).await?; } let mut out_files = Vec::with_capacity(children.len()); let mut out_remote_paths = Vec::with_capacity(children.len()); for c in &children { let remote_path = partition_file_name(c.get_id(), c.get_row().suffix()); - out_files.push(self.fs.temp_upload_path(&remote_path).await?); + out_files.push(self.fs.temp_upload_path(remote_path.clone()).await?); out_remote_paths.push(remote_path); } @@ -2139,7 +2139,7 @@ impl MultiSplit { let local_path = out_files[i].to_string(); let remote_path = out_files[i].to_string(); uploads.push(cube_ext::spawn(async move { - fs.upload_file(&local_path, &remote_path).await + fs.upload_file(local_path, remote_path).await })); } Ok(()) diff --git a/rust/cubestore/cubestore/src/store/mod.rs b/rust/cubestore/cubestore/src/store/mod.rs index 86fbc54a68ff3..9087ac99573bf 100644 --- a/rust/cubestore/cubestore/src/store/mod.rs +++ b/rust/cubestore/cubestore/src/store/mod.rs @@ -292,7 +292,7 @@ impl WALDataStore for WALStore { .create_wal(table.get_id(), data.len()) .await?; let remote_path = WALStore::wal_remote_path(wal.get_id()).clone(); - let local_file = self.remote_fs.local_file(&remote_path).await?; + let local_file = self.remote_fs.local_file(remote_path).await?; cube_ext::spawn_blocking(move || -> Result<(), CubeError> { save(local_file, data)?; Ok(()) @@ -315,8 +315,8 @@ impl WALDataStore for WALStore { ))); } let remote_path = WALStore::wal_remote_path(wal_id); - self.remote_fs.download_file(&remote_path, None).await?; - let local_file = self.remote_fs.local_file(&remote_path).await?; + self.remote_fs.download_file(remote_path.clone(), None).await?; + let local_file = self.remote_fs.local_file(remote_path.clone()).await?; Ok( cube_ext::spawn_blocking(move || -> Result { Ok(load::(local_file)?) @@ -735,7 +735,7 @@ impl ChunkStore { let file_size = chunk.get_row().file_size(); let chunk_id = chunk.get_id(); let remote_path = ChunkStore::chunk_file_name(chunk); - let result = self.remote_fs.download_file(&remote_path, file_size).await; + let result = self.remote_fs.download_file(remote_path.clone(), file_size).await; deactivate_table_on_corrupt_data( self.meta_store.clone(), @@ -746,7 +746,7 @@ impl ChunkStore { .await; Ok(( - self.remote_fs.local_file(&remote_path).await?, + self.remote_fs.local_file(remote_path.clone()).await?, index.into_row(), )) } @@ -1333,7 +1333,7 @@ impl ChunkStore { } else { trace!("New chunk allocated during partitioning: {:?}", chunk); let remote_path = ChunkStore::chunk_file_name(chunk.clone()).clone(); - let local_file = self.remote_fs.temp_upload_path(&remote_path).await?; + let local_file = self.remote_fs.temp_upload_path(remote_path.clone()).await?; let local_file = scopeguard::guard(local_file, ensure_temp_file_is_dropped); let local_file_copy = local_file.clone(); cube_ext::spawn_blocking(move || -> Result<(), CubeError> { @@ -1345,7 +1345,7 @@ impl ChunkStore { let fs = self.remote_fs.clone(); Ok(cube_ext::spawn(async move { - let file_size = fs.upload_file(&local_file, &remote_path).await?; + let file_size = fs.upload_file(local_file.to_string(), remote_path.clone()).await?; Ok((chunk, Some(file_size))) })) } From bf7e058457160ff09a2461b2e53ffa32e74a3613 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Mon, 18 Sep 2023 21:29:35 +0300 Subject: [PATCH 08/15] update --- rust/cubestore/cubestore/src/cluster/mod.rs | 6 +- .../cubestore/src/cluster/worker_pool.rs | 6 ++ .../cubestore/src/cluster/worker_services.rs | 5 ++ .../cubestore/src/metastore/rocks_fs.rs | 5 +- rust/cubestore/cubestore/src/remotefs/gcs.rs | 8 ++- .../cubestore/cubestore/src/remotefs/minio.rs | 8 ++- rust/cubestore/cubestore/src/remotefs/mod.rs | 43 +++++++++--- .../cubestore/cubestore/src/remotefs/queue.rs | 68 ++++++++++++++----- rust/cubestore/cubestore/src/remotefs/s3.rs | 8 ++- rust/cubestore/cubestore/src/sql/mod.rs | 6 +- rust/cubestore/cubestore/src/store/mod.rs | 13 +++- 11 files changed, 131 insertions(+), 45 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index 64abaf2df21bb..167487944ade9 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -15,7 +15,8 @@ pub mod ingestion; use crate::cluster::worker_pool::{worker_main, WorkerPool}; #[cfg(not(target_os = "windows"))] use crate::cluster::worker_services::{ - DefaultServicesServerProcessor, DefaultWorkerServicesDef, WorkerProcessing, + DefaultServicesServerProcessor, DefaultWorkerServicesDef, ServicesServerProcessor, + WorkerProcessing, }; use crate::ack_error; @@ -864,9 +865,10 @@ impl ClusterImpl { || self.config_obj.worker_bind_address().is_some()) && self.config_obj.select_worker_pool_size() > 0 { + let injector = self.injector.upgrade().unwrap(); let mut pool = self.select_process_pool.write().await; let arc = Arc::new(WorkerPool::new( - DefaultServicesServerProcessor::new(), + ServicesServerProcessor::init(injector).await, self.config_obj.select_worker_pool_size(), Duration::from_secs(self.config_obj.query_timeout()), "sel", diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 15af62b23f3e8..b25568a49cf6f 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -448,6 +448,7 @@ mod tests { use tokio::runtime::Builder; use crate::cluster::worker_pool::{worker_main, WorkerPool}; + use crate::config::injection::Injector; use crate::config::Config; use crate::queryplanner::serialized_plan::SerializedLogicalPlan; use crate::util::respawn; @@ -640,6 +641,11 @@ mod tests { impl ServicesServerProcessor for TestServicesServerProcessor { type Request = i64; type Response = bool; + + async fn init(_injector: Arc) -> Arc { + Arc::new(Self {}) + } + async fn process(&self, request: i64) -> bool { request % 2 == 0 } diff --git a/rust/cubestore/cubestore/src/cluster/worker_services.rs b/rust/cubestore/cubestore/src/cluster/worker_services.rs index 5a6b40c5e1f75..a6e959ef3e127 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_services.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_services.rs @@ -1,3 +1,4 @@ +use crate::config::injection::Injector; use crate::util::cancellation_token_guard::CancellationGuard; use crate::CubeError; use async_trait::async_trait; @@ -39,6 +40,7 @@ pub trait WorkerProcessing { pub trait ServicesServerProcessor { type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + async fn init(injector: Arc) -> Arc; async fn process(&self, request: Self::Request) -> Self::Response; } @@ -70,6 +72,9 @@ impl DefaultServicesServerProcessor { impl ServicesServerProcessor for DefaultServicesServerProcessor { type Request = (); type Response = (); + async fn init(_injector: Arc) -> Arc { + Arc::new(Self {}) + } async fn process(&self, _request: ()) -> () { () } diff --git a/rust/cubestore/cubestore/src/metastore/rocks_fs.rs b/rust/cubestore/cubestore/src/metastore/rocks_fs.rs index 0dd8d34cbe962..90d0a7a97a757 100644 --- a/rust/cubestore/cubestore/src/metastore/rocks_fs.rs +++ b/rust/cubestore/cubestore/src/metastore/rocks_fs.rs @@ -121,7 +121,10 @@ impl BaseRocksStoreFs { return async move { let local = remote_fs.local_file(f.clone()).await?; // TODO persist file size - Ok::<_, CubeError>((f.clone(), remote_fs.upload_file(local, f.clone()).await?)) + Ok::<_, CubeError>(( + f.clone(), + remote_fs.upload_file(local, f.clone()).await?, + )) }; }) .collect::>(), diff --git a/rust/cubestore/cubestore/src/remotefs/gcs.rs b/rust/cubestore/cubestore/src/remotefs/gcs.rs index c631e061de97c..aea7ac9a7285b 100644 --- a/rust/cubestore/cubestore/src/remotefs/gcs.rs +++ b/rust/cubestore/cubestore/src/remotefs/gcs.rs @@ -1,6 +1,6 @@ use crate::app_metrics; use crate::di_service; -use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs, CommonRemoteFsUtils}; +use crate::remotefs::{CommonRemoteFsUtils, LocalDirRemoteFs, RemoteFile, RemoteFs}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -119,7 +119,6 @@ di_service!(GCSRemoteFs, [RemoteFs]); #[async_trait] impl RemoteFs for GCSRemoteFs { - async fn temp_upload_path(&self, remote_path: String) -> Result { CommonRemoteFsUtils::temp_upload_path(self, remote_path).await } @@ -270,7 +269,10 @@ impl RemoteFs for GCSRemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + async fn list_with_metadata( + &self, + remote_prefix: String, + ) -> Result, CubeError> { let prefix = self.gcs_path(&remote_prefix); let list = Object::list_prefix(self.bucket.as_str(), prefix.as_str()).await?; let leading_slash = Regex::new(format!("^{}", self.gcs_path("")).as_str()).unwrap(); diff --git a/rust/cubestore/cubestore/src/remotefs/minio.rs b/rust/cubestore/cubestore/src/remotefs/minio.rs index 5f008a322589a..670295c223288 100644 --- a/rust/cubestore/cubestore/src/remotefs/minio.rs +++ b/rust/cubestore/cubestore/src/remotefs/minio.rs @@ -1,5 +1,5 @@ use crate::di_service; -use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs, CommonRemoteFsUtils}; +use crate::remotefs::{CommonRemoteFsUtils, LocalDirRemoteFs, RemoteFile, RemoteFs}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -144,7 +144,6 @@ di_service!(MINIORemoteFs, [RemoteFs]); #[async_trait] impl RemoteFs for MINIORemoteFs { - async fn temp_upload_path(&self, remote_path: String) -> Result { CommonRemoteFsUtils::temp_upload_path(self, remote_path).await } @@ -284,7 +283,10 @@ impl RemoteFs for MINIORemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + async fn list_with_metadata( + &self, + remote_prefix: String, + ) -> Result, CubeError> { let path = self.s3_path(&remote_prefix); let bucket = self.bucket.read().unwrap().clone(); let list = cube_ext::spawn_blocking(move || bucket.list_blocking(path, None)).await??; diff --git a/rust/cubestore/cubestore/src/remotefs/mod.rs b/rust/cubestore/cubestore/src/remotefs/mod.rs index 7eb2c1064dd0e..4d55ba665dfca 100644 --- a/rust/cubestore/cubestore/src/remotefs/mod.rs +++ b/rust/cubestore/cubestore/src/remotefs/mod.rs @@ -14,13 +14,13 @@ use datafusion::cube_ext; use futures::future::BoxFuture; use futures::FutureExt; use log::debug; +use serde::{Deserialize, Serialize}; use std::fmt::Debug; use std::path::{Path, PathBuf}; use std::sync::Arc; use tempfile::{NamedTempFile, PathPersistError}; use tokio::fs; use tokio::sync::{Mutex, RwLock}; -use serde::{Deserialize, Serialize}; #[derive(Debug, Clone, Serialize, Deserialize)] pub struct RemoteFile { @@ -54,7 +54,7 @@ pub trait RemoteFs: DIService + Send + Sync + Debug { &self, remote_path: String, expected_size: u64, - ) -> Result<(), CubeError>; + ) -> Result<(), CubeError>; /// In addition to uploading this file to the remote filesystem, this function moves the file /// from `temp_upload_path` to `self.local_path(remote_path)` on the local file system. @@ -74,7 +74,8 @@ pub trait RemoteFs: DIService + Send + Sync + Debug { async fn list(&self, remote_prefix: String) -> Result, CubeError>; - async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError>; + async fn list_with_metadata(&self, remote_prefix: String) + -> Result, CubeError>; async fn local_path(&self) -> Result; @@ -87,9 +88,14 @@ impl CommonRemoteFsUtils { /// /// Use this path to prepare files for upload. Writing into `local_path()` directly can result /// in files being deleted by the background cleanup process, see `QueueRemoteFs::cleanup_loop`. - pub async fn temp_upload_path(remote_fs: &dyn RemoteFs, remote_path: String) -> Result { + pub async fn temp_upload_path( + remote_fs: &dyn RemoteFs, + remote_path: String, + ) -> Result { // Putting files into a subdirectory prevents cleanups from removing them. - remote_fs.local_file(format!("uploads/{}", remote_path)).await + remote_fs + .local_file(format!("uploads/{}", remote_path)) + .await } /// Convention is to use this directory for creating files to be uploaded later. @@ -183,7 +189,6 @@ di_service!(RemoteFsRpcClient, [RemoteFs]); #[async_trait] impl RemoteFs for LocalDirRemoteFs { - async fn temp_upload_path(&self, remote_path: String) -> Result { CommonRemoteFsUtils::temp_upload_path(self, remote_path).await } @@ -333,7 +338,10 @@ impl RemoteFs for LocalDirRemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + async fn list_with_metadata( + &self, + remote_prefix: String, + ) -> Result, CubeError> { let remote_dir = self.remote_dir.read().await.as_ref().cloned(); let result = Self::list_recursive( remote_dir.clone().unwrap_or(self.dir.clone()), @@ -494,9 +502,16 @@ mod tests { name_maker: NameMaker, download_test: bool, ) { - assert_eq!(remote_fs.local_path().await.unwrap(), local_dir.to_str().unwrap()); + assert_eq!( + remote_fs.local_path().await.unwrap(), + local_dir.to_str().unwrap() + ); - let local_file = remote_fs.local_file("test.tst".to_string()).await.ok().unwrap(); + let local_file = remote_fs + .local_file("test.tst".to_string()) + .await + .ok() + .unwrap(); assert_eq!(local_file, local_dir.join("test.tst").to_str().unwrap()); let local_file_path = Path::new("test_dir") @@ -569,14 +584,20 @@ mod tests { for filename in root_files.iter().chain(subdir_files.iter()) { assert!(!local_dir.join(filename).is_file()); - remote_fs.download_file(filename.clone(), None).await.unwrap(); + remote_fs + .download_file(filename.clone(), None) + .await + .unwrap(); assert!(local_dir.join(filename).is_file()); } } for filename in root_files.iter().chain(subdir_files.iter()) { assert!(local_dir.join(&filename).is_file()); - assert_eq!(&remote_fs.list(filename.clone()).await.unwrap()[0], filename); + assert_eq!( + &remote_fs.list(filename.clone()).await.unwrap()[0], + filename + ); remote_fs.delete_file(filename.clone()).await.unwrap(); diff --git a/rust/cubestore/cubestore/src/remotefs/queue.rs b/rust/cubestore/cubestore/src/remotefs/queue.rs index ed667569edc36..6dd94cad30684 100644 --- a/rust/cubestore/cubestore/src/remotefs/queue.rs +++ b/rust/cubestore/cubestore/src/remotefs/queue.rs @@ -1,6 +1,6 @@ use crate::config::ConfigObj; use crate::di_service; -use crate::remotefs::{RemoteFile, RemoteFs, CommonRemoteFsUtils}; +use crate::remotefs::{CommonRemoteFsUtils, RemoteFile, RemoteFs}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -33,9 +33,9 @@ pub struct QueueRemoteFs { impl Debug for QueueRemoteFs { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { f.debug_struct("QueueRemoteFs").finish() - //TODO FIX IT - /* .field("remote_fs", &self.remote_fs) - .finish() */ + //TODO FIX IT + /* .field("remote_fs", &self.remote_fs) + .finish() */ } } @@ -187,7 +187,6 @@ impl QueueRemoteFs { #[async_trait] impl RemoteFs for QueueRemoteFs { - async fn temp_upload_path(&self, remote_path: String) -> Result { CommonRemoteFsUtils::temp_upload_path(self, remote_path).await } @@ -325,7 +324,10 @@ impl RemoteFs for QueueRemoteFs { self.remote_fs.list(remote_prefix).await } - async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + async fn list_with_metadata( + &self, + remote_prefix: String, + ) -> Result, CubeError> { self.remote_fs.list_with_metadata(remote_prefix).await } @@ -388,7 +390,6 @@ mod test { #[async_trait] impl RemoteFs for MockFs { - async fn temp_upload_path(&self, remote_path: String) -> Result { CommonRemoteFsUtils::temp_upload_path(self, remote_path).await } @@ -513,7 +514,10 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); let res = queue_fs - .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) + .upload_file( + path.to_str().unwrap().to_string(), + "temp-upload/foo.csv".to_string(), + ) .await; queue_fs.stop_processing_loops().unwrap(); r.await.unwrap().unwrap(); @@ -536,7 +540,10 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); let res = queue_fs - .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) + .upload_file( + path.to_str().unwrap().to_string(), + "temp-upload/foo.csv".to_string(), + ) .await; queue_fs.stop_processing_loops().unwrap(); r.await.unwrap().unwrap(); @@ -559,7 +566,10 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); let res = queue_fs - .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) + .upload_file( + path.to_str().unwrap().to_string(), + "temp-upload/foo.csv".to_string(), + ) .await; queue_fs.stop_processing_loops().unwrap(); r.await.unwrap().unwrap(); @@ -577,7 +587,9 @@ mod test { config.injector().get_service("original_remote_fs").await, ); let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); - let res = queue_fs.download_file("temp-upload/foo.csv".to_string(), None).await; + let res = queue_fs + .download_file("temp-upload/foo.csv".to_string(), None) + .await; match res { Ok(_) => assert!(false), Err(e) => assert!(e.is_corrupt_data()), @@ -599,13 +611,24 @@ mod test { ); let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); queue_fs - .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) + .upload_file( + path.to_str().unwrap().to_string(), + "temp-upload/foo.csv".to_string(), + ) .await .unwrap(); - std::fs::remove_file(queue_fs.local_file("temp-upload/foo.csv".to_string()).await.unwrap()).unwrap(); + std::fs::remove_file( + queue_fs + .local_file("temp-upload/foo.csv".to_string()) + .await + .unwrap(), + ) + .unwrap(); - let res = queue_fs.download_file("temp-upload/foo.csv".to_string(), Some(1)).await; + let res = queue_fs + .download_file("temp-upload/foo.csv".to_string(), Some(1)) + .await; match res { Ok(_) => assert!(false), @@ -632,12 +655,23 @@ mod test { let r = tokio::spawn(QueueRemoteFs::wait_processing_loops(queue_fs.clone())); queue_fs - .upload_file(path.to_str().unwrap().to_string(), "temp-upload/foo.csv".to_string()) + .upload_file( + path.to_str().unwrap().to_string(), + "temp-upload/foo.csv".to_string(), + ) .await .unwrap(); - std::fs::remove_file(queue_fs.local_file("temp-upload/foo.csv".to_string()).await.unwrap()).unwrap(); + std::fs::remove_file( + queue_fs + .local_file("temp-upload/foo.csv".to_string()) + .await + .unwrap(), + ) + .unwrap(); - let res = queue_fs.download_file("temp-upload/foo.csv".to_string(), None).await; + let res = queue_fs + .download_file("temp-upload/foo.csv".to_string(), None) + .await; match res { Ok(_) => assert!(false), diff --git a/rust/cubestore/cubestore/src/remotefs/s3.rs b/rust/cubestore/cubestore/src/remotefs/s3.rs index 142116ec93bd1..10285cdf9ed8f 100644 --- a/rust/cubestore/cubestore/src/remotefs/s3.rs +++ b/rust/cubestore/cubestore/src/remotefs/s3.rs @@ -1,6 +1,6 @@ use crate::app_metrics; use crate::di_service; -use crate::remotefs::{LocalDirRemoteFs, RemoteFile, RemoteFs, CommonRemoteFsUtils}; +use crate::remotefs::{CommonRemoteFsUtils, LocalDirRemoteFs, RemoteFile, RemoteFs}; use crate::util::lock::acquire_lock; use crate::CubeError; use async_trait::async_trait; @@ -133,7 +133,6 @@ di_service!(S3RemoteFs, [RemoteFs]); #[async_trait] impl RemoteFs for S3RemoteFs { - async fn temp_upload_path(&self, remote_path: String) -> Result { CommonRemoteFsUtils::temp_upload_path(self, remote_path).await } @@ -290,7 +289,10 @@ impl RemoteFs for S3RemoteFs { .collect::>()) } - async fn list_with_metadata(&self, remote_prefix: String) -> Result, CubeError> { + async fn list_with_metadata( + &self, + remote_prefix: String, + ) -> Result, CubeError> { let path = self.s3_path(&remote_prefix); let bucket = self.bucket.read().unwrap().clone(); let list = cube_ext::spawn_blocking(move || bucket.list_blocking(path, None)).await??; diff --git a/rust/cubestore/cubestore/src/sql/mod.rs b/rust/cubestore/cubestore/src/sql/mod.rs index b2711d05f43ff..c21574fdd5394 100644 --- a/rust/cubestore/cubestore/src/sql/mod.rs +++ b/rust/cubestore/cubestore/src/sql/mod.rs @@ -2316,7 +2316,6 @@ mod tests { #[async_trait::async_trait] impl RemoteFs for FailingRemoteFs { - async fn temp_upload_path(&self, remote_path: String) -> Result { CommonRemoteFsUtils::temp_upload_path(self, remote_path).await } @@ -4291,7 +4290,10 @@ mod tests { .injector .get_service_typed::() .await - .upload_file(path_2.to_str().unwrap().to_string(), "temp-uploads/orders.csv.gz".to_string()) + .upload_file( + path_2.to_str().unwrap().to_string(), + "temp-uploads/orders.csv.gz".to_string(), + ) .await .unwrap(); diff --git a/rust/cubestore/cubestore/src/store/mod.rs b/rust/cubestore/cubestore/src/store/mod.rs index 9087ac99573bf..88a28922e07d1 100644 --- a/rust/cubestore/cubestore/src/store/mod.rs +++ b/rust/cubestore/cubestore/src/store/mod.rs @@ -315,7 +315,9 @@ impl WALDataStore for WALStore { ))); } let remote_path = WALStore::wal_remote_path(wal_id); - self.remote_fs.download_file(remote_path.clone(), None).await?; + self.remote_fs + .download_file(remote_path.clone(), None) + .await?; let local_file = self.remote_fs.local_file(remote_path.clone()).await?; Ok( cube_ext::spawn_blocking(move || -> Result { @@ -735,7 +737,10 @@ impl ChunkStore { let file_size = chunk.get_row().file_size(); let chunk_id = chunk.get_id(); let remote_path = ChunkStore::chunk_file_name(chunk); - let result = self.remote_fs.download_file(remote_path.clone(), file_size).await; + let result = self + .remote_fs + .download_file(remote_path.clone(), file_size) + .await; deactivate_table_on_corrupt_data( self.meta_store.clone(), @@ -1345,7 +1350,9 @@ impl ChunkStore { let fs = self.remote_fs.clone(); Ok(cube_ext::spawn(async move { - let file_size = fs.upload_file(local_file.to_string(), remote_path.clone()).await?; + let file_size = fs + .upload_file(local_file.to_string(), remote_path.clone()) + .await?; Ok((chunk, Some(file_size))) })) } From b02fc23ba4c3cdabe7c2c8e78bd45998438ca03b Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Wed, 20 Sep 2023 23:06:25 +0300 Subject: [PATCH 09/15] update --- rust/cubestore/cubestore/src/cluster/mod.rs | 44 ++- .../cubestore/src/cluster/worker_pool.rs | 188 +++++----- .../cubestore/src/cluster/worker_services.rs | 346 ++++++++++-------- .../cubestore/src/metastore/rocks_fs.rs | 2 +- 4 files changed, 325 insertions(+), 255 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index 167487944ade9..0522f88cf64b2 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -15,7 +15,7 @@ pub mod ingestion; use crate::cluster::worker_pool::{worker_main, WorkerPool}; #[cfg(not(target_os = "windows"))] use crate::cluster::worker_services::{ - DefaultServicesServerProcessor, DefaultWorkerServicesDef, ServicesServerProcessor, + Callable, Configurator, DefaultServicesServerProcessor, DefaultServicesTransport, WorkerProcessing, }; @@ -76,8 +76,6 @@ use tokio::time::timeout; use tokio_util::sync::CancellationToken; use tracing::{instrument, Instrument}; -use self::worker_services::WorkerServicesDef; - #[automock] #[async_trait] pub trait Cluster: DIService + Send + Sync { @@ -194,7 +192,9 @@ pub struct ClusterImpl { long_running_job_notify: Arc, meta_store_sender: Sender, #[cfg(not(target_os = "windows"))] - select_process_pool: RwLock>>>, + select_process_pool: RwLock< + Option>>, + >, config_obj: Arc, query_executor: Arc, stop_token: CancellationToken, @@ -215,19 +215,19 @@ pub enum WorkerMessage { Option<(u64, u64)>, ), } -#[cfg(not(target_os = "windows"))] -pub struct WorkerProcessor; #[cfg(not(target_os = "windows"))] +pub struct WorkerConfigurator; + #[async_trait] -impl WorkerProcessing for WorkerProcessor { - type Request = WorkerMessage; - type Response = (SchemaRef, Vec, usize); +impl Configurator for WorkerConfigurator { type Config = Config; - type Services = DefaultWorkerServicesDef; - + type ServicesRequest = (); + type ServicesResponse = (); async fn configure( - _services_client: Arc<::Client>, + _services_client: Arc< + dyn Callable, + >, ) -> Result { let custom_fn = SELECT_WORKER_CONFIGURE_FN.read().unwrap().clone(); let config = if let Some(func) = custom_fn.as_ref() { @@ -239,6 +239,17 @@ impl WorkerProcessing for WorkerProcessor { }; Ok(config) } +} + +#[cfg(not(target_os = "windows"))] +pub struct WorkerProcessor; + +#[cfg(not(target_os = "windows"))] +#[async_trait] +impl WorkerProcessing for WorkerProcessor { + type Request = WorkerMessage; + type Response = (SchemaRef, Vec, usize); + type Config = Config; fn spawn_background_processes(config: Self::Config) -> Result<(), CubeError> { let custom_fn = SELECT_WORKER_SPAWN_BACKGROUND_FN.read().unwrap(); @@ -316,10 +327,14 @@ impl WorkerProcessing for WorkerProcessor { } } +type WorkerServicesTransport = DefaultServicesTransport; + #[cfg(not(target_os = "windows"))] #[ctor::ctor] fn proc_handler() { - crate::util::respawn::register_handler(worker_main::); + crate::util::respawn::register_handler( + worker_main::, + ); } lazy_static! { @@ -865,10 +880,9 @@ impl ClusterImpl { || self.config_obj.worker_bind_address().is_some()) && self.config_obj.select_worker_pool_size() > 0 { - let injector = self.injector.upgrade().unwrap(); let mut pool = self.select_process_pool.write().await; let arc = Arc::new(WorkerPool::new( - ServicesServerProcessor::init(injector).await, + DefaultServicesServerProcessor::new(), self.config_obj.select_worker_pool_size(), Duration::from_secs(self.config_obj.query_timeout()), "sel", diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index b25568a49cf6f..4e79bb97aee5c 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -18,20 +18,22 @@ use tokio::sync::{oneshot, watch, Mutex, Notify, RwLock}; use tracing::{instrument, Instrument}; use tracing_futures::WithSubscriber; -use crate::config::{env_parse, Config, WorkerServices}; use crate::cluster::worker_services::{ - ServicesClient, ServicesServer, WorkerProcessing, WorkerServicesDef, + Callable, Configurator, ServicesServer, ServicesTransport, WorkerProcessing, }; +use crate::config::env_parse; use crate::util::respawn::respawn; use crate::CubeError; use datafusion::cube_ext; use datafusion::cube_ext::catch_unwind::async_try_with_catch_unwind; -pub struct WorkerPool { - queue: Arc>>, +pub struct WorkerPool { + queue: Arc>>, stopped_tx: watch::Sender, - workers: Vec>>, - processor: PhantomData, + workers: Vec>>, + configurator: PhantomData, + processor: PhantomData

, + services_transport: PhantomData, } pub struct Message< @@ -44,9 +46,14 @@ pub struct Message< dispatcher: tracing::dispatcher::Dispatch, } -impl WorkerPool { +impl< + C: Configurator, + P: WorkerProcessing, + S: ServicesTransport, + > WorkerPool +{ pub fn new( - services_processor: Arc<::Processor>, + services_processor: Arc>, num: usize, timeout: Duration, name_prefix: &str, @@ -58,7 +65,7 @@ impl WorkerPool { let mut workers = Vec::new(); for i in 1..=num { - let process = Arc::new(WorkerProcess::::new( + let process = Arc::new(WorkerProcess::::new( services_processor.clone(), format!("{}{}", name_prefix, i), envs.clone(), @@ -74,6 +81,8 @@ impl WorkerPool { queue, workers, processor: PhantomData, + configurator: PhantomData, + services_transport: PhantomData, } } @@ -86,7 +95,7 @@ impl WorkerPool { join_all(futures).await; } - pub async fn process(&self, message: T::Request) -> Result { + pub async fn process(&self, message: P::Request) -> Result { let (tx, rx) = oneshot::channel(); self.queue.push(Message { message, @@ -132,24 +141,24 @@ impl Drop for ProcessHandleGuard { } } -pub struct WorkerProcess { +pub struct WorkerProcess { name: String, envs: Vec<(String, String)>, - queue: Arc>>, + queue: Arc>>, timeout: Duration, - processor: PhantomData, + processor: PhantomData<(C, P, S)>, stopped_rx: RwLock>, finished_notify: Arc, - services_processor: Arc<::Processor>, - services_server: Mutex::Server>>, + services_processor: Arc>, + services_server: Mutex>, } -impl WorkerProcess { +impl WorkerProcess { fn new( - services_processor: Arc<::Processor>, + services_processor: Arc>, name: String, envs: Vec<(String, String)>, - queue: Arc>>, + queue: Arc>>, timeout: Duration, stopped_rx: watch::Receiver, ) -> Self { @@ -266,14 +275,14 @@ impl WorkerProcess { #[instrument(level = "trace", skip(self, message, args_tx, res_rx))] async fn process_message( &self, - message: T::Request, - args_tx: IpcSender, - res_rx: IpcReceiver>, + message: P::Request, + args_tx: IpcSender, + res_rx: IpcReceiver>, ) -> Result< ( - T::Response, - IpcSender, - IpcReceiver>, + P::Response, + IpcSender, + IpcReceiver>, ), CubeError, > { @@ -286,8 +295,8 @@ impl WorkerProcess { &self, ) -> Result< ( - IpcSender, - IpcReceiver>, + IpcSender, + IpcReceiver>, Child, ), CubeError, @@ -309,7 +318,7 @@ impl WorkerProcess { } ctx += &self.name; - let title = T::process_titile(); + let title = P::process_titile(); let mut envs = vec![("CUBESTORE_LOG_CONTEXT".to_string(), ctx)]; envs.extend(self.envs.iter().cloned()); @@ -320,7 +329,7 @@ impl WorkerProcess { WorkerProcessArgs { args: args_rx, results: res_tx, - processor: PhantomData::::default(), + processor: PhantomData::<(C, P, S)>::default(), services_sender: service_request_tx, services_reciever: service_response_rx, }, @@ -328,7 +337,7 @@ impl WorkerProcess { &envs, )?; - *self.services_server.lock().await = Some(ServicesServer::start( + *self.services_server.lock().await = Some(S::start_server( service_request_rx, service_response_tx, self.services_processor.clone(), @@ -338,25 +347,19 @@ impl WorkerProcess { } #[derive(Serialize, Deserialize)] -pub struct WorkerProcessArgs { - args: IpcReceiver, - results: IpcSender>, - processor: PhantomData, - services_sender: IpcSender< - <::Server as ServicesServer< - ::Processor, - >>::IpcRequest, - >, - services_reciever: IpcReceiver< - <::Server as ServicesServer< - ::Processor, - >>::IpcResponse, - >, +pub struct WorkerProcessArgs { + args: IpcReceiver, + results: IpcSender>, + processor: PhantomData<(C, P, S)>, + services_sender: IpcSender, + services_reciever: IpcReceiver, } -pub fn worker_main(a: WorkerProcessArgs) -> i32 +pub fn worker_main(a: WorkerProcessArgs) -> i32 where - T: WorkerProcessing + Sync + Send + 'static, + C: Configurator, + P: WorkerProcessing, + S: ServicesTransport, { let (rx, tx, services_sender, services_reciever) = (a.args, a.results, a.services_sender, a.services_reciever); @@ -371,13 +374,12 @@ where let runtime = tokio_builder.build().unwrap(); worker_setup(&runtime); runtime.block_on(async move { - let services_client = - ::Client::connect(services_sender, services_reciever); - let config = match T::configure(services_client).await { + let services_client = S::connect(services_sender, services_reciever); + let config = match C::configure(services_client).await { Err(e) => { error!( "Error during {} worker configure: {}", - T::process_titile(), + P::process_titile(), e ); return 1; @@ -385,10 +387,10 @@ where Ok(config) => config, }; - if let Err(e) = T::spawn_background_processes(config.clone()) { + if let Err(e) = P::spawn_background_processes(config.clone()) { error!( "Error during {} worker background processes spawn: {}", - T::process_titile(), + P::process_titile(), e ); } @@ -397,7 +399,7 @@ where let res = rx.recv(); match res { Ok(args) => { - let result = match async_try_with_catch_unwind(T::process(&config, args)).await + let result = match async_try_with_catch_unwind(P::process(&config, args)).await { Ok(result) => result, Err(panic) => Err(CubeError::from(panic)), @@ -456,17 +458,17 @@ mod tests { use datafusion::cube_ext; use crate::cluster::worker_services::{ - DefaultServicesServerProcessor, DefaultWorkerServicesDef, ServicesClient, - ServicesClientImpl, ServicesServerImpl, ServicesServerProcessor, WorkerProcessing, - WorkerServicesDef, + Callable, Configurator, DefaultServicesServerProcessor, DefaultServicesTransport, + ServicesClient, ServicesClientImpl, ServicesServerImpl, ServicesTransport, + WorkerProcessing, }; - type TestPool = WorkerPool; + type TestPool = WorkerPool; #[ctor::ctor] fn test_support_init() { respawn::replace_cmd_args_in_tests(); - respawn::register_handler(worker_main::) + respawn::register_handler(worker_main::) } #[derive(Serialize, Deserialize, Eq, PartialEq, Debug)] @@ -480,22 +482,29 @@ mod tests { Foo(u64), } - pub struct Processor; + pub struct TestConfigurator; #[async_trait] - impl WorkerProcessing for Processor { + impl Configurator for TestConfigurator { type Config = Config; - type Request = Message; - type Response = Response; - type Services = DefaultWorkerServicesDef; - + type ServicesRequest = (); + type ServicesResponse = (); async fn configure( - _services_client: Arc<::Client>, + _services_client: Arc>, ) -> Result { let config = Config::default(); config.configure_injector().await; Ok(config) } + } + + pub struct Processor; + + #[async_trait] + impl WorkerProcessing for Processor { + type Config = Config; + type Request = Message; + type Response = Response; fn spawn_background_processes(_config: Self::Config) -> Result<(), CubeError> { Ok(()) @@ -517,6 +526,8 @@ mod tests { } } + type Transport = DefaultServicesTransport; + #[test] fn test_basic() { let runtime = Builder::new_current_thread().enable_all().build().unwrap(); @@ -632,22 +643,18 @@ mod tests { Ok(()) } - type TestServicePool = WorkerPool; + type TestServicePool = WorkerPool; #[derive(Debug)] pub struct TestServicesServerProcessor; #[async_trait] - impl ServicesServerProcessor for TestServicesServerProcessor { + impl Callable for TestServicesServerProcessor { type Request = i64; type Response = bool; - async fn init(_injector: Arc) -> Arc { - Arc::new(Self {}) - } - - async fn process(&self, request: i64) -> bool { - request % 2 == 0 + async fn call(&self, request: i64) -> Result { + Ok(request % 2 == 0) } } @@ -657,17 +664,9 @@ mod tests { } } - pub struct TestWorkerServicesDef; - - impl WorkerServicesDef for TestWorkerServicesDef { - type Processor = TestServicesServerProcessor; - type Server = ServicesServerImpl; - type Client = ServicesClientImpl; - } - #[derive(Clone)] pub struct TestConfig { - pub services_client: Arc<::Client>, + pub services_client: Arc>, } #[derive(Serialize, Deserialize, Eq, PartialEq, Debug)] @@ -681,21 +680,28 @@ mod tests { Odd(i64), } - pub struct ServProcessor; + pub struct ServConfigurator; #[async_trait] - impl WorkerProcessing for ServProcessor { + impl Configurator for ServConfigurator { type Config = TestConfig; - type Request = TestServReq; - type Response = TestServRes; - type Services = TestWorkerServicesDef; - + type ServicesRequest = i64; + type ServicesResponse = bool; async fn configure( - services_client: Arc<::Client>, + services_client: Arc>, ) -> Result { let config = TestConfig { services_client }; Ok(config) } + } + + pub struct ServProcessor; + + #[async_trait] + impl WorkerProcessing for ServProcessor { + type Config = TestConfig; + type Request = TestServReq; + type Response = TestServRes; fn spawn_background_processes(_config: Self::Config) -> Result<(), CubeError> { Ok(()) @@ -704,7 +710,7 @@ mod tests { config: &Self::Config, args: TestServReq, ) -> Result { - let r = config.services_client.send(args.v.clone()).await.unwrap(); + let r = config.services_client.call(args.v.clone()).await.unwrap(); let res = if r { TestServRes::Even(args.v) @@ -719,10 +725,12 @@ mod tests { } } + type ServTransport = DefaultServicesTransport; + #[ctor::ctor] fn test_services_support_init() { respawn::replace_cmd_args_in_tests(); - respawn::register_handler(worker_main::) + respawn::register_handler(worker_main::) } #[test] diff --git a/rust/cubestore/cubestore/src/cluster/worker_services.rs b/rust/cubestore/cubestore/src/cluster/worker_services.rs index a6e959ef3e127..0fb2bca3650b3 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_services.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_services.rs @@ -1,5 +1,3 @@ -use crate::config::injection::Injector; -use crate::util::cancellation_token_guard::CancellationGuard; use crate::CubeError; use async_trait::async_trait; use datafusion::cube_ext; @@ -11,20 +9,34 @@ use std::fmt::Debug; use std::marker::PhantomData; use std::sync::Arc; use std::time::Duration; -use tokio::sync::{broadcast, oneshot, Notify, RwLock}; +use tokio::sync::{broadcast, oneshot, RwLock}; use tokio::task::JoinHandle; -use tokio_util::sync::CancellationToken; #[async_trait] -pub trait WorkerProcessing { - type Config: Sync + Send + Clone + 'static; +pub trait Callable: Send + Sync + 'static { type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; - type Services: WorkerServicesDef; + async fn call(&self, req: Self::Request) -> Result; +} + +#[async_trait] +pub trait Configurator: Send + Sync + 'static { + type Config: Sync + Send + Clone + 'static; + type ServicesRequest: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type ServicesResponse: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; async fn configure( - services_client: Arc<::Client>, + services_client: Arc< + dyn Callable, + >, ) -> Result; +} + +#[async_trait] +pub trait WorkerProcessing: Send + Sync + 'static { + type Config: Sync + Send + Clone + 'static; + type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; fn spawn_background_processes(config: Self::Config) -> Result<(), CubeError>; @@ -36,27 +48,39 @@ pub trait WorkerProcessing { fn process_titile() -> String; } -#[async_trait] -pub trait ServicesServerProcessor { +pub trait ServicesTransport { type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; - async fn init(injector: Arc) -> Arc; - async fn process(&self, request: Self::Request) -> Self::Response; -} - -pub trait WorkerServicesDef { - type Processor: ServicesServerProcessor + Send + Sync + 'static; - type Server: ServicesServer + Send + Sync + 'static; - type Client: ServicesClient + Send + Sync + 'static; -} - -#[derive(Debug)] -pub struct DefaultWorkerServicesDef; + type TransportRequest: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type TransportResponse: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + + type Server: ServicesServer< + Request = Self::Request, + Response = Self::Response, + TransportRequest = Self::TransportRequest, + TransportResponse = Self::TransportResponse, + >; + type Client: ServicesClient< + Request = Self::Request, + Response = Self::Response, + TransportRequest = Self::TransportRequest, + TransportResponse = Self::TransportResponse, + >; + + fn start_server( + reciever: IpcReceiver, + sender: IpcSender, + processor: Arc>, + ) -> Self::Server { + Self::Server::start(reciever, sender, processor) + } -impl WorkerServicesDef for DefaultWorkerServicesDef { - type Processor = DefaultServicesServerProcessor; - type Server = ServicesServerImpl; - type Client = ServicesClientImpl; + fn connect( + sender: IpcSender, + reciever: IpcReceiver, + ) -> Arc { + Self::Client::connect(sender, reciever) + } } #[derive(Debug)] @@ -69,150 +93,69 @@ impl DefaultServicesServerProcessor { } #[async_trait] -impl ServicesServerProcessor for DefaultServicesServerProcessor { +impl Callable for DefaultServicesServerProcessor { type Request = (); type Response = (); - async fn init(_injector: Arc) -> Arc { - Arc::new(Self {}) - } - async fn process(&self, _request: ()) -> () { - () - } -} - -#[derive(Serialize, Deserialize, Debug)] -pub struct RequestMessage { - pub message_id: u64, - pub payload: S::Request, -} - -#[derive(Serialize, Deserialize, Debug)] -pub struct ResponseMessage { - pub message_id: u64, - pub payload: S::Response, -} -pub trait ServicesServer { - type IpcRequest: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; - type IpcResponse: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; - - fn start( - reciever: IpcReceiver, - sender: IpcSender, - processor: Arc

, - ) -> Self; - - fn stop(&self); + async fn call(&self, _request: ()) -> Result<(), CubeError> { + Ok(()) + } } -pub struct ServicesServerImpl { - join_handle: JoinHandle<()>, +pub struct DefaultServicesTransport { processor: PhantomData

, } -impl ServicesServer

- for ServicesServerImpl

-{ - type IpcRequest = RequestMessage

; - type IpcResponse = ResponseMessage

; +impl ServicesTransport for DefaultServicesTransport

{ + type Request = P::Request; + type Response = P::Response; + type TransportRequest = TransportMessage; + type TransportResponse = TransportMessage>; - fn start( - reciever: IpcReceiver, - sender: IpcSender, - processor: Arc

, - ) -> Self { - let join_handle = Self::processing_loop(reciever, sender, processor); - Self { - join_handle, - processor: PhantomData, - } - } - - fn stop(&self) { - self.join_handle.abort(); - } -} - -impl ServicesServerImpl

{ - fn processing_loop( - reciever: IpcReceiver>, - sender: IpcSender>, - processor: Arc

, - ) -> JoinHandle<()> { - cube_ext::spawn_blocking(move || loop { - let req = reciever.recv(); - - let RequestMessage { - message_id, - payload, - } = match req { - Ok(message) => message, - Err(e) => { - log::error!("Error while reading ipc service request: {:?}", e); - break; - } - }; - - let processor_to_move = processor.clone(); - let sender_to_move = sender.clone(); - - cube_ext::spawn(async move { - let res = processor_to_move.process(payload).await; - match sender_to_move.send(ResponseMessage { - message_id, - payload: res, - }) { - Ok(_) => {} - Err(e) => { - log::error!("Error while sending IPC response: {:?}", e); - } - } - }); - }) - } + type Server = ServicesServerImpl

; + type Client = ServicesClientImpl

; } -impl Drop for ServicesServerImpl

{ - fn drop(&mut self) { - self.stop(); - } +#[derive(Serialize, Deserialize, Debug)] +pub struct TransportMessage { + pub message_id: u64, + pub payload: T, } #[async_trait] -pub trait ServicesClient< - P: ServicesServerProcessor + Send + Sync + 'static, - S: ServicesServer

+ Send + Sync + 'static, -> -{ +pub trait ServicesClient: Callable { + type TransportRequest: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type TransportResponse: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; fn connect( - sender: IpcSender, - reciever: IpcReceiver, + sender: IpcSender, + reciever: IpcReceiver, ) -> Arc; - async fn send(&self, request: P::Request) -> Result; + fn stop(&self); } struct ServicesClientMessage< T: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, - R: Serialize + DeserializeOwned + Sync + Send + 'static, + R: Debug + Serialize + DeserializeOwned + Sync + Send + 'static, > { message: T, result_sender: oneshot::Sender>, } -pub struct ServicesClientImpl { +pub struct ServicesClientImpl { queue: Arc>>, handle: JoinHandle<()>, processor: PhantomData

, } #[async_trait] -impl - ServicesClient> for ServicesClientImpl

-{ +impl ServicesClient for ServicesClientImpl

{ + type TransportRequest = TransportMessage; + type TransportResponse = TransportMessage>; + fn connect( - sender: IpcSender>, - reciever: IpcReceiver>, + sender: IpcSender, + reciever: IpcReceiver, ) -> Arc { let queue = Arc::new(unlimited::Queue::new()); let handle = Self::processing_loop(sender, reciever, queue.clone()); @@ -222,7 +165,18 @@ impl queue, }) } - async fn send(&self, request: P::Request) -> Result { + + fn stop(&self) { + self.handle.abort(); + } +} + +#[async_trait] +impl Callable for ServicesClientImpl

{ + type Request = P::Request; + type Response = P::Response; + + async fn call(&self, request: Self::Request) -> Result { let (tx, rx) = oneshot::channel(); self.queue.push(ServicesClientMessage { message: request, @@ -230,16 +184,17 @@ impl }); rx.await? } - fn stop(&self) { - self.handle.abort(); - } } -impl ServicesClientImpl

{ +impl ServicesClientImpl

{ fn processing_loop( - sender: IpcSender>, - reciever: IpcReceiver>, - queue: Arc>>, + sender: IpcSender<::TransportRequest>, + reciever: IpcReceiver<::TransportResponse>, + queue: Arc< + unlimited::Queue< + ServicesClientMessage<::Request, ::Response>, + >, + >, ) -> JoinHandle<()> { let (message_broadcast_tx, _) = broadcast::channel(10000); @@ -248,7 +203,7 @@ impl ServicesClientI let recieve_loop = cube_ext::spawn_blocking(move || loop { let res = reciever.recv(); match res { - Ok(ResponseMessage { + Ok(TransportMessage { message_id, payload, }) => { @@ -280,7 +235,7 @@ impl ServicesClientI let message_id = id_counter; id_counter += 1; - let ipc_message = RequestMessage { + let ipc_message = TransportMessage { message_id, payload: message, }; @@ -312,7 +267,7 @@ impl ServicesClientI if id == message_id { let mut option = res.write().await; if let Some(res) = option.take() { - Some(Ok(res)) + Some(res) } else { Some(Err(CubeError::internal(format!( "Worker service result consumed by another listener for message id {}", @@ -356,7 +311,100 @@ impl ServicesClientI } } -impl Drop for ServicesClientImpl

{ +impl Drop for ServicesClientImpl

{ + fn drop(&mut self) { + self.stop(); + } +} + +pub trait ServicesServer { + type Request: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type Response: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type TransportRequest: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + type TransportResponse: Debug + Serialize + DeserializeOwned + Sync + Send + 'static; + + fn start( + reciever: IpcReceiver, + sender: IpcSender, + processor: Arc>, + ) -> Self; + + fn stop(&self); +} + +pub struct ServicesServerImpl { + join_handle: JoinHandle<()>, + processor: PhantomData

, +} + +impl ServicesServer for ServicesServerImpl

{ + type Request = P::Request; + type Response = P::Response; + type TransportRequest = TransportMessage; + type TransportResponse = TransportMessage>; + + fn start( + reciever: IpcReceiver, + sender: IpcSender, + processor: Arc>, + ) -> Self { + let join_handle = Self::processing_loop(reciever, sender, processor); + Self { + join_handle, + processor: PhantomData, + } + } + + fn stop(&self) { + self.join_handle.abort(); + } +} + +impl ServicesServerImpl

{ + fn processing_loop( + reciever: IpcReceiver<::TransportRequest>, + sender: IpcSender<::TransportResponse>, + processor: Arc< + dyn Callable< + Request = ::Request, + Response = ::Response, + >, + >, + ) -> JoinHandle<()> { + cube_ext::spawn_blocking(move || loop { + let req = reciever.recv(); + + let TransportMessage { + message_id, + payload, + } = match req { + Ok(message) => message, + Err(e) => { + log::error!("Error while reading ipc service request: {:?}", e); + break; + } + }; + + let processor_to_move = processor.clone(); + let sender_to_move = sender.clone(); + + cube_ext::spawn(async move { + let res = processor_to_move.call(payload).await; + match sender_to_move.send(TransportMessage { + message_id, + payload: res, + }) { + Ok(_) => {} + Err(e) => { + log::error!("Error while sending IPC response: {:?}", e); + } + } + }); + }) + } +} + +impl Drop for ServicesServerImpl

{ fn drop(&mut self) { self.stop(); } diff --git a/rust/cubestore/cubestore/src/metastore/rocks_fs.rs b/rust/cubestore/cubestore/src/metastore/rocks_fs.rs index 90d0a7a97a757..99a7464364d08 100644 --- a/rust/cubestore/cubestore/src/metastore/rocks_fs.rs +++ b/rust/cubestore/cubestore/src/metastore/rocks_fs.rs @@ -7,7 +7,7 @@ use async_trait::async_trait; use datafusion::cube_ext; use futures::future::join_all; use itertools::Itertools; -use log::{error, info, trace}; +use log::{error, info}; use regex::Regex; use std::collections::BTreeSet; use std::collections::HashSet; From 5dcc4598812109acf79a8e69ceb7add98f2394a2 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Wed, 27 Sep 2023 23:29:33 +0300 Subject: [PATCH 10/15] spawn subprocess on demand --- rust/cubestore/cubestore/src/cluster/mod.rs | 1 + .../cubestore/src/cluster/worker_pool.rs | 188 ++++++++++-------- .../cubestore/src/cluster/worker_services.rs | 10 +- rust/cubestore/cubestore/src/config/mod.rs | 14 ++ 4 files changed, 124 insertions(+), 89 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index 0522f88cf64b2..ddc3277ed3055 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -885,6 +885,7 @@ impl ClusterImpl { DefaultServicesServerProcessor::new(), self.config_obj.select_worker_pool_size(), Duration::from_secs(self.config_obj.query_timeout()), + Duration::from_secs(self.config_obj.select_worker_idle_timeout()), "sel", vec![( "_CUBESTORE_SUBPROCESS_TYPE".to_string(), diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 4e79bb97aee5c..0749a811ed266 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -15,6 +15,7 @@ use serde::{Deserialize, Serialize}; use tokio::runtime::{Builder, Runtime}; use tokio::sync::oneshot::Sender; use tokio::sync::{oneshot, watch, Mutex, Notify, RwLock}; +use tokio_util::sync::CancellationToken; use tracing::{instrument, Instrument}; use tracing_futures::WithSubscriber; @@ -56,6 +57,7 @@ impl< services_processor: Arc>, num: usize, timeout: Duration, + idle_timeout: Duration, name_prefix: &str, envs: Vec<(String, String)>, ) -> Self { @@ -71,6 +73,7 @@ impl< envs.clone(), queue.clone(), timeout.clone(), + idle_timeout.clone(), stopped_rx.clone(), )); workers.push(process.clone()); @@ -120,6 +123,7 @@ impl ProcessHandleGuard { pub fn new(handle: Child) -> Self { Self { handle } } + #[allow(dead_code)] pub fn try_wait(&mut self) -> std::io::Result> { self.handle.try_wait() } @@ -146,6 +150,7 @@ pub struct WorkerProcess, queue: Arc>>, timeout: Duration, + idle_timeout: Duration, processor: PhantomData<(C, P, S)>, stopped_rx: RwLock>, finished_notify: Arc, @@ -160,6 +165,7 @@ impl WorkerProcess, queue: Arc>>, timeout: Duration, + idle_timeout: Duration, stopped_rx: watch::Receiver, ) -> Self { WorkerProcess { @@ -168,6 +174,7 @@ impl WorkerProcess WorkerProcess { - let mut handle_guard = ProcessHandleGuard::new(handle); - loop { - let mut stopped_rx = self.stopped_rx.write().await; - let Message { - message, - sender, - span, - dispatcher, - } = tokio::select! { - res = stopped_rx.changed() => { - if res.is_err() || *stopped_rx.borrow() { - self.finished_notify.notify_waiters(); - return; - } - continue; - } - message = self.queue.pop() => { - message - } - }; - //Check if child process is killed - match handle_guard.try_wait() { - Ok(Some(_)) => { - error!( - "Worker process is killed, reshedule message in another process" - ); - self.queue.push(Message { - message, - sender, - span, - dispatcher, - }); - break; - } - Ok(None) => {} - Err(_) => { - error!( - "Can't read worker process status, reshedule message in another process" - ); - self.queue.push(Message { - message, - sender, - span, - dispatcher, - }); - break; - } + let mut handle_guard: Option = None; + let mut cancel_token: Option = None; + let mut args_channel = None; + + loop { + let mut stopped_rx = self.stopped_rx.write().await; + let Message { + message, + sender, + span, + dispatcher, + } = tokio::select! { + res = stopped_rx.changed() => { + if res.is_err() || *stopped_rx.borrow() { + self.finished_notify.notify_waiters(); + return; } - - let process_message_res_timeout = tokio::time::timeout( - self.timeout, - self.process_message(message, args_tx, res_rx), - ) - .instrument(span) - .with_subscriber(dispatcher) - .await; - let process_message_res = match process_message_res_timeout { - Ok(r) => r, - Err(e) => Err(CubeError::internal(format!( - "Timed out after waiting for {}", - e - ))), - }; - match process_message_res { - Ok((res, a, r)) => { - if sender.send(Ok(res)).is_err() { - error!("Error during worker message processing: Send Error"); - } - args_tx = a; - res_rx = r; - } - Err(e) => { - error!( - "Error during worker message processing: {}", - e.display_with_backtrace() - ); - if sender.send(Err(e.clone())).is_err() { - error!("Error during worker message processing: Send Error"); - } - break; + continue; + } + message = self.queue.pop() => { + message + } + _ = tokio::time::sleep(self.idle_timeout), if handle_guard.is_some() => { + break; + } + _ = cancel_token.as_ref().unwrap().cancelled(), if cancel_token.is_some() => { + break; + } + }; + let (args_tx, res_rx) = if args_channel.is_none() + || handle_guard.is_none() + || !handle_guard.as_mut().unwrap().is_alive() + { + let process = self.spawn_process().await; + match process { + Ok((args_tx, res_rx, handle, c_t)) => { + handle_guard = Some(ProcessHandleGuard::new(handle)); + cancel_token = Some(c_t); + (args_tx, res_rx) + } + Err(e) => { + error!("Can't start process: {}", e); + if sender + .send(Err(CubeError::internal(format!( + "Error during spawn worker pool process: {}", + e + )))) + .is_err() + { + error!("Error during worker message processing: Send Error"); } + break; } } - } - Err(e) => { - error!("Can't start process: {}", e); + } else { + args_channel.unwrap() + }; + + let process_message_res_timeout = tokio::time::timeout( + self.timeout, + self.process_message(message, args_tx, res_rx), + ) + .instrument(span) + .with_subscriber(dispatcher) + .await; + let process_message_res = match process_message_res_timeout { + Ok(r) => r, + Err(e) => Err(CubeError::internal(format!( + "Timed out after waiting for {}", + e + ))), + }; + match process_message_res { + Ok((res, a, r)) => { + if sender.send(Ok(res)).is_err() { + error!("Error during worker message processing: Send Error"); + } + args_channel = Some((a, r)); + } + Err(e) => { + error!( + "Error during worker message processing: {}", + e.display_with_backtrace() + ); + if sender.send(Err(e.clone())).is_err() { + error!("Error during worker message processing: Send Error"); + } + break; + } } } } @@ -298,6 +304,7 @@ impl WorkerProcess, IpcReceiver>, Child, + CancellationToken, ), CubeError, > { @@ -337,12 +344,14 @@ impl WorkerProcess, sender: IpcSender, processor: Arc>, + cancell_token: CancellationToken, ) -> Self::Server { - Self::Server::start(reciever, sender, processor) + Self::Server::start(reciever, sender, processor, cancell_token) } fn connect( @@ -327,6 +329,7 @@ pub trait ServicesServer { reciever: IpcReceiver, sender: IpcSender, processor: Arc>, + cancell_token: CancellationToken, ) -> Self; fn stop(&self); @@ -347,8 +350,9 @@ impl ServicesServer for ServicesServerImpl

{ reciever: IpcReceiver, sender: IpcSender, processor: Arc>, + cancell_token: CancellationToken, ) -> Self { - let join_handle = Self::processing_loop(reciever, sender, processor); + let join_handle = Self::processing_loop(reciever, sender, processor, cancell_token); Self { join_handle, processor: PhantomData, @@ -370,6 +374,7 @@ impl ServicesServerImpl

{ Response = ::Response, >, >, + cancell_token: CancellationToken, ) -> JoinHandle<()> { cube_ext::spawn_blocking(move || loop { let req = reciever.recv(); @@ -381,6 +386,7 @@ impl ServicesServerImpl

{ Ok(message) => message, Err(e) => { log::error!("Error while reading ipc service request: {:?}", e); + cancell_token.cancel(); break; } }; diff --git a/rust/cubestore/cubestore/src/config/mod.rs b/rust/cubestore/cubestore/src/config/mod.rs index ce60a54cc927f..c6c117de1eba6 100644 --- a/rust/cubestore/cubestore/src/config/mod.rs +++ b/rust/cubestore/cubestore/src/config/mod.rs @@ -390,6 +390,8 @@ pub trait ConfigObj: DIService { fn select_worker_pool_size(&self) -> usize; + fn select_worker_idle_timeout(&self) -> u64; + fn job_runners_count(&self) -> usize; fn long_term_job_runners_count(&self) -> usize; @@ -554,6 +556,7 @@ pub struct ConfigObjImpl { pub dump_dir: Option, pub store_provider: FileStoreProvider, pub select_worker_pool_size: usize, + pub select_worker_idle_timeout: u64, pub job_runners_count: usize, pub long_term_job_runners_count: usize, pub bind_address: Option, @@ -695,6 +698,10 @@ impl ConfigObj for ConfigObjImpl { self.select_worker_pool_size } + fn select_worker_idle_timeout(&self) -> u64 { + self.select_worker_idle_timeout + } + fn job_runners_count(&self) -> usize { self.job_runners_count } @@ -1247,6 +1254,12 @@ impl Config { } }, select_worker_pool_size: env_parse("CUBESTORE_SELECT_WORKERS", 4), + select_worker_idle_timeout: env_parse_duration( + "CUBESTORE_SELECT_WORKERS_IDLE_TIMEOUT", + 10 * 60, + Some(2 * 60 * 60), + None, + ), bind_address: Some( env::var("CUBESTORE_BIND_ADDR") .ok() @@ -1501,6 +1514,7 @@ impl Config { ), }, select_worker_pool_size: 0, + select_worker_idle_timeout: 600, job_runners_count: 4, long_term_job_runners_count: 8, bind_address: None, From 258c4bd981fe8edcbf0d87ec2c2ffa94962beac8 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Thu, 28 Sep 2023 01:00:49 +0300 Subject: [PATCH 11/15] update --- rust/cubestore/cubestore/src/cluster/worker_pool.rs | 2 +- rust/cubestore/cubestore/src/cluster/worker_services.rs | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 0749a811ed266..a14c2e9885f27 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -209,7 +209,7 @@ impl WorkerProcess { break; } - _ = cancel_token.as_ref().unwrap().cancelled(), if cancel_token.is_some() => { + _ = async { if cancel_token.is_none() { cancel_token.as_ref().unwrap().cancelled().await } }, if cancel_token.is_some() => { break; } }; diff --git a/rust/cubestore/cubestore/src/cluster/worker_services.rs b/rust/cubestore/cubestore/src/cluster/worker_services.rs index 4d4660e7a21b8..6d7889386b358 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_services.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_services.rs @@ -385,8 +385,10 @@ impl ServicesServerImpl

{ } = match req { Ok(message) => message, Err(e) => { - log::error!("Error while reading ipc service request: {:?}", e); - cancell_token.cancel(); + if !cancell_token.is_cancelled() { + log::error!("Error while reading ipc service request: {:?}", e); + cancell_token.cancel(); + } break; } }; From acca422b1a4fb5aefc1bf70ec043cc60b865166c Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Mon, 2 Oct 2023 17:29:01 +0300 Subject: [PATCH 12/15] update --- .../cubestore/src/cluster/worker_pool.rs | 3 +++ .../cubestore/src/cluster/worker_services.rs | 16 ++++++++-------- .../src/util/cancellation_token_guard.rs | 10 +++++----- 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index a14c2e9885f27..403b0b68fc3aa 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -5,6 +5,7 @@ use std::process::{Child, ExitStatus}; use std::sync::Arc; use std::time::Duration; +use crate::util::cancellation_token_guard::CancellationGuard; use deadqueue::unlimited; use futures::future::join_all; use ipc_channel::ipc; @@ -186,6 +187,7 @@ impl WorkerProcess = None; let mut cancel_token: Option = None; + let mut _cancel_token_guard: Option = None; let mut args_channel = None; loop { @@ -221,6 +223,7 @@ impl WorkerProcess { handle_guard = Some(ProcessHandleGuard::new(handle)); + _cancel_token_guard = Some(CancellationGuard::new(c_t.clone())); cancel_token = Some(c_t); (args_tx, res_rx) } diff --git a/rust/cubestore/cubestore/src/cluster/worker_services.rs b/rust/cubestore/cubestore/src/cluster/worker_services.rs index 6d7889386b358..cebb56117f7a7 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_services.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_services.rs @@ -72,9 +72,9 @@ pub trait ServicesTransport { reciever: IpcReceiver, sender: IpcSender, processor: Arc>, - cancell_token: CancellationToken, + cancel_token: CancellationToken, ) -> Self::Server { - Self::Server::start(reciever, sender, processor, cancell_token) + Self::Server::start(reciever, sender, processor, cancel_token) } fn connect( @@ -329,7 +329,7 @@ pub trait ServicesServer { reciever: IpcReceiver, sender: IpcSender, processor: Arc>, - cancell_token: CancellationToken, + cancel_token: CancellationToken, ) -> Self; fn stop(&self); @@ -350,9 +350,9 @@ impl ServicesServer for ServicesServerImpl

{ reciever: IpcReceiver, sender: IpcSender, processor: Arc>, - cancell_token: CancellationToken, + cancel_token: CancellationToken, ) -> Self { - let join_handle = Self::processing_loop(reciever, sender, processor, cancell_token); + let join_handle = Self::processing_loop(reciever, sender, processor, cancel_token); Self { join_handle, processor: PhantomData, @@ -374,7 +374,7 @@ impl ServicesServerImpl

{ Response = ::Response, >, >, - cancell_token: CancellationToken, + cancel_token: CancellationToken, ) -> JoinHandle<()> { cube_ext::spawn_blocking(move || loop { let req = reciever.recv(); @@ -385,9 +385,9 @@ impl ServicesServerImpl

{ } = match req { Ok(message) => message, Err(e) => { - if !cancell_token.is_cancelled() { + if !cancel_token.is_cancelled() { log::error!("Error while reading ipc service request: {:?}", e); - cancell_token.cancel(); + cancel_token.cancel(); } break; } diff --git a/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs b/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs index 2efc350048d0a..2125c94b657a9 100644 --- a/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs +++ b/rust/cubestore/cubestore/src/util/cancellation_token_guard.rs @@ -1,15 +1,15 @@ use tokio_util::sync::CancellationToken; -pub struct CancellationGuard<'a> { - token: &'a CancellationToken, +pub struct CancellationGuard { + token: CancellationToken, } -impl<'a> CancellationGuard<'a> { - pub fn new(token: &'a CancellationToken) -> Self { +impl CancellationGuard { + pub fn new(token: CancellationToken) -> Self { Self { token } } } -impl<'a> Drop for CancellationGuard<'a> { +impl Drop for CancellationGuard { fn drop(&mut self) { self.token.cancel() } From 27021b4ece0786f6aa20d904b7ae9c67e047d076 Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Mon, 2 Oct 2023 17:38:39 +0300 Subject: [PATCH 13/15] update --- rust/cubestore/cubestore/src/cluster/worker_pool.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/rust/cubestore/cubestore/src/cluster/worker_pool.rs b/rust/cubestore/cubestore/src/cluster/worker_pool.rs index 403b0b68fc3aa..bf24004282d65 100644 --- a/rust/cubestore/cubestore/src/cluster/worker_pool.rs +++ b/rust/cubestore/cubestore/src/cluster/worker_pool.rs @@ -462,7 +462,6 @@ mod tests { use tokio::runtime::Builder; use crate::cluster::worker_pool::{worker_main, WorkerPool}; - use crate::config::injection::Injector; use crate::config::Config; use crate::queryplanner::serialized_plan::SerializedLogicalPlan; use crate::util::respawn; @@ -471,7 +470,6 @@ mod tests { use crate::cluster::worker_services::{ Callable, Configurator, DefaultServicesServerProcessor, DefaultServicesTransport, - ServicesClient, ServicesClientImpl, ServicesServerImpl, ServicesTransport, WorkerProcessing, }; From cab66b6204749339e770f4b534d407abd0fcdedd Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Fri, 6 Oct 2023 00:43:02 +0300 Subject: [PATCH 14/15] fix windows build --- rust/cubestore/cubestore/src/cluster/mod.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/rust/cubestore/cubestore/src/cluster/mod.rs b/rust/cubestore/cubestore/src/cluster/mod.rs index ddc3277ed3055..9fd5a88316192 100644 --- a/rust/cubestore/cubestore/src/cluster/mod.rs +++ b/rust/cubestore/cubestore/src/cluster/mod.rs @@ -219,6 +219,7 @@ pub enum WorkerMessage { #[cfg(not(target_os = "windows"))] pub struct WorkerConfigurator; +#[cfg(not(target_os = "windows"))] #[async_trait] impl Configurator for WorkerConfigurator { type Config = Config; @@ -327,6 +328,7 @@ impl WorkerProcessing for WorkerProcessor { } } +#[cfg(not(target_os = "windows"))] type WorkerServicesTransport = DefaultServicesTransport; #[cfg(not(target_os = "windows"))] From 714eafe1f71c5b1a7af132428db3a31daec531eb Mon Sep 17 00:00:00 2001 From: Alexandr Romanenko Date: Sat, 21 Oct 2023 18:56:12 +0300 Subject: [PATCH 15/15] update --- rust/cubestore/cubestore/src/import/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rust/cubestore/cubestore/src/import/mod.rs b/rust/cubestore/cubestore/src/import/mod.rs index 399fc690665fe..d2f907c88428c 100644 --- a/rust/cubestore/cubestore/src/import/mod.rs +++ b/rust/cubestore/cubestore/src/import/mod.rs @@ -861,7 +861,7 @@ impl LocationHelper { } } else if location.starts_with("temp://") { let remote_path = Self::temp_uploads_path(location); - match remote_fs.list_with_metadata(&remote_path).await { + match remote_fs.list_with_metadata(remote_path).await { Ok(list) => { let list_res = list.iter().next().ok_or(CubeError::internal(format!( "Location {} can't be listed in remote_fs",