Skip to content

Commit

Permalink
feat: use seperated runtime for source manager (risingwavelabs#12571)
Browse files Browse the repository at this point in the history
Signed-off-by: Shanicky Chen <[email protected]>
  • Loading branch information
shanicky authored Sep 27, 2023
1 parent 1376e0e commit 454e72d
Show file tree
Hide file tree
Showing 2 changed files with 19 additions and 5 deletions.
4 changes: 2 additions & 2 deletions src/meta/src/rpc/election/etcd.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,9 @@ use tokio::sync::{oneshot, watch};
use tokio::time;
use tokio_stream::StreamExt;

use crate::rpc::election::{ElectionClient, ElectionMember, META_ELECTION_KEY};
use crate::rpc::election::META_ELECTION_KEY;
use crate::storage::WrappedEtcdClient;
use crate::MetaResult;
use crate::{ElectionClient, ElectionMember, MetaResult};

pub struct EtcdElectionClient {
id: String,
Expand Down
20 changes: 17 additions & 3 deletions src/meta/src/stream/source_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ use std::time::Duration;
use anyhow::anyhow;
use itertools::Itertools;
use risingwave_common::catalog::TableId;
use risingwave_common::util::runtime::BackgroundShutdownRuntime;
use risingwave_connector::dispatch_source_prop;
use risingwave_connector::source::{
ConnectorProperties, SourceEnumeratorContext, SourceEnumeratorInfo, SourceProperties,
Expand Down Expand Up @@ -52,6 +53,7 @@ pub struct SourceManager {
barrier_scheduler: BarrierScheduler,
core: Mutex<SourceManagerCore>,
metrics: Arc<MetaMetrics>,
runtime: Arc<BackgroundShutdownRuntime>,
}

const MAX_FAIL_CNT: u32 = 10;
Expand Down Expand Up @@ -496,11 +498,20 @@ impl SourceManager {
fragment_manager: FragmentManagerRef,
metrics: Arc<MetaMetrics>,
) -> MetaResult<Self> {
let runtime = tokio::runtime::Builder::new_multi_thread()
.thread_name("risingwave-source-manager")
.enable_all()
.build()
.map_err(|e| anyhow!(e))?;

let runtime: Arc<BackgroundShutdownRuntime> = Arc::new(runtime.into());

let mut managed_sources = HashMap::new();
{
let sources = catalog_manager.list_sources().await;
for source in sources {
Self::create_source_worker_async(
runtime.clone(),
env.connector_client(),
source,
&mut managed_sources,
Expand Down Expand Up @@ -534,6 +545,7 @@ impl SourceManager {
core,
paused: Mutex::new(()),
metrics,
runtime,
})
}

Expand Down Expand Up @@ -675,6 +687,7 @@ impl SourceManager {
tracing::warn!("source {} already registered", source.get_id());
} else {
Self::create_source_worker(
self.runtime.clone(),
self.env.connector_client(),
source,
&mut core.managed_sources,
Expand All @@ -687,6 +700,7 @@ impl SourceManager {
}

fn create_source_worker_async(
runtime: Arc<BackgroundShutdownRuntime>,
connector_client: Option<ConnectorClient>,
source: Source,
managed_sources: &mut HashMap<SourceId, ConnectorSourceWorkerHandle>,
Expand All @@ -701,8 +715,7 @@ impl SourceManager {
let source_id = source.id;

let connector_properties = extract_prop_from_source(&source)?;

let handle = tokio::spawn(async move {
let handle = runtime.spawn(async move {
let mut ticker = time::interval(Self::DEFAULT_SOURCE_TICK_INTERVAL);
ticker.set_missed_tick_behavior(MissedTickBehavior::Skip);

Expand Down Expand Up @@ -745,6 +758,7 @@ impl SourceManager {
}

async fn create_source_worker(
runtime: Arc<BackgroundShutdownRuntime>,
connector_client: Option<ConnectorClient>,
source: &Source,
managed_sources: &mut HashMap<SourceId, ConnectorSourceWorkerHandle>,
Expand Down Expand Up @@ -785,7 +799,7 @@ impl SourceManager {
})??;
}

tokio::spawn(async move { worker.run(sync_call_rx).await })
runtime.spawn(async move { worker.run(sync_call_rx).await })
});

managed_sources.insert(
Expand Down

0 comments on commit 454e72d

Please sign in to comment.