@@ -28,11 +28,17 @@ use std::time::Instant;
2828use std:: time:: SystemTime ;
2929
3030use databend_common_ast:: ast:: ExplainKind ;
31+ use databend_common_base:: base:: escape_for_key;
3132use databend_common_base:: base:: GlobalInstance ;
3233use databend_common_catalog:: table_context:: TableContext ;
34+ use databend_common_config:: InnerConfig ;
3335use databend_common_exception:: ErrorCode ;
3436use databend_common_exception:: Result ;
3537use databend_common_meta_app:: principal:: UserInfo ;
38+ use databend_common_meta_semaphore:: acquirer:: Permit ;
39+ use databend_common_meta_semaphore:: errors:: AcquireError ;
40+ use databend_common_meta_store:: MetaStore ;
41+ use databend_common_meta_store:: MetaStoreProvider ;
3642use databend_common_metrics:: session:: dec_session_running_acquired_queries;
3743use databend_common_metrics:: session:: inc_session_running_acquired_queries;
3844use databend_common_metrics:: session:: incr_session_queue_abort_count;
@@ -47,18 +53,17 @@ use databend_common_sql::PlanExtras;
4753use log:: info;
4854use parking_lot:: Mutex ;
4955use pin_project_lite:: pin_project;
50- use tokio:: sync:: AcquireError ;
51- use tokio:: sync:: OwnedSemaphorePermit ;
52- use tokio:: sync:: Semaphore ;
5356use tokio:: time:: error:: Elapsed ;
5457
5558use crate :: sessions:: QueryContext ;
5659
5760pub trait QueueData : Send + Sync + ' static {
58- type Key : Send + Sync + Eq + Hash + Display + Clone + ' static ;
61+ type Key : Send + Sync + Eq + Hash + Display + Clone + ToString + ' static ;
5962
6063 fn get_key ( & self ) -> Self :: Key ;
6164
65+ fn get_lock_key ( & self ) -> String ;
66+
6267 fn remove_error_message ( key : Option < Self :: Key > ) -> ErrorCode ;
6368
6469 fn timeout ( & self ) -> Duration ;
@@ -78,29 +83,39 @@ pub(crate) struct Inner<Data: QueueData> {
7883}
7984
8085pub struct QueueManager < Data : QueueData > {
81- semaphore : Arc < Semaphore > ,
86+ permits : usize ,
87+ meta_store : MetaStore ,
8288 queue : Mutex < HashMap < Data :: Key , Inner < Data > > > ,
8389}
8490
8591impl < Data : QueueData > QueueManager < Data > {
86- pub fn init ( permits : usize ) -> Result < ( ) > {
92+ pub async fn init ( permits : usize , conf : & InnerConfig ) -> Result < ( ) > {
93+ let metastore = {
94+ let provider = Arc :: new ( MetaStoreProvider :: new ( conf. meta . to_meta_grpc_client_conf ( ) ) ) ;
95+
96+ provider. create_meta_store ( ) . await . map_err ( |e| {
97+ ErrorCode :: MetaServiceError ( format ! ( "Failed to create meta store: {}" , e) )
98+ } ) ?
99+ } ;
100+
87101 info ! ( "queue manager permits: {:?}" , permits) ;
88- GlobalInstance :: set ( Self :: create ( permits) ) ;
102+ GlobalInstance :: set ( Self :: create ( permits, metastore ) ) ;
89103 Ok ( ( ) )
90104 }
91105
92106 pub fn instance ( ) -> Arc < Self > {
93107 GlobalInstance :: get :: < Arc < Self > > ( )
94108 }
95109
96- pub fn create ( mut permits : usize ) -> Arc < QueueManager < Data > > {
110+ pub fn create ( mut permits : usize , meta_store : MetaStore ) -> Arc < QueueManager < Data > > {
97111 if permits == 0 {
98112 permits = usize:: MAX >> 4 ;
99113 }
100114
101115 Arc :: new ( QueueManager {
116+ permits,
117+ meta_store,
102118 queue : Mutex :: new ( HashMap :: new ( ) ) ,
103- semaphore : Arc :: new ( Semaphore :: new ( permits) ) ,
104119 } )
105120 }
106121
@@ -139,9 +154,16 @@ impl<Data: QueueData> QueueManager<Data> {
139154 ) ;
140155
141156 let timeout = data. timeout ( ) ;
157+ let semaphore_acquire = self . meta_store . new_acquired (
158+ data. get_lock_key ( ) ,
159+ self . permits as u64 ,
160+ data. get_key ( ) , // ID of this acquirer
161+ Duration :: from_secs ( 3 ) ,
162+ ) ;
163+
142164 let future = AcquireQueueFuture :: create (
143165 Arc :: new ( data) ,
144- tokio:: time:: timeout ( timeout, self . semaphore . clone ( ) . acquire_owned ( ) ) ,
166+ tokio:: time:: timeout ( timeout, semaphore_acquire ) ,
145167 self . clone ( ) ,
146168 ) ;
147169 let start_time = SystemTime :: now ( ) ;
@@ -209,9 +231,12 @@ impl<Data: QueueData> QueueManager<Data> {
209231
210232pub struct AcquireQueueGuard {
211233 #[ allow( dead_code) ]
212- permit : Option < OwnedSemaphorePermit > ,
234+ permit : Option < Permit > ,
213235}
214236
237+ unsafe impl Send for AcquireQueueGuard { }
238+ unsafe impl Sync for AcquireQueueGuard { }
239+
215240impl Drop for AcquireQueueGuard {
216241 fn drop ( & mut self ) {
217242 if self . permit . is_some ( ) {
@@ -221,14 +246,14 @@ impl Drop for AcquireQueueGuard {
221246}
222247
223248impl AcquireQueueGuard {
224- pub fn create ( permit : Option < OwnedSemaphorePermit > ) -> Self {
249+ pub fn create ( permit : Option < Permit > ) -> Self {
225250 AcquireQueueGuard { permit }
226251 }
227252}
228253
229254pin_project ! {
230255 pub struct AcquireQueueFuture <Data : QueueData , T >
231- where T : Future <Output = std:: result:: Result < std:: result:: Result <OwnedSemaphorePermit , AcquireError >, Elapsed >>
256+ where T : Future <Output = std:: result:: Result <std:: result:: Result <Permit , AcquireError >, Elapsed >>
232257{
233258 #[ pin]
234259 inner: T ,
@@ -243,12 +268,7 @@ where T: Future<Output = std::result::Result< std::result::Result<OwnedSemaphor
243268}
244269
245270impl < Data : QueueData , T > AcquireQueueFuture < Data , T >
246- where T : Future <
247- Output = std:: result:: Result <
248- std:: result:: Result < OwnedSemaphorePermit , AcquireError > ,
249- Elapsed ,
250- > ,
251- >
271+ where T : Future < Output = std:: result:: Result < std:: result:: Result < Permit , AcquireError > , Elapsed > >
252272{
253273 pub fn create ( data : Arc < Data > , inner : T , mgr : Arc < QueueManager < Data > > ) -> Self {
254274 AcquireQueueFuture {
@@ -263,12 +283,7 @@ where T: Future<
263283}
264284
265285impl < Data : QueueData , T > Future for AcquireQueueFuture < Data , T >
266- where T : Future <
267- Output = std:: result:: Result <
268- std:: result:: Result < OwnedSemaphorePermit , AcquireError > ,
269- Elapsed ,
270- > ,
271- >
286+ where T : Future < Output = std:: result:: Result < std:: result:: Result < Permit , AcquireError > , Elapsed > >
272287{
273288 type Output = Result < AcquireQueueGuard > ;
274289
@@ -443,6 +458,22 @@ impl QueueData for QueryEntry {
443458 self . query_id . clone ( )
444459 }
445460
461+ fn get_lock_key ( & self ) -> String {
462+ let cluster = self . ctx . get_cluster ( ) ;
463+ let local_id = escape_for_key ( & cluster. local_id ) . unwrap ( ) ;
464+ let mut lock_key = format ! ( "__fd_queries_queue/lost_contact/{}" , local_id) ;
465+
466+ for node in & cluster. nodes {
467+ if node. id == cluster. local_id {
468+ let cluster_id = escape_for_key ( & node. cluster_id ) . unwrap ( ) ;
469+ let warehouse_id = escape_for_key ( & node. warehouse_id ) . unwrap ( ) ;
470+ lock_key = format ! ( "__fd_queries_queue/queue/{}/{}" , warehouse_id, cluster_id) ;
471+ }
472+ }
473+
474+ lock_key
475+ }
476+
446477 fn remove_error_message ( key : Option < Self :: Key > ) -> ErrorCode {
447478 match key {
448479 None => ErrorCode :: AbortedQuery ( "The query has be kill while in queries queue" ) ,
0 commit comments