datanode/heartbeat/handler/
apply_staging_manifest.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use common_meta::instruction::{
16    ApplyStagingManifest, ApplyStagingManifestReply, ApplyStagingManifestsReply, InstructionReply,
17};
18use common_telemetry::{error, warn};
19use futures::future::join_all;
20use store_api::region_request::{ApplyStagingManifestRequest, RegionRequest};
21
22use crate::heartbeat::handler::{HandlerContext, InstructionHandler};
23
24pub struct ApplyStagingManifestsHandler;
25
26#[async_trait::async_trait]
27impl InstructionHandler for ApplyStagingManifestsHandler {
28    type Instruction = Vec<ApplyStagingManifest>;
29    async fn handle(
30        &self,
31        ctx: &HandlerContext,
32        requests: Self::Instruction,
33    ) -> Option<InstructionReply> {
34        let results = join_all(
35            requests
36                .into_iter()
37                .map(|request| Self::handle_apply_staging_manifest(ctx, request)),
38        )
39        .await;
40        Some(InstructionReply::ApplyStagingManifests(
41            ApplyStagingManifestsReply::new(results),
42        ))
43    }
44}
45
46impl ApplyStagingManifestsHandler {
47    async fn handle_apply_staging_manifest(
48        ctx: &HandlerContext,
49        request: ApplyStagingManifest,
50    ) -> ApplyStagingManifestReply {
51        let ApplyStagingManifest {
52            region_id,
53            ref partition_expr,
54            central_region_id,
55            ref manifest_path,
56        } = request;
57        common_telemetry::info!(
58            "Datanode received apply staging manifest request, region_id: {}, central_region_id: {}, partition_expr: {}, manifest_path: {}",
59            region_id,
60            central_region_id,
61            partition_expr,
62            manifest_path
63        );
64        let Some(leader) = ctx.region_server.is_region_leader(region_id) else {
65            warn!("Region: {} is not found", region_id);
66            return ApplyStagingManifestReply {
67                region_id,
68                exists: false,
69                ready: false,
70                error: None,
71            };
72        };
73        if !leader {
74            warn!("Region: {} is not leader", region_id);
75            return ApplyStagingManifestReply {
76                region_id,
77                exists: true,
78                ready: false,
79                error: Some("Region is not leader".into()),
80            };
81        }
82
83        match ctx
84            .region_server
85            .handle_request(
86                region_id,
87                RegionRequest::ApplyStagingManifest(ApplyStagingManifestRequest {
88                    partition_expr: partition_expr.clone(),
89                    central_region_id,
90                    manifest_path: manifest_path.clone(),
91                }),
92            )
93            .await
94        {
95            Ok(_) => ApplyStagingManifestReply {
96                region_id,
97                exists: true,
98                ready: true,
99                error: None,
100            },
101            Err(err) => {
102                error!(err; "Failed to apply staging manifest, region_id: {}", region_id);
103                ApplyStagingManifestReply {
104                    region_id,
105                    exists: true,
106                    ready: false,
107                    error: Some(format!("{err:?}")),
108                }
109            }
110        }
111    }
112}
113
114#[cfg(test)]
115mod tests {
116    use std::collections::HashMap;
117    use std::sync::Arc;
118
119    use common_meta::instruction::RemapManifest;
120    use datatypes::value::Value;
121    use mito2::config::MitoConfig;
122    use mito2::engine::MITO_ENGINE_NAME;
123    use mito2::test_util::{CreateRequestBuilder, TestEnv};
124    use partition::expr::{PartitionExpr, col};
125    use store_api::path_utils::table_dir;
126    use store_api::region_engine::RegionRole;
127    use store_api::region_request::EnterStagingRequest;
128    use store_api::storage::RegionId;
129
130    use super::*;
131    use crate::heartbeat::handler::remap_manifest::RemapManifestHandler;
132    use crate::region_server::RegionServer;
133    use crate::tests::{MockRegionEngine, mock_region_server};
134
135    #[tokio::test]
136    async fn test_region_not_exist() {
137        let mut mock_region_server = mock_region_server();
138        let (mock_engine, _) = MockRegionEngine::new(MITO_ENGINE_NAME);
139        mock_region_server.register_engine(mock_engine);
140        let handler_context = HandlerContext::new_for_test(mock_region_server);
141        let region_id = RegionId::new(1024, 1);
142        let reply = ApplyStagingManifestsHandler
143            .handle(
144                &handler_context,
145                vec![ApplyStagingManifest {
146                    region_id,
147                    partition_expr: "".to_string(),
148                    central_region_id: RegionId::new(1024, 9999), // use a dummy value
149                    manifest_path: "".to_string(),
150                }],
151            )
152            .await
153            .unwrap();
154        let replies = reply.expect_apply_staging_manifests_reply();
155        let reply = &replies[0];
156        assert!(!reply.exists);
157        assert!(!reply.ready);
158        assert!(reply.error.is_none());
159    }
160
161    #[tokio::test]
162    async fn test_region_not_leader() {
163        let mock_region_server = mock_region_server();
164        let region_id = RegionId::new(1024, 1);
165        let (mock_engine, _) =
166            MockRegionEngine::with_custom_apply_fn(MITO_ENGINE_NAME, |region_engine| {
167                region_engine.mock_role = Some(Some(RegionRole::Follower));
168                region_engine.handle_request_mock_fn = Some(Box::new(|_, _| Ok(0)));
169            });
170        mock_region_server.register_test_region(region_id, mock_engine);
171        let handler_context = HandlerContext::new_for_test(mock_region_server);
172        let region_id = RegionId::new(1024, 1);
173        let reply = ApplyStagingManifestsHandler
174            .handle(
175                &handler_context,
176                vec![ApplyStagingManifest {
177                    region_id,
178                    partition_expr: "".to_string(),
179                    central_region_id: RegionId::new(1024, 2),
180                    manifest_path: "".to_string(),
181                }],
182            )
183            .await
184            .unwrap();
185        let replies = reply.expect_apply_staging_manifests_reply();
186        let reply = &replies[0];
187        assert!(reply.exists);
188        assert!(!reply.ready);
189        assert!(reply.error.is_some());
190    }
191
192    fn range_expr(col_name: &str, start: i64, end: i64) -> PartitionExpr {
193        col(col_name)
194            .gt_eq(Value::Int64(start))
195            .and(col(col_name).lt(Value::Int64(end)))
196    }
197
198    async fn prepare_region(region_server: &RegionServer) {
199        let region_specs = [
200            (RegionId::new(1024, 1), range_expr("x", 0, 49)),
201            (RegionId::new(1024, 2), range_expr("x", 49, 100)),
202        ];
203
204        for (region_id, partition_expr) in region_specs {
205            let builder = CreateRequestBuilder::new();
206            let mut create_req = builder.build();
207            create_req.table_dir = table_dir("test", 1024);
208            region_server
209                .handle_request(region_id, RegionRequest::Create(create_req))
210                .await
211                .unwrap();
212            region_server
213                .handle_request(
214                    region_id,
215                    RegionRequest::EnterStaging(EnterStagingRequest {
216                        partition_expr: partition_expr.as_json_str().unwrap(),
217                    }),
218                )
219                .await
220                .unwrap();
221        }
222    }
223
224    #[tokio::test]
225    async fn test_apply_staging_manifest() {
226        common_telemetry::init_default_ut_logging();
227        let mut region_server = mock_region_server();
228        let region_id = RegionId::new(1024, 1);
229        let mut engine_env = TestEnv::new().await;
230        let engine = engine_env.create_engine(MitoConfig::default()).await;
231        region_server.register_engine(Arc::new(engine.clone()));
232        prepare_region(&region_server).await;
233
234        let handler_context = HandlerContext::new_for_test(region_server);
235        let region_id2 = RegionId::new(1024, 2);
236        let reply = RemapManifestHandler
237            .handle(
238                &handler_context,
239                RemapManifest {
240                    region_id,
241                    input_regions: vec![region_id, region_id2],
242                    region_mapping: HashMap::from([
243                        // [0,49) <- [0, 50)
244                        (region_id, vec![region_id]),
245                        // [49, 100) <- [0, 50), [50,100)
246                        (region_id2, vec![region_id, region_id2]),
247                    ]),
248                    new_partition_exprs: HashMap::from([
249                        (region_id, range_expr("x", 0, 49).as_json_str().unwrap()),
250                        (region_id2, range_expr("x", 49, 100).as_json_str().unwrap()),
251                    ]),
252                },
253            )
254            .await
255            .unwrap();
256        let reply = reply.expect_remap_manifest_reply();
257        assert!(reply.exists);
258        assert!(reply.error.is_none(), "{}", reply.error.unwrap());
259        assert_eq!(reply.manifest_paths.len(), 2);
260        let manifest_path_1 = reply.manifest_paths[&region_id].clone();
261        let manifest_path_2 = reply.manifest_paths[&region_id2].clone();
262
263        let reply = ApplyStagingManifestsHandler
264            .handle(
265                &handler_context,
266                vec![ApplyStagingManifest {
267                    region_id,
268                    partition_expr: range_expr("x", 0, 49).as_json_str().unwrap(),
269                    central_region_id: region_id,
270                    manifest_path: manifest_path_1,
271                }],
272            )
273            .await
274            .unwrap();
275        let replies = reply.expect_apply_staging_manifests_reply();
276        let reply = &replies[0];
277        assert!(reply.exists);
278        assert!(reply.ready);
279        assert!(reply.error.is_none());
280
281        // partition expr mismatch
282        let reply = ApplyStagingManifestsHandler
283            .handle(
284                &handler_context,
285                vec![ApplyStagingManifest {
286                    region_id: region_id2,
287                    partition_expr: range_expr("x", 50, 100).as_json_str().unwrap(),
288                    central_region_id: region_id,
289                    manifest_path: manifest_path_2,
290                }],
291            )
292            .await
293            .unwrap();
294        let replies = reply.expect_apply_staging_manifests_reply();
295        let reply = &replies[0];
296        assert!(reply.exists);
297        assert!(!reply.ready);
298        assert!(reply.error.is_some());
299    }
300}