|
| 1 | +// Copyright (C) 2024 Quickwit, Inc. |
| 2 | +// |
| 3 | +// Quickwit is offered under the AGPL v3.0 and as commercial software. |
| 4 | +// For commercial licensing, contact us at [email protected]. |
| 5 | +// |
| 6 | +// AGPL: |
| 7 | +// This program is free software: you can redistribute it and/or modify |
| 8 | +// it under the terms of the GNU Affero General Public License as |
| 9 | +// published by the Free Software Foundation, either version 3 of the |
| 10 | +// License, or (at your option) any later version. |
| 11 | +// |
| 12 | +// This program is distributed in the hope that it will be useful, |
| 13 | +// but WITHOUT ANY WARRANTY; without even the implied warranty of |
| 14 | +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the |
| 15 | +// GNU Affero General Public License for more details. |
| 16 | +// |
| 17 | +// You should have received a copy of the GNU Affero General Public License |
| 18 | +// along with this program. If not, see <http://www.gnu.org/licenses/>. |
| 19 | + |
| 20 | +use std::collections::VecDeque; |
| 21 | +use std::sync::{Arc, Mutex}; |
| 22 | + |
| 23 | +use quickwit_common::metrics::GaugeGuard; |
| 24 | +use tokio::sync::oneshot; |
| 25 | + |
| 26 | +/// `SearchPermitProvider` is a distributor of permits to perform single split |
| 27 | +/// search operation. |
| 28 | +/// |
| 29 | +/// Requests are served in order. |
| 30 | +#[derive(Clone)] |
| 31 | +pub struct SearchPermitProvider { |
| 32 | + inner_arc: Arc<Mutex<InnerSearchPermitProvider>>, |
| 33 | +} |
| 34 | + |
| 35 | +impl SearchPermitProvider { |
| 36 | + pub fn new(num_permits: usize) -> SearchPermitProvider { |
| 37 | + SearchPermitProvider { |
| 38 | + inner_arc: Arc::new(Mutex::new(InnerSearchPermitProvider { |
| 39 | + num_permits_available: num_permits, |
| 40 | + permits_requests: VecDeque::new(), |
| 41 | + })), |
| 42 | + } |
| 43 | + } |
| 44 | + |
| 45 | + /// Returns a future permit in the form of a oneshot Receiver channel. |
| 46 | + /// |
| 47 | + /// At this point the permit is not acquired yet. |
| 48 | + #[must_use] |
| 49 | + pub fn get_permit(&self) -> oneshot::Receiver<SearchPermit> { |
| 50 | + let mut permits_lock = self.inner_arc.lock().unwrap(); |
| 51 | + permits_lock.get_permit(&self.inner_arc) |
| 52 | + } |
| 53 | + |
| 54 | + /// Returns a list of future permits in the form of oneshot Receiver channels. |
| 55 | + /// |
| 56 | + /// The permits returned are guaranteed to be resolved in order. |
| 57 | + /// In addition, the permits are guaranteed to be resolved before permits returned by |
| 58 | + /// subsequent calls to this function (or `get_permit`). |
| 59 | + #[must_use] |
| 60 | + pub fn get_permits(&self, num_permits: usize) -> Vec<oneshot::Receiver<SearchPermit>> { |
| 61 | + let mut permits_lock = self.inner_arc.lock().unwrap(); |
| 62 | + permits_lock.get_permits(num_permits, &self.inner_arc) |
| 63 | + } |
| 64 | +} |
| 65 | + |
| 66 | +struct InnerSearchPermitProvider { |
| 67 | + num_permits_available: usize, |
| 68 | + permits_requests: VecDeque<oneshot::Sender<SearchPermit>>, |
| 69 | +} |
| 70 | + |
| 71 | +impl InnerSearchPermitProvider { |
| 72 | + fn get_permit( |
| 73 | + &mut self, |
| 74 | + inner_arc: &Arc<Mutex<InnerSearchPermitProvider>>, |
| 75 | + ) -> oneshot::Receiver<SearchPermit> { |
| 76 | + let (tx, rx) = oneshot::channel(); |
| 77 | + self.permits_requests.push_back(tx); |
| 78 | + self.assign_available_permits(inner_arc); |
| 79 | + rx |
| 80 | + } |
| 81 | + |
| 82 | + fn get_permits( |
| 83 | + &mut self, |
| 84 | + num_permits: usize, |
| 85 | + inner_arc: &Arc<Mutex<InnerSearchPermitProvider>>, |
| 86 | + ) -> Vec<oneshot::Receiver<SearchPermit>> { |
| 87 | + let mut permits = Vec::with_capacity(num_permits); |
| 88 | + for _ in 0..num_permits { |
| 89 | + let (tx, rx) = oneshot::channel(); |
| 90 | + self.permits_requests.push_back(tx); |
| 91 | + permits.push(rx); |
| 92 | + } |
| 93 | + self.assign_available_permits(inner_arc); |
| 94 | + permits |
| 95 | + } |
| 96 | + |
| 97 | + fn recycle_permit(&mut self, inner_arc: &Arc<Mutex<InnerSearchPermitProvider>>) { |
| 98 | + self.num_permits_available += 1; |
| 99 | + self.assign_available_permits(inner_arc); |
| 100 | + } |
| 101 | + |
| 102 | + fn assign_available_permits(&mut self, inner_arc: &Arc<Mutex<InnerSearchPermitProvider>>) { |
| 103 | + while self.num_permits_available > 0 { |
| 104 | + let Some(sender) = self.permits_requests.pop_front() else { |
| 105 | + break; |
| 106 | + }; |
| 107 | + let mut ongoing_gauge_guard = GaugeGuard::from_gauge( |
| 108 | + &crate::SEARCH_METRICS.leaf_search_single_split_tasks_ongoing, |
| 109 | + ); |
| 110 | + ongoing_gauge_guard.add(1); |
| 111 | + let send_res = sender.send(SearchPermit { |
| 112 | + _ongoing_gauge_guard: ongoing_gauge_guard, |
| 113 | + inner_arc: inner_arc.clone(), |
| 114 | + recycle_on_drop: true, |
| 115 | + }); |
| 116 | + match send_res { |
| 117 | + Ok(()) => { |
| 118 | + self.num_permits_available -= 1; |
| 119 | + } |
| 120 | + Err(search_permit) => { |
| 121 | + search_permit.drop_without_recycling_permit(); |
| 122 | + } |
| 123 | + } |
| 124 | + } |
| 125 | + crate::SEARCH_METRICS |
| 126 | + .leaf_search_single_split_tasks_pending |
| 127 | + .set(self.permits_requests.len() as i64); |
| 128 | + } |
| 129 | +} |
| 130 | + |
| 131 | +pub struct SearchPermit { |
| 132 | + _ongoing_gauge_guard: GaugeGuard<'static>, |
| 133 | + inner_arc: Arc<Mutex<InnerSearchPermitProvider>>, |
| 134 | + recycle_on_drop: bool, |
| 135 | +} |
| 136 | + |
| 137 | +impl SearchPermit { |
| 138 | + fn drop_without_recycling_permit(mut self) { |
| 139 | + self.recycle_on_drop = false; |
| 140 | + drop(self); |
| 141 | + } |
| 142 | +} |
| 143 | + |
| 144 | +impl Drop for SearchPermit { |
| 145 | + fn drop(&mut self) { |
| 146 | + if !self.recycle_on_drop { |
| 147 | + return; |
| 148 | + } |
| 149 | + let mut inner_guard = self.inner_arc.lock().unwrap(); |
| 150 | + inner_guard.recycle_permit(&self.inner_arc.clone()); |
| 151 | + } |
| 152 | +} |
| 153 | + |
| 154 | +#[cfg(test)] |
| 155 | +mod tests { |
| 156 | + use tokio::task::JoinSet; |
| 157 | + |
| 158 | + use super::*; |
| 159 | + |
| 160 | + #[tokio::test] |
| 161 | + async fn test_search_permits_get_permits_future() { |
| 162 | + // We test here that `get_permits_futures` does not interleave |
| 163 | + let search_permits = SearchPermitProvider::new(1); |
| 164 | + let mut all_futures = Vec::new(); |
| 165 | + let first_batch_of_permits = search_permits.get_permits(10); |
| 166 | + assert_eq!(first_batch_of_permits.len(), 10); |
| 167 | + all_futures.extend( |
| 168 | + first_batch_of_permits |
| 169 | + .into_iter() |
| 170 | + .enumerate() |
| 171 | + .map(move |(i, fut)| ((1, i), fut)), |
| 172 | + ); |
| 173 | + |
| 174 | + let second_batch_of_permits = search_permits.get_permits(10); |
| 175 | + assert_eq!(second_batch_of_permits.len(), 10); |
| 176 | + all_futures.extend( |
| 177 | + second_batch_of_permits |
| 178 | + .into_iter() |
| 179 | + .enumerate() |
| 180 | + .map(move |(i, fut)| ((2, i), fut)), |
| 181 | + ); |
| 182 | + |
| 183 | + use rand::seq::SliceRandom; |
| 184 | + // not super useful, considering what join set does, but still a tiny bit more sound. |
| 185 | + all_futures.shuffle(&mut rand::thread_rng()); |
| 186 | + |
| 187 | + let mut join_set = JoinSet::new(); |
| 188 | + for (res, fut) in all_futures { |
| 189 | + join_set.spawn(async move { |
| 190 | + let permit = fut.await; |
| 191 | + (res, permit) |
| 192 | + }); |
| 193 | + } |
| 194 | + let mut ordered_result: Vec<(usize, usize)> = Vec::with_capacity(20); |
| 195 | + while let Some(Ok(((batch_id, order), _permit))) = join_set.join_next().await { |
| 196 | + ordered_result.push((batch_id, order)); |
| 197 | + } |
| 198 | + |
| 199 | + assert_eq!(ordered_result.len(), 20); |
| 200 | + for (i, res) in ordered_result[0..10].iter().enumerate() { |
| 201 | + assert_eq!(res, &(1, i)); |
| 202 | + } |
| 203 | + for (i, res) in ordered_result[10..20].iter().enumerate() { |
| 204 | + assert_eq!(res, &(2, i)); |
| 205 | + } |
| 206 | + } |
| 207 | + |
| 208 | + #[tokio::test] |
| 209 | + async fn test_search_permits_receiver_race_condition() { |
| 210 | + // Here we test that we don't have a problem if the Receiver is dropped. |
| 211 | + // In particular, we want to check that there is not a race condition where drop attempts to |
| 212 | + // lock the mutex. |
| 213 | + let search_permits = SearchPermitProvider::new(1); |
| 214 | + let permit_rx = search_permits.get_permit(); |
| 215 | + let permit_rx2 = search_permits.get_permit(); |
| 216 | + drop(permit_rx2); |
| 217 | + drop(permit_rx); |
| 218 | + let _permit_rx = search_permits.get_permit(); |
| 219 | + } |
| 220 | +} |
0 commit comments