|
12 | 12 | // See the License for the specific language governing permissions and |
13 | 13 | // limitations under the License. |
14 | 14 |
|
| 15 | +use std::str::FromStr; |
| 16 | +use std::sync::Arc; |
| 17 | + |
15 | 18 | use base64::prelude::*; |
16 | 19 | use prost::Message; |
| 20 | +use quickwit_common::uri::Uri; |
| 21 | +use quickwit_doc_mapper::DocMapper; |
17 | 22 | use quickwit_proto::search::lambda_single_split_result::Outcome; |
18 | 23 | use quickwit_proto::search::{ |
19 | | - LambdaSearchResponses, LambdaSingleSplitResult, LeafSearchRequest, SplitIdAndFooterOffsets, |
| 24 | + LambdaSearchResponses, LambdaSingleSplitResult, LeafRequestRef, LeafSearchRequest, |
| 25 | + SearchRequest, |
20 | 26 | }; |
21 | | -use quickwit_search::leaf::multi_index_leaf_search; |
| 27 | +use quickwit_search::leaf::single_doc_mapping_leaf_search; |
| 28 | +use quickwit_storage::Storage; |
22 | 29 | use serde::{Deserialize, Serialize}; |
23 | 30 | use tracing::{error, info, instrument, warn}; |
24 | 31 |
|
@@ -50,57 +57,106 @@ pub async fn handle_leaf_search( |
50 | 57 | ctx: &LambdaSearcherContext, |
51 | 58 | ) -> LambdaResult<LambdaSearchResponsePayload> { |
52 | 59 | // Decode base64 payload |
53 | | - let request_bytes = BASE64_STANDARD |
| 60 | + let request_bytes: Vec<u8> = BASE64_STANDARD |
54 | 61 | .decode(&event.payload) |
55 | | - .map_err(|e| LambdaError::Serialization(format!("base64 decode error: {}", e)))?; |
| 62 | + .map_err(|err| LambdaError::Serialization(format!("base64 decode error: {}", err)))?; |
56 | 63 |
|
57 | 64 | // Deserialize LeafSearchRequest |
58 | 65 | let leaf_search_request = LeafSearchRequest::decode(&request_bytes[..])?; |
59 | 66 |
|
60 | | - let all_splits: Vec<(usize, SplitIdAndFooterOffsets)> = |
61 | | - leaf_search_request |
62 | | - .leaf_requests |
63 | | - .iter() |
64 | | - .enumerate() |
65 | | - .flat_map(|(leaf_req_idx, leaf_request_ref)| { |
66 | | - leaf_request_ref.split_offsets.iter().cloned().map( |
67 | | - move |split_id_and_footer_offsets| (leaf_req_idx, split_id_and_footer_offsets), |
68 | | - ) |
69 | | - }) |
70 | | - .collect(); |
| 67 | + // Unpack the shared fields once instead of cloning per split. |
| 68 | + let search_request: Arc<SearchRequest> = leaf_search_request |
| 69 | + .search_request |
| 70 | + .ok_or_else(|| LambdaError::Internal("no search request".to_string()))? |
| 71 | + .into(); |
71 | 72 |
|
72 | | - let num_splits = all_splits.len(); |
73 | | - info!(num_splits, "processing leaf search request (per-split)"); |
| 73 | + let doc_mappers: Vec<Arc<DocMapper>> = leaf_search_request |
| 74 | + .doc_mappers |
| 75 | + .iter() |
| 76 | + .map(String::as_str) |
| 77 | + .map(serde_json::from_str::<Arc<DocMapper>>) |
| 78 | + .collect::<Result<Vec<_>, _>>() |
| 79 | + .map_err(|err| { |
| 80 | + LambdaError::Internal(format!("failed to deserialize doc mapper: `{err}`")) |
| 81 | + })?; |
| 82 | + |
| 83 | + // Resolve storage for every index URI upfront. |
| 84 | + let mut storages: Vec<Arc<dyn quickwit_storage::Storage>> = |
| 85 | + Vec::with_capacity(leaf_search_request.index_uris.len()); |
| 86 | + for uri_str in &leaf_search_request.index_uris { |
| 87 | + let uri = Uri::from_str(uri_str) |
| 88 | + .map_err(|err| LambdaError::Internal(format!("invalid index uri: {err}")))?; |
| 89 | + let storage = |
| 90 | + ctx.storage_resolver.resolve(&uri).await.map_err(|err| { |
| 91 | + LambdaError::Internal(format!("failed to resolve storage: {err}")) |
| 92 | + })?; |
| 93 | + storages.push(storage); |
| 94 | + } |
| 95 | + |
| 96 | + let split_results: Vec<LambdaSingleSplitResult> = lambda_leaf_search( |
| 97 | + search_request, |
| 98 | + leaf_search_request.leaf_requests, |
| 99 | + &doc_mappers[..], |
| 100 | + &storages[..], |
| 101 | + ctx, |
| 102 | + ) |
| 103 | + .await?; |
| 104 | + let wrapper = LambdaSearchResponses { split_results }; |
| 105 | + let response_bytes = wrapper.encode_to_vec(); |
| 106 | + let payload = BASE64_STANDARD.encode(&response_bytes); |
| 107 | + |
| 108 | + Ok(LambdaSearchResponsePayload { payload }) |
| 109 | +} |
74 | 110 |
|
75 | | - // Process each split in parallel using a JoinSet. The SearchPermitProvider |
76 | | - // inside SearcherContext gates concurrency based on memory budget. |
| 111 | +/// Lambda leaf search returns individual split results. |
| 112 | +async fn lambda_leaf_search( |
| 113 | + search_request: Arc<SearchRequest>, |
| 114 | + leaf_req_ref: Vec<LeafRequestRef>, |
| 115 | + doc_mappers: &[Arc<DocMapper>], |
| 116 | + storages: &[Arc<dyn Storage>], |
| 117 | + ctx: &LambdaSearcherContext, |
| 118 | +) -> LambdaResult<Vec<LambdaSingleSplitResult>> { |
| 119 | + // Flatten leaf_requests into per-split tasks using pre-resolved Arc references. |
77 | 120 | let mut split_search_joinset: tokio::task::JoinSet<(String, Result<_, String>)> = |
78 | 121 | tokio::task::JoinSet::new(); |
79 | | - for (leaf_req_idx, split) in all_splits { |
80 | | - let split_id = split.split_id.clone(); |
81 | | - let leaf_request_ref = &leaf_search_request.leaf_requests[leaf_req_idx]; |
82 | | - let single_split_request = LeafSearchRequest { |
83 | | - search_request: leaf_search_request.search_request.clone(), |
84 | | - doc_mappers: leaf_search_request.doc_mappers.clone(), |
85 | | - index_uris: leaf_search_request.index_uris.clone(), |
86 | | - leaf_requests: vec![quickwit_proto::search::LeafRequestRef { |
87 | | - index_uri_ord: leaf_request_ref.index_uri_ord, |
88 | | - doc_mapper_ord: leaf_request_ref.doc_mapper_ord, |
89 | | - split_offsets: vec![split], |
90 | | - }], |
91 | | - }; |
92 | | - |
93 | | - let searcher_context = ctx.searcher_context.clone(); |
94 | | - let storage_resolver = ctx.storage_resolver.clone(); |
95 | | - split_search_joinset.spawn(async move { |
96 | | - let result = |
97 | | - multi_index_leaf_search(searcher_context, single_split_request, storage_resolver) |
98 | | - .await |
99 | | - .map_err(|err| format!("{err}")); |
100 | | - (split_id, result) |
101 | | - }); |
| 122 | + |
| 123 | + for leaf_req in leaf_req_ref { |
| 124 | + let doc_mapper = doc_mappers |
| 125 | + .get(leaf_req.doc_mapper_ord as usize) |
| 126 | + .ok_or_else(|| { |
| 127 | + LambdaError::Internal(format!( |
| 128 | + "doc_mapper_ord out of bounds: {}", |
| 129 | + leaf_req.doc_mapper_ord |
| 130 | + )) |
| 131 | + })? |
| 132 | + .clone(); |
| 133 | + let storage = storages[leaf_req.index_uri_ord as usize].clone(); |
| 134 | + |
| 135 | + for split_id_and_footer_offsets in leaf_req.split_offsets { |
| 136 | + let split_id = split_id_and_footer_offsets.split_id.clone(); |
| 137 | + let searcher_context = ctx.searcher_context.clone(); |
| 138 | + let search_request = search_request.clone(); |
| 139 | + let doc_mapper = doc_mapper.clone(); |
| 140 | + let storage = storage.clone(); |
| 141 | + let split = split_id_and_footer_offsets.clone(); |
| 142 | + split_search_joinset.spawn(async move { |
| 143 | + let result = single_doc_mapping_leaf_search( |
| 144 | + searcher_context, |
| 145 | + search_request, |
| 146 | + storage, |
| 147 | + vec![split], |
| 148 | + doc_mapper, |
| 149 | + ) |
| 150 | + .await |
| 151 | + .map_err(|err| format!("{err}")); |
| 152 | + (split_id, result) |
| 153 | + }); |
| 154 | + } |
102 | 155 | } |
103 | 156 |
|
| 157 | + let num_splits = split_search_joinset.len(); |
| 158 | + info!(num_splits, "processing leaf search request (per-split)"); |
| 159 | + |
104 | 160 | // Collect results. Order is irrelevant: each result is tagged with its split_id. |
105 | 161 | let mut split_results: Vec<LambdaSingleSplitResult> = Vec::with_capacity(num_splits); |
106 | 162 | let mut num_successes: usize = 0; |
@@ -140,9 +196,5 @@ pub async fn handle_leaf_search( |
140 | 196 | num_failures, "leaf search completed (per-split)" |
141 | 197 | ); |
142 | 198 |
|
143 | | - let wrapper = LambdaSearchResponses { split_results }; |
144 | | - let response_bytes = wrapper.encode_to_vec(); |
145 | | - let payload = BASE64_STANDARD.encode(&response_bytes); |
146 | | - |
147 | | - Ok(LambdaSearchResponsePayload { payload }) |
| 199 | + Ok(split_results) |
148 | 200 | } |
0 commit comments