-
Notifications
You must be signed in to change notification settings - Fork 492
Expand file tree
/
Copy pathcompaction.rs
More file actions
2411 lines (2184 loc) · 86.4 KB
/
compaction.rs
File metadata and controls
2411 lines (2184 loc) · 86.4 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
mod buckets;
pub mod compactor;
pub mod memory_manager;
pub mod picker;
pub mod run;
mod task;
#[cfg(test)]
mod test_util;
mod twcs;
mod window;
use std::collections::HashMap;
use std::sync::{Arc, Mutex};
use std::time::Instant;
use api::v1::region::compact_request;
use api::v1::region::compact_request::Options;
use arrow_schema::Schema;
use common_base::Plugins;
use common_base::cancellation::CancellationHandle;
use common_memory_manager::OnExhaustedPolicy;
use common_meta::key::SchemaMetadataManagerRef;
use common_telemetry::{debug, error, info, warn};
use common_time::range::TimestampRange;
use common_time::timestamp::TimeUnit;
use common_time::{TimeToLive, Timestamp};
use datafusion_common::ScalarValue;
use datafusion_expr::Expr;
use datatypes::extension::json::is_json_extension_type;
use datatypes::schema::ext::ArrowSchemaExt;
use datatypes::types::json_type::JsonNativeType;
use parquet::arrow::parquet_to_arrow_schema;
use parquet::file::metadata::PageIndexPolicy;
use serde::{Deserialize, Serialize};
use snafu::{OptionExt, ResultExt};
use store_api::metadata::RegionMetadataRef;
use store_api::storage::RegionId;
use task::MAX_PARALLEL_COMPACTION;
use tokio::sync::mpsc::{self, Sender};
use crate::access_layer::AccessLayerRef;
use crate::cache::{CacheManagerRef, CacheStrategy};
use crate::compaction::compactor::{CompactionRegion, CompactionVersion, DefaultCompactor};
use crate::compaction::memory_manager::CompactionMemoryManager;
use crate::compaction::picker::{CompactionTask, PickerOutput, new_picker};
use crate::compaction::task::CompactionTaskImpl;
use crate::config::MitoConfig;
use crate::error::{
CompactRegionSnafu, CompactionCancelledSnafu, DataTypeMismatchSnafu, Error,
GetSchemaMetadataSnafu, ManualCompactionOverrideSnafu, ParquetToArrowSchemaSnafu,
RegionClosedSnafu, RegionDroppedSnafu, RegionTruncatedSnafu, RemoteCompactionSnafu, Result,
TimeRangePredicateOverflowSnafu, TimeoutSnafu,
};
use crate::metrics::{COMPACTION_STAGE_ELAPSED, INFLIGHT_COMPACTION_COUNT};
use crate::read::FlatSource;
use crate::read::flat_projection::FlatProjectionMapper;
use crate::read::read_columns::ReadColumns;
use crate::read::scan_region::{PredicateGroup, ScanInput};
use crate::read::seq_scan::SeqScan;
use crate::region::options::{MergeMode, RegionOptions};
use crate::region::version::VersionControlRef;
use crate::region::{ManifestContextRef, RegionLeaderState, RegionRoleState};
use crate::request::{OptionOutputTx, OutputTx, SenderDdlRequest, WorkerRequestWithTime};
use crate::schedule::remote_job_scheduler::{
CompactionJob, DefaultNotifier, RemoteJob, RemoteJobSchedulerRef,
};
use crate::schedule::scheduler::SchedulerRef;
use crate::sst::file::{FileHandle, FileMeta, Level};
use crate::sst::parquet::reader::MetadataCacheMetrics;
use crate::sst::version::LevelMeta;
use crate::worker::WorkerListener;
/// Region compaction request.
pub struct CompactionRequest {
pub(crate) engine_config: Arc<MitoConfig>,
pub(crate) current_version: CompactionVersion,
pub(crate) access_layer: AccessLayerRef,
/// Sender to send notification to the region worker.
pub(crate) request_sender: mpsc::Sender<WorkerRequestWithTime>,
/// Waiters of the compaction request.
pub(crate) waiters: Vec<OutputTx>,
/// Start time of compaction task.
pub(crate) start_time: Instant,
pub(crate) cache_manager: CacheManagerRef,
pub(crate) manifest_ctx: ManifestContextRef,
pub(crate) listener: WorkerListener,
pub(crate) schema_metadata_manager: SchemaMetadataManagerRef,
pub(crate) max_parallelism: usize,
}
impl CompactionRequest {
pub(crate) fn region_id(&self) -> RegionId {
self.current_version.metadata.region_id
}
}
/// Compaction scheduler tracks and manages compaction tasks.
pub(crate) struct CompactionScheduler {
scheduler: SchedulerRef,
/// Compacting regions.
region_status: HashMap<RegionId, CompactionStatus>,
/// Request sender of the worker that this scheduler belongs to.
request_sender: Sender<WorkerRequestWithTime>,
cache_manager: CacheManagerRef,
engine_config: Arc<MitoConfig>,
memory_manager: Arc<CompactionMemoryManager>,
memory_policy: OnExhaustedPolicy,
listener: WorkerListener,
/// Plugins for the compaction scheduler.
plugins: Plugins,
}
impl CompactionScheduler {
#[allow(clippy::too_many_arguments)]
pub(crate) fn new(
scheduler: SchedulerRef,
request_sender: Sender<WorkerRequestWithTime>,
cache_manager: CacheManagerRef,
engine_config: Arc<MitoConfig>,
listener: WorkerListener,
plugins: Plugins,
memory_manager: Arc<CompactionMemoryManager>,
memory_policy: OnExhaustedPolicy,
) -> Self {
Self {
scheduler,
region_status: HashMap::new(),
request_sender,
cache_manager,
engine_config,
memory_manager,
memory_policy,
listener,
plugins,
}
}
/// Schedules a compaction for the region.
#[allow(clippy::too_many_arguments)]
pub(crate) async fn schedule_compaction(
&mut self,
region_id: RegionId,
compact_options: compact_request::Options,
version_control: &VersionControlRef,
access_layer: &AccessLayerRef,
waiter: OptionOutputTx,
manifest_ctx: &ManifestContextRef,
schema_metadata_manager: SchemaMetadataManagerRef,
max_parallelism: usize,
) -> Result<()> {
// skip compaction if region is in staging state
let current_state = manifest_ctx.current_state();
if current_state == RegionRoleState::Leader(RegionLeaderState::Staging) {
info!(
"Skipping compaction for region {} in staging mode, options: {:?}",
region_id, compact_options
);
waiter.send(Ok(0));
return Ok(());
}
if let Some(status) = self.region_status.get_mut(®ion_id) {
match compact_options {
Options::Regular(_) => {
// Region is compacting. Add the waiter to pending list.
status.merge_waiter(waiter);
}
options @ Options::StrictWindow(_) => {
// Incoming compaction request is manually triggered.
status.set_pending_request(PendingCompaction {
options,
waiter,
max_parallelism,
});
info!(
"Region {} is compacting, manually compaction will be re-scheduled.",
region_id
);
}
}
return Ok(());
}
// The region can compact directly.
let mut status =
CompactionStatus::new(region_id, version_control.clone(), access_layer.clone());
let request = status.new_compaction_request(
self.request_sender.clone(),
waiter,
self.engine_config.clone(),
self.cache_manager.clone(),
manifest_ctx,
self.listener.clone(),
schema_metadata_manager,
max_parallelism,
);
let result = match self
.schedule_compaction_request(request, compact_options)
.await
{
Ok(Some(active_compaction)) => {
// Publish CompactionStatus only after a task has been accepted by the scheduler.
// This avoids exposing a half-initialized region status that could collect pending
// DDL/compaction state even though no compaction is actually running.
status.active_compaction = Some(active_compaction);
self.region_status.insert(region_id, status);
Ok(())
}
Ok(None) => Ok(()),
Err(e) => Err(e),
};
self.listener.on_compaction_scheduled(region_id);
result
}
// Handle pending manual compaction request for the region.
//
// Returns true if should early return, false otherwise.
pub(crate) async fn handle_pending_compaction_request(
&mut self,
region_id: RegionId,
manifest_ctx: &ManifestContextRef,
schema_metadata_manager: SchemaMetadataManagerRef,
) -> bool {
let Some(status) = self.region_status.get_mut(®ion_id) else {
return true;
};
// If there is a pending manual compaction request, schedule it.
// and defer returning the pending DDL requests to the caller.
let Some(pending_request) = std::mem::take(&mut status.pending_request) else {
return false;
};
let PendingCompaction {
options,
waiter,
max_parallelism,
} = pending_request;
let request = {
status.new_compaction_request(
self.request_sender.clone(),
waiter,
self.engine_config.clone(),
self.cache_manager.clone(),
manifest_ctx,
self.listener.clone(),
schema_metadata_manager,
max_parallelism,
)
};
match self.schedule_compaction_request(request, options).await {
Ok(Some(active_compaction)) => {
let status = self.region_status.get_mut(®ion_id).unwrap();
status.active_compaction = Some(active_compaction);
debug!(
"Successfully scheduled manual compaction for region id: {}",
region_id
);
true
}
Ok(None) => {
// We still need to handle the pending DDL requests.
// So we can't return early here.
false
}
Err(e) => {
error!(e; "Failed to continue pending manual compaction for region id: {}", region_id);
self.remove_region_on_failure(region_id, Arc::new(e));
true
}
}
}
/// Notifies the scheduler that the compaction job is finished successfully.
pub(crate) async fn on_compaction_finished(
&mut self,
region_id: RegionId,
manifest_ctx: &ManifestContextRef,
schema_metadata_manager: SchemaMetadataManagerRef,
) -> Vec<SenderDdlRequest> {
let Some(status) = self.region_status.get_mut(®ion_id) else {
return Vec::new();
};
status.clear_running_task();
// If there a pending compaction request, handle it first
// and defer returning the pending DDL requests to the caller.
if self
.handle_pending_compaction_request(
region_id,
manifest_ctx,
schema_metadata_manager.clone(),
)
.await
{
return Vec::new();
}
let Some(status) = self.region_status.get_mut(®ion_id) else {
// The region status might be removed by the previous steps.
// So we return empty DDL requests.
return Vec::new();
};
for waiter in std::mem::take(&mut status.waiters) {
waiter.send(Ok(0));
}
// If there are pending DDL requests, run them.
let pending_ddl_requests = std::mem::take(&mut status.pending_ddl_requests);
if !pending_ddl_requests.is_empty() {
self.region_status.remove(®ion_id);
// If there are pending DDL requests, we should return them to the caller.
// And skip try to schedule next compaction task.
return pending_ddl_requests;
}
// We should always try to compact the region until picker returns None.
let request = status.new_compaction_request(
self.request_sender.clone(),
OptionOutputTx::none(),
self.engine_config.clone(),
self.cache_manager.clone(),
manifest_ctx,
self.listener.clone(),
schema_metadata_manager,
MAX_PARALLEL_COMPACTION,
);
// Try to schedule next compaction task for this region.
match self
.schedule_compaction_request(
request,
compact_request::Options::Regular(Default::default()),
)
.await
{
Ok(Some(active_compaction)) => {
self.region_status
.get_mut(®ion_id)
.unwrap()
.active_compaction = Some(active_compaction);
debug!(
"Successfully scheduled next compaction for region id: {}",
region_id
);
}
Ok(None) => {
// No further compaction tasks can be scheduled; cleanup the `CompactionStatus` for this region.
// All DDL requests and pending compaction requests have already been processed.
// Safe to remove the region from status tracking.
self.region_status.remove(®ion_id);
}
Err(e) => {
error!(e; "Failed to schedule next compaction for region {}", region_id);
self.remove_region_on_failure(region_id, Arc::new(e));
}
}
Vec::new()
}
/// Notifies the scheduler that the compaction job is cancelled cooperatively.
pub(crate) async fn on_compaction_cancelled(
&mut self,
region_id: RegionId,
) -> Vec<SenderDdlRequest> {
self.remove_region_on_cancel(region_id)
}
/// Notifies the scheduler that the compaction job is failed.
pub(crate) fn on_compaction_failed(&mut self, region_id: RegionId, err: Arc<Error>) {
error!(err; "Region {} failed to compact, cancel all pending tasks", region_id);
self.remove_region_on_failure(region_id, err);
}
/// Notifies the scheduler that the region is dropped.
pub(crate) fn on_region_dropped(&mut self, region_id: RegionId) {
self.remove_region_on_failure(
region_id,
Arc::new(RegionDroppedSnafu { region_id }.build()),
);
}
/// Notifies the scheduler that the region is closed.
pub(crate) fn on_region_closed(&mut self, region_id: RegionId) {
self.remove_region_on_failure(region_id, Arc::new(RegionClosedSnafu { region_id }.build()));
}
/// Notifies the scheduler that the region is truncated.
pub(crate) fn on_region_truncated(&mut self, region_id: RegionId) {
self.remove_region_on_failure(
region_id,
Arc::new(RegionTruncatedSnafu { region_id }.build()),
);
}
/// Add ddl request to pending queue.
///
/// # Panics
/// Panics if region didn't request compaction.
pub(crate) fn add_ddl_request_to_pending(&mut self, request: SenderDdlRequest) {
debug!(
"Added pending DDL request for region: {}, ddl: {:?}",
request.region_id, request.request
);
let status = self.region_status.get_mut(&request.region_id).unwrap();
status.pending_ddl_requests.push(request);
}
#[cfg(test)]
pub(crate) fn has_pending_ddls(&self, region_id: RegionId) -> bool {
let has_pending = self
.region_status
.get(®ion_id)
.map(|status| !status.pending_ddl_requests.is_empty())
.unwrap_or(false);
debug!(
"Checked pending DDL requests for region: {}, has_pending: {}",
region_id, has_pending
);
has_pending
}
pub(crate) fn request_cancel(&mut self, region_id: RegionId) -> RequestCancelResult {
let Some(status) = self.region_status.get_mut(®ion_id) else {
return RequestCancelResult::NotRunning;
};
status.request_cancel()
}
/// Schedules a compaction request.
///
/// Returns the active compaction state if the request is scheduled successfully.
/// Returns `None` if no compaction task can be scheduled for this region.
async fn schedule_compaction_request(
&mut self,
request: CompactionRequest,
options: compact_request::Options,
) -> Result<Option<ActiveCompaction>> {
let region_id = request.region_id();
let (dynamic_compaction_opts, ttl) = find_dynamic_options(
region_id,
&request.current_version.options,
&request.schema_metadata_manager,
)
.await
.unwrap_or_else(|e| {
warn!(e; "Failed to find dynamic options for region: {}", region_id);
(
request.current_version.options.compaction.clone(),
request.current_version.options.ttl.unwrap_or_default(),
)
});
let picker = new_picker(
&options,
&dynamic_compaction_opts,
request.current_version.options.append_mode,
Some(self.engine_config.max_background_compactions),
);
let region_id = request.region_id();
let CompactionRequest {
engine_config,
current_version,
access_layer,
request_sender,
waiters,
start_time,
cache_manager,
manifest_ctx,
listener,
schema_metadata_manager: _,
max_parallelism,
} = request;
debug!(
"Pick compaction strategy {:?} for region: {}, ttl: {:?}",
picker, region_id, ttl
);
let compaction_region = CompactionRegion {
region_id,
current_version: current_version.clone(),
region_options: RegionOptions {
compaction: dynamic_compaction_opts.clone(),
..current_version.options.clone()
},
engine_config: engine_config.clone(),
region_metadata: current_version.metadata.clone(),
cache_manager: cache_manager.clone(),
access_layer: access_layer.clone(),
manifest_ctx: manifest_ctx.clone(),
file_purger: None,
ttl: Some(ttl),
max_parallelism,
plugins: self.plugins.clone(),
};
let picker_output = {
let _pick_timer = COMPACTION_STAGE_ELAPSED
.with_label_values(&["pick"])
.start_timer();
picker.pick(&compaction_region)
};
let picker_output = if let Some(picker_output) = picker_output {
picker_output
} else {
// Nothing to compact, we are done. Notifies all waiters as we consume the compaction request.
for waiter in waiters {
waiter.send(Ok(0));
}
return Ok(None);
};
// If specified to run compaction remotely, we schedule the compaction job remotely.
// It will fall back to local compaction if there is no remote job scheduler.
let waiters = if dynamic_compaction_opts.remote_compaction() {
if let Some(remote_job_scheduler) = &self.plugins.get::<RemoteJobSchedulerRef>() {
let remote_compaction_job = CompactionJob {
compaction_region: compaction_region.clone(),
picker_output: picker_output.clone(),
start_time,
waiters,
ttl,
};
let result = remote_job_scheduler
.schedule(
RemoteJob::CompactionJob(remote_compaction_job),
Box::new(DefaultNotifier {
request_sender: request_sender.clone(),
}),
)
.await;
match result {
Ok(job_id) => {
info!(
"Scheduled remote compaction job {} for region {}",
job_id, region_id
);
INFLIGHT_COMPACTION_COUNT.inc();
return Ok(Some(ActiveCompaction::Remote));
}
Err(e) => {
if !dynamic_compaction_opts.fallback_to_local() {
error!(e; "Failed to schedule remote compaction job for region {}", region_id);
return RemoteCompactionSnafu {
region_id,
job_id: None,
reason: e.reason,
}
.fail();
}
error!(e; "Failed to schedule remote compaction job for region {}, fallback to local compaction", region_id);
// Return the waiters back to the caller for local compaction.
e.waiters
}
}
} else {
debug!(
"Remote compaction is not enabled, fallback to local compaction for region {}",
region_id
);
waiters
}
} else {
waiters
};
// Create a local compaction task.
let estimated_bytes = estimate_compaction_bytes(&picker_output);
let cancel_handle = Arc::new(CancellationHandle::default());
let state = LocalCompactionState::new(cancel_handle.clone());
let local_compaction_task = Box::new(CompactionTaskImpl {
state: state.clone(),
request_sender,
waiters,
start_time,
listener,
picker_output,
compaction_region,
compactor: Arc::new(DefaultCompactor::with_cancel_handle(cancel_handle.clone())),
memory_manager: self.memory_manager.clone(),
memory_policy: self.memory_policy,
estimated_memory_bytes: estimated_bytes,
});
self.submit_compaction_task(local_compaction_task, region_id)
.map(|_| Some(ActiveCompaction::Local { state }))
}
fn submit_compaction_task(
&mut self,
mut task: Box<CompactionTaskImpl>,
region_id: RegionId,
) -> Result<()> {
self.scheduler
.schedule(Box::pin(async move {
INFLIGHT_COMPACTION_COUNT.inc();
task.run().await;
INFLIGHT_COMPACTION_COUNT.dec();
}))
.inspect_err(
|e| error!(e; "Failed to submit compaction request for region {}", region_id),
)
}
fn remove_region_on_failure(&mut self, region_id: RegionId, err: Arc<Error>) {
// Remove this region.
let Some(status) = self.region_status.remove(®ion_id) else {
return;
};
// Notifies all pending tasks.
status.on_failure(err);
}
fn remove_region_on_cancel(&mut self, region_id: RegionId) -> Vec<SenderDdlRequest> {
let Some(status) = self.region_status.remove(®ion_id) else {
return Vec::new();
};
status.on_cancel()
}
}
#[derive(Debug, Clone)]
pub(crate) struct LocalCompactionState {
cancel_handle: Arc<CancellationHandle>,
commit_started: Arc<Mutex<bool>>,
}
#[derive(Debug)]
enum ActiveCompaction {
Local { state: LocalCompactionState },
Remote,
}
impl LocalCompactionState {
fn new(cancel_handle: Arc<CancellationHandle>) -> Self {
Self {
cancel_handle,
commit_started: Arc::new(Mutex::new(false)),
}
}
/// Returns the cancellation handle for this compaction task.
pub(crate) fn cancel_handle(&self) -> Arc<CancellationHandle> {
self.cancel_handle.clone()
}
/// Marks the compaction task as started to commit,
/// which means the compaction task is in the final stage and is about to update region version and manifest.
/// It will reject cancellation request after this method is called.
///
/// Returns true if this is the first time to mark commit started, false otherwise.
pub(crate) fn mark_commit_started(&self) -> bool {
let mut commit_started = self.commit_started.lock().unwrap();
if self.cancel_handle.is_cancelled() {
return false;
}
*commit_started = true;
true
}
/// Request cancellation for this compaction task.
pub(crate) fn request_cancel(&self) -> RequestCancelResult {
// The cancel handle must under the lock of `commit_started` to avoid racing between cancellation and commit.
let commit_started = self.commit_started.lock().unwrap();
if *commit_started {
return RequestCancelResult::TooLateToCancel;
}
if self.cancel_handle.is_cancelled() {
return RequestCancelResult::AlreadyCancelling;
}
self.cancel_handle.cancel();
RequestCancelResult::CancelIssued
}
}
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub(crate) enum RequestCancelResult {
CancelIssued,
AlreadyCancelling,
TooLateToCancel,
NotRunning,
}
impl Drop for CompactionScheduler {
fn drop(&mut self) {
for (region_id, status) in self.region_status.drain() {
// We are shutting down so notify all pending tasks.
status.on_failure(Arc::new(RegionClosedSnafu { region_id }.build()));
}
}
}
/// Finds compaction options and TTL together with a single metadata fetch to reduce RTT.
async fn find_dynamic_options(
region_id: RegionId,
region_options: &crate::region::options::RegionOptions,
schema_metadata_manager: &SchemaMetadataManagerRef,
) -> Result<(crate::region::options::CompactionOptions, TimeToLive)> {
let table_id = region_id.table_id();
if let (true, Some(ttl)) = (region_options.compaction_override, region_options.ttl) {
debug!(
"Use region options directly for table {}: compaction={:?}, ttl={:?}",
table_id, region_options.compaction, region_options.ttl
);
return Ok((region_options.compaction.clone(), ttl));
}
let db_options = tokio::time::timeout(
crate::config::FETCH_OPTION_TIMEOUT,
schema_metadata_manager.get_schema_options_by_table_id(table_id),
)
.await
.context(TimeoutSnafu)?
.context(GetSchemaMetadataSnafu)?;
let ttl = if let Some(ttl) = region_options.ttl {
debug!(
"Use region TTL directly for table {}: ttl={:?}",
table_id, region_options.ttl
);
ttl
} else {
db_options
.as_ref()
.and_then(|options| options.ttl)
.unwrap_or_default()
.into()
};
let compaction = if !region_options.compaction_override {
if let Some(schema_opts) = db_options {
let map: HashMap<String, String> = schema_opts
.extra_options
.iter()
.filter_map(|(k, v)| {
if k.starts_with("compaction.") {
Some((k.clone(), v.clone()))
} else {
None
}
})
.collect();
if map.is_empty() {
region_options.compaction.clone()
} else {
crate::region::options::RegionOptions::try_from_options(region_id, &map)
.map(|o| o.compaction)
.unwrap_or_else(|e| {
error!(e; "Failed to create RegionOptions from map");
region_options.compaction.clone()
})
}
} else {
debug!(
"DB options is None for table {}, use region compaction: compaction={:?}",
table_id, region_options.compaction
);
region_options.compaction.clone()
}
} else {
debug!(
"No schema options for table {}, use region compaction: compaction={:?}",
table_id, region_options.compaction
);
region_options.compaction.clone()
};
debug!(
"Resolved dynamic options for table {}: compaction={:?}, ttl={:?}",
table_id, compaction, ttl
);
Ok((compaction, ttl))
}
/// Status of running and pending region compaction tasks.
struct CompactionStatus {
/// Id of the region.
region_id: RegionId,
/// Version control of the region.
version_control: VersionControlRef,
/// Access layer of the region.
access_layer: AccessLayerRef,
/// Pending waiters for compaction.
waiters: Vec<OutputTx>,
/// Pending compactions that are supposed to run as soon as current compaction task finished.
pending_request: Option<PendingCompaction>,
/// Pending DDL requests that should run when compaction is done.
pending_ddl_requests: Vec<SenderDdlRequest>,
/// Active compaction state.
active_compaction: Option<ActiveCompaction>,
}
impl CompactionStatus {
/// Creates a new [CompactionStatus]
fn new(
region_id: RegionId,
version_control: VersionControlRef,
access_layer: AccessLayerRef,
) -> CompactionStatus {
CompactionStatus {
region_id,
version_control,
access_layer,
waiters: Vec::new(),
pending_request: None,
pending_ddl_requests: Vec::new(),
active_compaction: None,
}
}
#[cfg(test)]
fn start_local_task(&mut self) -> LocalCompactionState {
let state = LocalCompactionState::new(Arc::new(CancellationHandle::default()));
self.active_compaction = Some(ActiveCompaction::Local {
state: state.clone(),
});
state
}
#[cfg(test)]
fn start_remote_task(&mut self) {
self.active_compaction = Some(ActiveCompaction::Remote);
}
fn request_cancel(&mut self) -> RequestCancelResult {
let Some(active_compaction) = &self.active_compaction else {
return RequestCancelResult::NotRunning;
};
match active_compaction {
ActiveCompaction::Local { state, .. } => state.request_cancel(),
ActiveCompaction::Remote => RequestCancelResult::TooLateToCancel,
}
}
fn clear_running_task(&mut self) -> bool {
self.active_compaction.take().is_some()
}
/// Merge the waiter to the pending compaction.
fn merge_waiter(&mut self, mut waiter: OptionOutputTx) {
if let Some(waiter) = waiter.take_inner() {
self.waiters.push(waiter);
}
}
/// Set pending compaction request or replace current value if already exist.
fn set_pending_request(&mut self, pending: PendingCompaction) {
if let Some(prev) = self.pending_request.replace(pending) {
debug!(
"Replace pending compaction options with new request {:?} for region: {}",
prev.options, self.region_id
);
prev.waiter.send(ManualCompactionOverrideSnafu.fail());
}
}
fn on_failure(mut self, err: Arc<Error>) {
for waiter in self.waiters.drain(..) {
waiter.send(Err(err.clone()).context(CompactRegionSnafu {
region_id: self.region_id,
}));
}
if let Some(pending_compaction) = self.pending_request {
pending_compaction
.waiter
.send(Err(err.clone()).context(CompactRegionSnafu {
region_id: self.region_id,
}));
}
for pending_ddl in self.pending_ddl_requests {
pending_ddl
.sender
.send(Err(err.clone()).context(CompactRegionSnafu {
region_id: self.region_id,
}));
}
}
#[must_use]
fn on_cancel(mut self) -> Vec<SenderDdlRequest> {
for waiter in self.waiters.drain(..) {
waiter.send(CompactionCancelledSnafu.fail());
}
if let Some(pending_compaction) = self.pending_request {
pending_compaction.waiter.send(
Err(Arc::new(CompactionCancelledSnafu.build())).context(CompactRegionSnafu {
region_id: self.region_id,
}),
);
}
std::mem::take(&mut self.pending_ddl_requests)
}
/// Creates a new compaction request for compaction picker.
///
/// It consumes all pending compaction waiters.
#[allow(clippy::too_many_arguments)]
fn new_compaction_request(
&mut self,
request_sender: Sender<WorkerRequestWithTime>,
mut waiter: OptionOutputTx,
engine_config: Arc<MitoConfig>,
cache_manager: CacheManagerRef,
manifest_ctx: &ManifestContextRef,
listener: WorkerListener,
schema_metadata_manager: SchemaMetadataManagerRef,
max_parallelism: usize,
) -> CompactionRequest {
let current_version = CompactionVersion::from(self.version_control.current().version);
let start_time = Instant::now();
let mut waiters = Vec::with_capacity(self.waiters.len() + 1);
waiters.extend(std::mem::take(&mut self.waiters));
if let Some(waiter) = waiter.take_inner() {
waiters.push(waiter);
}
CompactionRequest {
engine_config,
current_version,
access_layer: self.access_layer.clone(),
request_sender: request_sender.clone(),
waiters,
start_time,
cache_manager,
manifest_ctx: manifest_ctx.clone(),
listener,
schema_metadata_manager,
max_parallelism,
}
}
}
#[derive(Debug, Clone)]
pub struct CompactionOutput {
/// Compaction output file level.
pub output_level: Level,
/// Compaction input files.
pub inputs: Vec<FileHandle>,
/// Whether to remove deletion markers.
pub filter_deleted: bool,
/// Compaction output time range. Only windowed compaction specifies output time range.
pub output_time_range: Option<TimestampRange>,
}
/// SerializedCompactionOutput is a serialized version of [CompactionOutput] by replacing [FileHandle] with [FileMeta].
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct SerializedCompactionOutput {
output_level: Level,
inputs: Vec<FileMeta>,
filter_deleted: bool,
output_time_range: Option<TimestampRange>,
}
/// Builders to create [BoxedRecordBatchStream] for compaction.
struct CompactionSstReaderBuilder<'a> {
metadata: RegionMetadataRef,
sst_layer: AccessLayerRef,
cache: CacheManagerRef,
inputs: &'a [FileHandle],
append_mode: bool,
filter_deleted: bool,