-
Notifications
You must be signed in to change notification settings - Fork 3.5k
/
segment_state.rs
700 lines (621 loc) · 22.9 KB
/
segment_state.rs
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
//! State for the write buffer segments.
use crate::catalog::{Catalog, DatabaseSchema};
use crate::chunk::BufferChunk;
use crate::wal::WalSegmentWriterNoopImpl;
use crate::write_buffer::buffer_segment::{ClosedBufferSegment, OpenBufferSegment, WriteBatch};
use crate::{
persister, wal, write_buffer, ParquetFile, PersistedSegment, Persister, SegmentDuration,
SegmentId, SegmentRange, SequenceNumber, Wal, WalOp,
};
use arrow::datatypes::SchemaRef;
#[cfg(test)]
use arrow::record_batch::RecordBatch;
use data_types::{ChunkId, ChunkOrder, TableId, TransitionPartitionId};
use datafusion::common::DataFusionError;
use datafusion::execution::context::SessionState;
use datafusion::logical_expr::Expr;
use iox_query::chunk_statistics::create_chunk_statistics;
use iox_query::QueryChunk;
use iox_time::{Time, TimeProvider};
use observability_deps::tracing::error;
use parking_lot::RwLock;
#[cfg(test)]
use schema::Schema;
use std::collections::BTreeMap;
use std::sync::Arc;
use std::time::Duration;
use tokio::sync::watch;
// The maximum number of open segments that can be open at any one time. Each one of these will
// have an open wal file and a buffer segment in memory.
const OPEN_SEGMENT_LIMIT: usize = 100;
#[derive(Debug)]
pub(crate) struct SegmentState<T, W> {
segment_duration: SegmentDuration,
last_segment_id: SegmentId,
catalog: Arc<Catalog>,
wal: Option<Arc<W>>,
time_provider: Arc<T>,
// Map of segment start times to open segments. Should always have a segment open for the
// start time that time.now falls into.
segments: BTreeMap<Time, OpenBufferSegment>,
persisting_segments: BTreeMap<Time, Arc<ClosedBufferSegment>>,
persisted_segments: BTreeMap<Time, Arc<PersistedSegment>>,
}
impl<T: TimeProvider, W: Wal> SegmentState<T, W> {
#[allow(clippy::too_many_arguments)]
pub(crate) fn new(
segment_duration: SegmentDuration,
last_segment_id: SegmentId,
catalog: Arc<Catalog>,
time_provider: Arc<T>,
open_segments: Vec<OpenBufferSegment>,
persisting_segments: Vec<ClosedBufferSegment>,
persisted_segments: Vec<PersistedSegment>,
wal: Option<Arc<W>>,
) -> Self {
let mut segments = BTreeMap::new();
for segment in open_segments {
segments.insert(segment.segment_range().start_time, segment);
}
let mut persisting_segments_map = BTreeMap::new();
for segment in persisting_segments {
persisting_segments_map.insert(segment.segment_range.start_time, Arc::new(segment));
}
let mut persisted_segments_map = BTreeMap::new();
for segment in persisted_segments {
persisted_segments_map.insert(
Time::from_timestamp_nanos(segment.segment_min_time),
Arc::new(segment),
);
}
Self {
segment_duration,
last_segment_id,
catalog,
time_provider,
wal,
segments,
persisting_segments: persisting_segments_map,
persisted_segments: persisted_segments_map,
}
}
pub(crate) fn write_ops_to_segment(
&mut self,
segment_start: Time,
ops: Vec<WalOp>,
starting_catalog_sequence_number: SequenceNumber,
) -> wal::Result<()> {
let segment =
self.get_or_create_segment_for_time(segment_start, starting_catalog_sequence_number)?;
segment.write_wal_ops(ops)
}
pub(crate) fn write_batch_to_segment(
&mut self,
segment_start: Time,
write_batch: WriteBatch,
starting_catalog_sequence_number: SequenceNumber,
) -> crate::write_buffer::Result<()> {
let segment =
self.get_or_create_segment_for_time(segment_start, starting_catalog_sequence_number)?;
segment.buffer_writes(write_batch)
}
pub(crate) fn get_table_chunks(
&self,
db_schema: Arc<DatabaseSchema>,
table_name: &str,
filters: &[Expr],
projection: Option<&Vec<usize>>,
_ctx: &SessionState,
) -> Result<Vec<Arc<dyn QueryChunk>>, DataFusionError> {
let table = db_schema
.tables
.get(table_name)
.ok_or_else(|| DataFusionError::Execution(format!("table {} not found", table_name)))?;
let arrow_schema: SchemaRef = match projection {
Some(projection) => Arc::new(table.schema.as_arrow().project(projection).unwrap()),
None => table.schema.as_arrow(),
};
let schema = schema::Schema::try_from(Arc::clone(&arrow_schema))
.map_err(|e| DataFusionError::Execution(format!("schema error {}", e)))?;
let mut chunks: Vec<Arc<dyn QueryChunk>> = vec![];
for segment in self.segments.values() {
if let Some(batch) = segment.table_record_batch(
&db_schema.name,
table_name,
Arc::clone(&arrow_schema),
filters,
) {
let batch = batch.map_err(|e| {
DataFusionError::Execution(format!("error getting batches {}", e))
})?;
let row_count = batch.num_rows();
let chunk_stats = create_chunk_statistics(
Some(row_count),
&schema,
Some(segment.segment_range().timestamp_min_max()),
None,
);
chunks.push(Arc::new(BufferChunk {
batches: vec![batch],
schema: schema.clone(),
stats: Arc::new(chunk_stats),
partition_id: TransitionPartitionId::new(
TableId::new(0),
segment.segment_key(),
),
sort_key: None,
id: ChunkId::new(),
chunk_order: ChunkOrder::new(
chunks
.len()
.try_into()
.expect("should never have this many chunks"),
),
}));
}
}
for persisting_segment in self.persisting_segments.values() {
if let Some(batch) = persisting_segment.buffered_data.table_record_batches(
&db_schema.name,
table_name,
Arc::clone(&arrow_schema),
filters,
) {
let batch = batch.map_err(|e| {
DataFusionError::Execution(format!("error getting batches {}", e))
})?;
let row_count = batch.num_rows();
let chunk_stats = create_chunk_statistics(
Some(row_count),
&schema,
Some(persisting_segment.segment_range.timestamp_min_max()),
None,
);
chunks.push(Arc::new(BufferChunk {
batches: vec![batch],
schema: schema.clone(),
stats: Arc::new(chunk_stats),
partition_id: TransitionPartitionId::new(
TableId::new(0),
&persisting_segment.segment_key,
),
sort_key: None,
id: ChunkId::new(),
chunk_order: ChunkOrder::new(
chunks
.len()
.try_into()
.expect("should never have this many chunks"),
),
}));
}
}
Ok(chunks)
}
pub(crate) fn get_parquet_files(
&self,
database_name: &str,
table_name: &str,
) -> Vec<ParquetFile> {
let mut parquet_files = vec![];
for segment in self.persisted_segments.values() {
segment.databases.get(database_name).map(|db| {
db.tables.get(table_name).map(|table| {
parquet_files.extend(table.parquet_files.clone());
})
});
}
parquet_files
}
#[cfg(test)]
pub(crate) fn persisted_segments(&self) -> Vec<Arc<PersistedSegment>> {
self.persisted_segments.values().cloned().collect()
}
#[cfg(test)]
pub(crate) fn open_segment_times(&self) -> Vec<Time> {
self.segments.keys().cloned().collect()
}
#[cfg(test)]
pub(crate) fn open_segments_table_record_batches(
&self,
db_name: &str,
table_name: &str,
schema: &Schema,
) -> Vec<RecordBatch> {
self.segments
.values()
.map(|segment| {
segment
.table_record_batch(db_name, table_name, schema.as_arrow(), &[])
.unwrap()
.unwrap()
})
.collect()
}
#[allow(dead_code)]
pub(crate) fn segment_for_time(&self, time: Time) -> Option<&OpenBufferSegment> {
self.segments.get(&time)
}
// Looks at the open buffer segments and returns the start `Time` of any that meet the following
// criteria (in time ascending order):
// 1. The segment is not in the current time or next time
// 2. The segment has been open for longer than half the segment duration
fn segments_to_persist(&self, current_time: Time) -> Vec<Time> {
let mut segments_to_persist = vec![];
for (start_time, segment) in &self.segments {
if segment.should_persist(current_time) {
segments_to_persist.push(*start_time);
}
}
segments_to_persist.sort();
segments_to_persist
}
fn close_segment(&mut self, segment_start: Time) -> Option<Arc<ClosedBufferSegment>> {
self.segments.remove(&segment_start).map(|segment| {
let closed_segment = Arc::new(segment.into_closed_segment(Arc::clone(&self.catalog)));
self.persisting_segments
.insert(segment_start, Arc::clone(&closed_segment));
closed_segment
})
}
// return the segment with this start time or open up a new one if it isn't currently open.
fn get_or_create_segment_for_time(
&mut self,
time: Time,
starting_catalog_sequence_number: SequenceNumber,
) -> wal::Result<&mut OpenBufferSegment> {
if !self.segments.contains_key(&time) {
if self.segments.len() >= OPEN_SEGMENT_LIMIT {
return Err(wal::Error::OpenSegmentLimitReached(OPEN_SEGMENT_LIMIT));
}
self.last_segment_id = self.last_segment_id.next();
let segment_id = self.last_segment_id;
let segment_range =
SegmentRange::from_time_and_duration(time, self.segment_duration, false);
let segment_writer = match &self.wal {
Some(wal) => wal.new_segment_writer(segment_id, segment_range)?,
None => Box::new(WalSegmentWriterNoopImpl::new(segment_id)),
};
let segment = OpenBufferSegment::new(
Arc::clone(&self.catalog),
segment_id,
segment_range,
self.time_provider.now(),
starting_catalog_sequence_number,
segment_writer,
None,
);
self.segments.insert(time, segment);
}
Ok(self.segments.get_mut(&time).unwrap())
}
}
#[cfg(test)]
const PERSISTER_CHECK_INTERVAL: Duration = Duration::from_millis(10);
#[cfg(not(test))]
const PERSISTER_CHECK_INTERVAL: Duration = Duration::from_secs(1);
pub(crate) async fn run_buffer_segment_persist_and_cleanup<P, T, W>(
persister: Arc<P>,
segment_state: Arc<RwLock<SegmentState<T, W>>>,
mut shutdown_rx: watch::Receiver<()>,
time_provider: Arc<T>,
wal: Option<Arc<W>>,
executor: Arc<iox_query::exec::Executor>,
) where
P: Persister,
persister::Error: From<<P as Persister>::Error>,
T: TimeProvider,
W: Wal,
write_buffer::Error: From<<P as Persister>::Error>,
{
loop {
tokio::select! {
_ = shutdown_rx.changed() => {
break;
}
_ = tokio::time::sleep(PERSISTER_CHECK_INTERVAL) => {
if let Err(e) = persist_and_cleanup_ready_segments(Arc::clone(&persister), Arc::clone(&segment_state), Arc::clone(&time_provider), wal.clone(), Arc::clone(&executor)).await {
error!("Error persisting and cleaning up segments: {}", e);
}
}
}
}
}
async fn persist_and_cleanup_ready_segments<P, T, W>(
persister: Arc<P>,
segment_state: Arc<RwLock<SegmentState<T, W>>>,
time_provider: Arc<T>,
wal: Option<Arc<W>>,
executor: Arc<iox_query::exec::Executor>,
) -> Result<(), crate::Error>
where
P: Persister,
persister::Error: From<<P as Persister>::Error>,
T: TimeProvider,
W: Wal,
write_buffer::Error: From<<P as Persister>::Error>,
{
// this loop is where persistence happens so if anything is in persisting,
// it's either been dropped or remaining from a restart, so clear those out first.
let persisting_segments = {
let segment_state = segment_state.read();
segment_state
.persisting_segments
.values()
.cloned()
.collect::<Vec<_>>()
};
for segment in persisting_segments {
persist_closed_segment_and_cleanup(
segment,
Arc::clone(&persister),
Arc::clone(&segment_state),
wal.clone(),
Arc::clone(&executor),
)
.await
.unwrap()
}
// check for open segments to persist
let current_time = time_provider.now();
let segments_to_persist = {
let segment_state = segment_state.read();
segment_state.segments_to_persist(current_time)
};
// close and persist each one in turn
for segment_start in segments_to_persist {
let closed_segment = {
let mut segment_state = segment_state.write();
segment_state.close_segment(segment_start)
};
if let Some(closed_segment) = closed_segment {
persist_closed_segment_and_cleanup(
closed_segment,
Arc::clone(&persister),
Arc::clone(&segment_state),
wal.clone(),
Arc::clone(&executor),
)
.await
.unwrap()
}
}
Ok(())
}
// Performs the following:
// 1. persist the segment to the object store
// 2. remove the segment from the persisting_segments map and add it to the persisted_segments map
// 3. remove the wal segment file
async fn persist_closed_segment_and_cleanup<P, T, W>(
closed_segment: Arc<ClosedBufferSegment>,
persister: Arc<P>,
segment_state: Arc<RwLock<SegmentState<T, W>>>,
wal: Option<Arc<W>>,
executor: Arc<iox_query::exec::Executor>,
) -> Result<(), crate::Error>
where
P: Persister,
persister::Error: From<<P as Persister>::Error>,
T: TimeProvider,
W: Wal,
write_buffer::Error: From<<P as Persister>::Error>,
{
let closed_segment_start_time = closed_segment.segment_range.start_time;
let closed_segment_id = closed_segment.segment_id;
let persisted_segment = closed_segment.persist(persister, executor, None).await?;
{
let mut segment_state = segment_state.write();
segment_state
.persisting_segments
.remove(&closed_segment_start_time);
segment_state
.persisted_segments
.insert(closed_segment_start_time, Arc::new(persisted_segment));
}
if let Some(wal) = wal {
wal.delete_wal_segment(closed_segment_id)?;
}
Ok(())
}
#[cfg(test)]
mod tests {
use super::*;
use crate::test_helpers::lp_to_write_batch;
use crate::wal::WalImpl;
use crate::{SegmentFile, WalSegmentReader, WalSegmentWriter};
use iox_time::MockProvider;
use parking_lot::Mutex;
use std::any::Any;
use std::fmt::Debug;
use write_buffer::buffer_segment::tests::TestPersister;
#[test]
fn segments_to_persist_sorts_oldest_first() {
let catalog = Arc::new(Catalog::new());
let time_provider = Arc::new(MockProvider::new(Time::from_timestamp_nanos(0)));
let segment_duration = SegmentDuration::new_5m();
let first_segment_range = SegmentRange::from_time_and_duration(
Time::from_timestamp_nanos(0),
segment_duration,
false,
);
let open_segment1 = OpenBufferSegment::new(
Arc::clone(&catalog),
SegmentId::new(1),
first_segment_range,
time_provider.now(),
catalog.sequence_number(),
Box::new(WalSegmentWriterNoopImpl::new(SegmentId::new(1))),
None,
);
let open_segment2 = OpenBufferSegment::new(
Arc::clone(&catalog),
SegmentId::new(2),
SegmentRange::from_time_and_duration(
Time::from_timestamp(300, 0).unwrap(),
segment_duration,
false,
),
time_provider.now(),
catalog.sequence_number(),
Box::new(WalSegmentWriterNoopImpl::new(SegmentId::new(2))),
None,
);
let open_segment3 = OpenBufferSegment::new(
Arc::clone(&catalog),
SegmentId::new(3),
SegmentRange::from_time_and_duration(
Time::from_timestamp(600, 0).unwrap(),
segment_duration,
false,
),
time_provider.now(),
catalog.sequence_number(),
Box::new(WalSegmentWriterNoopImpl::new(SegmentId::new(3))),
None,
);
let segment_state: SegmentState<MockProvider, WalImpl> = SegmentState::new(
SegmentDuration::new_5m(),
SegmentId::new(4),
Arc::clone(&catalog),
Arc::clone(&time_provider),
vec![open_segment1, open_segment2, open_segment3],
vec![],
vec![],
None,
);
let segments_to_persist =
segment_state.segments_to_persist(Time::from_timestamp(800, 0).unwrap());
assert_eq!(
segments_to_persist,
vec![
Time::from_timestamp_nanos(0),
Time::from_timestamp(300, 0).unwrap()
]
);
}
#[tokio::test]
async fn persist_and_cleanup_ready_segments_handles_persisting_and_rotates_old() {
let catalog = Arc::new(Catalog::new());
let time_provider = Arc::new(MockProvider::new(Time::from_timestamp_nanos(0)));
let segment_duration = SegmentDuration::new_5m();
let first_segment_range = SegmentRange::from_time_and_duration(
Time::from_timestamp_nanos(0),
segment_duration,
false,
);
let mut open_segment1 = OpenBufferSegment::new(
Arc::clone(&catalog),
SegmentId::new(1),
first_segment_range,
time_provider.now(),
catalog.sequence_number(),
Box::new(WalSegmentWriterNoopImpl::new(SegmentId::new(1))),
None,
);
open_segment1
.buffer_writes(lp_to_write_batch(&catalog, "foo", "cpu bar=1 10"))
.unwrap();
let mut open_segment2 = OpenBufferSegment::new(
Arc::clone(&catalog),
SegmentId::new(2),
SegmentRange::from_time_and_duration(
Time::from_timestamp(300, 0).unwrap(),
segment_duration,
false,
),
time_provider.now(),
catalog.sequence_number(),
Box::new(WalSegmentWriterNoopImpl::new(SegmentId::new(2))),
None,
);
open_segment2
.buffer_writes(lp_to_write_batch(&catalog, "foo", "cpu bar=2 300000000000"))
.unwrap();
let mut open_segment3 = OpenBufferSegment::new(
Arc::clone(&catalog),
SegmentId::new(3),
SegmentRange::from_time_and_duration(
Time::from_timestamp(600, 0).unwrap(),
segment_duration,
false,
),
time_provider.now(),
catalog.sequence_number(),
Box::new(WalSegmentWriterNoopImpl::new(SegmentId::new(3))),
None,
);
open_segment3
.buffer_writes(lp_to_write_batch(&catalog, "foo", "cpu bar=3 700000000000"))
.unwrap();
let wal = Arc::new(TestWal::default());
let segment_state: SegmentState<MockProvider, TestWal> = SegmentState::new(
SegmentDuration::new_5m(),
SegmentId::new(4),
Arc::clone(&catalog),
Arc::clone(&time_provider),
vec![open_segment2, open_segment3],
vec![open_segment1.into_closed_segment(Arc::clone(&catalog))],
vec![],
Some(Arc::clone(&wal)),
);
let segment_state = Arc::new(RwLock::new(segment_state));
let persister = Arc::new(TestPersister::default());
time_provider.set(Time::from_timestamp(900, 0).unwrap());
persist_and_cleanup_ready_segments(
Arc::clone(&persister),
Arc::clone(&segment_state),
Arc::clone(&time_provider),
Some(Arc::clone(&wal)),
crate::test_help::make_exec(),
)
.await
.unwrap();
let persisted_state = persister
.as_any()
.downcast_ref::<TestPersister>()
.unwrap()
.state
.lock();
assert_eq!(persisted_state.catalog.len(), 1);
assert_eq!(persisted_state.segments.len(), 2);
assert_eq!(persisted_state.parquet_files.len(), 2);
let wal_state = wal.as_any().downcast_ref::<TestWal>().unwrap();
let deleted_segments = wal_state.deleted_wal_segments.lock().clone();
assert_eq!(deleted_segments, vec![SegmentId::new(1), SegmentId::new(2)]);
}
#[derive(Debug, Default)]
struct TestWal {
deleted_wal_segments: Mutex<Vec<SegmentId>>,
}
impl Wal for TestWal {
fn new_segment_writer(
&self,
_segment_id: SegmentId,
_range: SegmentRange,
) -> wal::Result<Box<dyn WalSegmentWriter>> {
todo!()
}
fn open_segment_writer(
&self,
_segment_id: SegmentId,
) -> wal::Result<Box<dyn WalSegmentWriter>> {
todo!()
}
fn open_segment_reader(
&self,
_segment_id: SegmentId,
) -> wal::Result<Box<dyn WalSegmentReader>> {
todo!()
}
fn segment_files(&self) -> wal::Result<Vec<SegmentFile>> {
todo!()
}
fn delete_wal_segment(&self, segment_id: SegmentId) -> wal::Result<()> {
self.deleted_wal_segments.lock().push(segment_id);
Ok(())
}
fn as_any(&self) -> &dyn Any {
self as &dyn Any
}
}
}