-
Notifications
You must be signed in to change notification settings - Fork 598
/
Copy pathcursor_manager.rs
808 lines (758 loc) · 28.5 KB
/
cursor_manager.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
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
// Copyright 2024 RisingWave Labs
//
// 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.
use core::mem;
use core::time::Duration;
use std::collections::{HashMap, VecDeque};
use std::rc::Rc;
use std::sync::Arc;
use std::time::Instant;
use bytes::Bytes;
use fixedbitset::FixedBitSet;
use futures::StreamExt;
use pgwire::pg_field_descriptor::PgFieldDescriptor;
use pgwire::pg_response::StatementType;
use pgwire::types::{Format, Row};
use risingwave_common::catalog::Field;
use risingwave_common::error::BoxedError;
use risingwave_common::session_config::QueryMode;
use risingwave_common::types::DataType;
use risingwave_sqlparser::ast::{Ident, ObjectName, Statement};
use super::SessionImpl;
use crate::catalog::subscription_catalog::SubscriptionCatalog;
use crate::catalog::TableId;
use crate::error::{ErrorCode, Result};
use crate::handler::declare_cursor::{
create_chunk_stream_for_cursor, create_stream_for_cursor_stmt,
};
use crate::handler::query::{gen_batch_plan_fragmenter, BatchQueryPlanResult};
use crate::handler::util::{
convert_logstore_u64_to_unix_millis, gen_query_from_table_name, pg_value_format, to_pg_field,
DataChunkToRowSetAdapter, StaticSessionData,
};
use crate::handler::HandlerArgs;
use crate::optimizer::plan_node::{generic, BatchLogSeqScan};
use crate::optimizer::property::{Order, RequiredDist};
use crate::optimizer::PlanRoot;
use crate::scheduler::{DistributedQueryStream, LocalQueryStream, ReadSnapshot};
use crate::{OptimizerContext, OptimizerContextRef, PgResponseStream, PlanRef, TableCatalog};
pub enum CursorDataChunkStream {
LocalDataChunk(Option<LocalQueryStream>),
DistributedDataChunk(Option<DistributedQueryStream>),
PgResponse(PgResponseStream),
}
impl CursorDataChunkStream {
pub fn init_row_stream(
&mut self,
fields: &Vec<Field>,
formats: &Vec<Format>,
session: Arc<SessionImpl>,
) {
let columns_type = fields.iter().map(|f| f.data_type().clone()).collect();
match self {
CursorDataChunkStream::LocalDataChunk(data_chunk) => {
let data_chunk = mem::take(data_chunk).unwrap();
let row_stream = PgResponseStream::LocalQuery(DataChunkToRowSetAdapter::new(
data_chunk,
columns_type,
formats.clone(),
session,
));
*self = CursorDataChunkStream::PgResponse(row_stream);
}
CursorDataChunkStream::DistributedDataChunk(data_chunk) => {
let data_chunk = mem::take(data_chunk).unwrap();
let row_stream = PgResponseStream::DistributedQuery(DataChunkToRowSetAdapter::new(
data_chunk,
columns_type,
formats.clone(),
session,
));
*self = CursorDataChunkStream::PgResponse(row_stream);
}
_ => {}
}
}
pub async fn next(&mut self) -> Result<Option<std::result::Result<Vec<Row>, BoxedError>>> {
match self {
CursorDataChunkStream::PgResponse(row_stream) => Ok(row_stream.next().await),
_ => Err(ErrorCode::InternalError(
"Only 'CursorDataChunkStream' can call next and return rows".to_string(),
)
.into()),
}
}
}
pub enum Cursor {
Subscription(SubscriptionCursor),
Query(QueryCursor),
}
impl Cursor {
pub async fn next(
&mut self,
count: u32,
handle_args: HandlerArgs,
formats: &Vec<Format>,
) -> Result<(Vec<Row>, Vec<PgFieldDescriptor>)> {
match self {
Cursor::Subscription(cursor) => cursor.next(count, handle_args, formats).await,
Cursor::Query(cursor) => cursor.next(count, formats, handle_args).await,
}
}
pub fn get_fields(&mut self) -> Vec<Field> {
match self {
Cursor::Subscription(cursor) => cursor.fields.clone(),
Cursor::Query(cursor) => cursor.fields.clone(),
}
}
}
pub struct QueryCursor {
chunk_stream: CursorDataChunkStream,
fields: Vec<Field>,
remaining_rows: VecDeque<Row>,
}
impl QueryCursor {
pub fn new(chunk_stream: CursorDataChunkStream, fields: Vec<Field>) -> Result<Self> {
Ok(Self {
chunk_stream,
fields,
remaining_rows: VecDeque::<Row>::new(),
})
}
pub async fn next_once(&mut self) -> Result<Option<Row>> {
while self.remaining_rows.is_empty() {
let rows = self.chunk_stream.next().await?;
let rows = match rows {
None => return Ok(None),
Some(row) => row?,
};
self.remaining_rows = rows.into_iter().collect();
}
let row = self.remaining_rows.pop_front().unwrap();
Ok(Some(row))
}
pub async fn next(
&mut self,
count: u32,
formats: &Vec<Format>,
handle_args: HandlerArgs,
) -> Result<(Vec<Row>, Vec<PgFieldDescriptor>)> {
// `FETCH NEXT` is equivalent to `FETCH 1`.
// min with 100 to avoid allocating too many memory at once.
let session = handle_args.session;
let mut ans = Vec::with_capacity(std::cmp::min(100, count) as usize);
let mut cur = 0;
let desc = self.fields.iter().map(to_pg_field).collect();
self.chunk_stream
.init_row_stream(&self.fields, formats, session);
while cur < count
&& let Some(row) = self.next_once().await?
{
cur += 1;
ans.push(row);
}
Ok((ans, desc))
}
}
enum State {
InitLogStoreQuery {
// The rw_timestamp used to initiate the query to read from subscription logstore.
seek_timestamp: u64,
// If specified, the expected_timestamp must be an exact match for the next rw_timestamp.
expected_timestamp: Option<u64>,
},
Fetch {
// Whether the query is reading from snapshot
// true: read from the upstream table snapshot
// false: read from subscription logstore
from_snapshot: bool,
// The rw_timestamp used to initiate the query to read from subscription logstore.
rw_timestamp: u64,
// The row stream to from the batch query read.
// It is returned from the batch execution.
chunk_stream: CursorDataChunkStream,
// A cache to store the remaining rows from the row stream.
remaining_rows: VecDeque<Row>,
expected_timestamp: Option<u64>,
},
Invalid,
}
pub struct SubscriptionCursor {
cursor_name: String,
subscription: Arc<SubscriptionCatalog>,
dependent_table_id: TableId,
cursor_need_drop_time: Instant,
state: State,
// fields will be set in the table's catalog when the cursor is created,
// and will be reset each time it is created chunk_stream, this is to avoid changes in the catalog due to alter.
fields: Vec<Field>,
}
impl SubscriptionCursor {
pub async fn new(
cursor_name: String,
start_timestamp: Option<u64>,
subscription: Arc<SubscriptionCatalog>,
dependent_table_id: TableId,
handle_args: &HandlerArgs,
) -> Result<Self> {
let (state, fields) = if let Some(start_timestamp) = start_timestamp {
let table_catalog = handle_args.session.get_table_by_id(&dependent_table_id)?;
let fields = table_catalog
.columns
.iter()
.filter(|c| !c.is_hidden)
.map(|c| Field::with_name(c.data_type().clone(), c.name()))
.collect();
let fields = Self::build_desc(fields, true);
(
State::InitLogStoreQuery {
seek_timestamp: start_timestamp,
expected_timestamp: None,
},
fields,
)
} else {
// The query stream needs to initiated on cursor creation to make sure
// future fetch on the cursor starts from the snapshot when the cursor is declared.
//
// TODO: is this the right behavior? Should we delay the query stream initiation till the first fetch?
let (chunk_stream, fields) =
Self::initiate_query(None, &dependent_table_id, handle_args.clone()).await?;
let pinned_epoch = match handle_args.session.get_pinned_snapshot().ok_or_else(|| {
ErrorCode::InternalError("Fetch Cursor can't find snapshot epoch".to_string())
})? {
ReadSnapshot::FrontendPinned { snapshot, .. } => snapshot.committed_epoch(),
ReadSnapshot::Other(_) => {
return Err(ErrorCode::InternalError("Fetch Cursor can't start from specified query epoch. May run `set query_epoch = 0;`".to_string()).into());
}
};
let start_timestamp = pinned_epoch;
(
State::Fetch {
from_snapshot: true,
rw_timestamp: start_timestamp,
chunk_stream,
remaining_rows: VecDeque::new(),
expected_timestamp: None,
},
fields,
)
};
let cursor_need_drop_time =
Instant::now() + Duration::from_secs(subscription.retention_seconds);
Ok(Self {
cursor_name,
subscription,
dependent_table_id,
cursor_need_drop_time,
state,
fields,
})
}
async fn next_row(
&mut self,
handle_args: &HandlerArgs,
formats: &Vec<Format>,
) -> Result<Option<Row>> {
loop {
match &mut self.state {
State::InitLogStoreQuery {
seek_timestamp,
expected_timestamp,
} => {
let from_snapshot = false;
// Initiate a new batch query to continue fetching
match Self::get_next_rw_timestamp(
*seek_timestamp,
&self.dependent_table_id,
*expected_timestamp,
handle_args.clone(),
&self.subscription,
)
.await
{
Ok((Some(rw_timestamp), expected_timestamp)) => {
let (mut chunk_stream, fields) = Self::initiate_query(
Some(rw_timestamp),
&self.dependent_table_id,
handle_args.clone(),
)
.await?;
Self::init_row_stream(
&mut chunk_stream,
formats,
&from_snapshot,
&fields,
handle_args.session.clone(),
);
self.cursor_need_drop_time = Instant::now()
+ Duration::from_secs(self.subscription.retention_seconds);
let mut remaining_rows = VecDeque::new();
Self::try_refill_remaining_rows(&mut chunk_stream, &mut remaining_rows)
.await?;
// Transition to the Fetch state
self.state = State::Fetch {
from_snapshot,
rw_timestamp,
chunk_stream,
remaining_rows,
expected_timestamp,
};
if self.fields.ne(&fields) {
self.fields = fields;
return Ok(None);
}
}
Ok((None, _)) => return Ok(None),
Err(e) => {
self.state = State::Invalid;
return Err(e);
}
}
}
State::Fetch {
from_snapshot,
rw_timestamp,
chunk_stream,
remaining_rows,
expected_timestamp,
} => {
let session_data = StaticSessionData {
timezone: handle_args.session.config().timezone(),
};
let from_snapshot = *from_snapshot;
let rw_timestamp = *rw_timestamp;
// Try refill remaining rows
Self::try_refill_remaining_rows(chunk_stream, remaining_rows).await?;
if let Some(row) = remaining_rows.pop_front() {
// 1. Fetch the next row
let new_row = row.take();
if from_snapshot {
return Ok(Some(Row::new(Self::build_row(
new_row,
None,
formats,
&session_data,
)?)));
} else {
return Ok(Some(Row::new(Self::build_row(
new_row,
Some(rw_timestamp),
formats,
&session_data,
)?)));
}
} else {
// 2. Reach EOF for the current query.
if let Some(expected_timestamp) = expected_timestamp {
self.state = State::InitLogStoreQuery {
seek_timestamp: *expected_timestamp,
expected_timestamp: Some(*expected_timestamp),
};
} else {
self.state = State::InitLogStoreQuery {
seek_timestamp: rw_timestamp + 1,
expected_timestamp: None,
};
}
}
}
State::Invalid => {
// TODO: auto close invalid cursor?
return Err(ErrorCode::InternalError(
"Cursor is in invalid state. Please close and re-create the cursor."
.to_string(),
)
.into());
}
}
}
}
pub async fn next(
&mut self,
count: u32,
handle_args: HandlerArgs,
formats: &Vec<Format>,
) -> Result<(Vec<Row>, Vec<PgFieldDescriptor>)> {
if Instant::now() > self.cursor_need_drop_time {
return Err(ErrorCode::InternalError(
"The cursor has exceeded its maximum lifetime, please recreate it (close then declare cursor).".to_string(),
)
.into());
}
let mut ans = Vec::with_capacity(std::cmp::min(100, count) as usize);
let mut cur = 0;
let desc = self.fields.iter().map(to_pg_field).collect();
if let State::Fetch {
from_snapshot,
chunk_stream,
..
} = &mut self.state
{
Self::init_row_stream(
chunk_stream,
formats,
from_snapshot,
&self.fields,
handle_args.session.clone(),
);
}
while cur < count {
let row = self.next_row(&handle_args, formats).await?;
match row {
Some(row) => {
cur += 1;
ans.push(row);
}
None => {
break;
}
}
}
Ok((ans, desc))
}
async fn get_next_rw_timestamp(
seek_timestamp: u64,
table_id: &TableId,
expected_timestamp: Option<u64>,
handle_args: HandlerArgs,
dependent_subscription: &SubscriptionCatalog,
) -> Result<(Option<u64>, Option<u64>)> {
let session = handle_args.session;
// Test subscription existence
session.get_subscription_by_schema_id_name(
dependent_subscription.schema_id,
&dependent_subscription.name,
)?;
// The epoch here must be pulled every time, otherwise there will be cache consistency issues
let new_epochs = session
.list_change_log_epochs(table_id.table_id(), seek_timestamp, 2)
.await?;
if let Some(expected_timestamp) = expected_timestamp
&& (new_epochs.is_empty() || &expected_timestamp != new_epochs.first().unwrap())
{
return Err(ErrorCode::CatalogError(
format!(
" No data found for rw_timestamp {:?}, data may have been recycled, please recreate cursor",
convert_logstore_u64_to_unix_millis(expected_timestamp)
)
.into(),
)
.into());
}
Ok((new_epochs.get(0).cloned(), new_epochs.get(1).cloned()))
}
async fn initiate_query(
rw_timestamp: Option<u64>,
dependent_table_id: &TableId,
handle_args: HandlerArgs,
) -> Result<(CursorDataChunkStream, Vec<Field>)> {
let session = handle_args.clone().session;
let table_catalog = session.get_table_by_id(dependent_table_id)?;
let (chunk_stream, fields) = if let Some(rw_timestamp) = rw_timestamp {
let context = OptimizerContext::from_handler_args(handle_args);
let plan_fragmenter_result = gen_batch_plan_fragmenter(
&session,
Self::create_batch_plan_for_cursor(
&table_catalog,
&session,
context.into(),
rw_timestamp,
rw_timestamp,
)?,
)?;
create_chunk_stream_for_cursor(session, plan_fragmenter_result).await?
} else {
let subscription_from_table_name =
ObjectName(vec![Ident::from(table_catalog.name.as_ref())]);
let query_stmt = Statement::Query(Box::new(gen_query_from_table_name(
subscription_from_table_name,
)));
create_stream_for_cursor_stmt(handle_args, query_stmt).await?
};
Ok((
chunk_stream,
Self::build_desc(fields, rw_timestamp.is_none()),
))
}
async fn try_refill_remaining_rows(
chunk_stream: &mut CursorDataChunkStream,
remaining_rows: &mut VecDeque<Row>,
) -> Result<()> {
if remaining_rows.is_empty()
&& let Some(row_set) = chunk_stream.next().await?
{
remaining_rows.extend(row_set?);
}
Ok(())
}
pub fn build_row(
mut row: Vec<Option<Bytes>>,
rw_timestamp: Option<u64>,
formats: &Vec<Format>,
session_data: &StaticSessionData,
) -> Result<Vec<Option<Bytes>>> {
let row_len = row.len();
let new_row = if let Some(rw_timestamp) = rw_timestamp {
let rw_timestamp_formats = formats.get(row_len).unwrap_or(&Format::Text);
let rw_timestamp = convert_logstore_u64_to_unix_millis(rw_timestamp);
let rw_timestamp = pg_value_format(
&DataType::Int64,
risingwave_common::types::ScalarRefImpl::Int64(rw_timestamp as i64),
*rw_timestamp_formats,
session_data,
)?;
vec![Some(rw_timestamp)]
} else {
let op_formats = formats.get(row_len).unwrap_or(&Format::Text);
let op = pg_value_format(
&DataType::Varchar,
risingwave_common::types::ScalarRefImpl::Utf8("Insert"),
*op_formats,
session_data,
)?;
vec![Some(op), None]
};
row.extend(new_row);
Ok(row)
}
pub fn build_desc(mut descs: Vec<Field>, from_snapshot: bool) -> Vec<Field> {
if from_snapshot {
descs.push(Field::with_name(DataType::Varchar, "op"));
}
descs.push(Field::with_name(DataType::Int64, "rw_timestamp"));
descs
}
pub fn create_batch_plan_for_cursor(
table_catalog: &TableCatalog,
session: &SessionImpl,
context: OptimizerContextRef,
old_epoch: u64,
new_epoch: u64,
) -> Result<BatchQueryPlanResult> {
let out_col_idx = table_catalog
.columns
.iter()
.enumerate()
.filter(|(_, v)| !v.is_hidden)
.map(|(i, _)| i)
.collect::<Vec<_>>();
let core = generic::LogScan::new(
table_catalog.name.clone(),
out_col_idx,
Rc::new(table_catalog.table_desc()),
context,
old_epoch,
new_epoch,
);
let batch_log_seq_scan = BatchLogSeqScan::new(core);
let schema = batch_log_seq_scan
.core()
.schema_without_table_name()
.clone();
let out_fields = FixedBitSet::from_iter(0..schema.len());
let out_names = batch_log_seq_scan.core().column_names();
// Here we just need a plan_root to call the method, only out_fields and out_names will be used
let plan_root = PlanRoot::new_with_batch_plan(
PlanRef::from(batch_log_seq_scan.clone()),
RequiredDist::single(),
Order::default(),
out_fields,
out_names,
);
let (batch_log_seq_scan, query_mode) = match session.config().query_mode() {
QueryMode::Auto => (plan_root.gen_batch_local_plan()?, QueryMode::Local),
QueryMode::Local => (plan_root.gen_batch_local_plan()?, QueryMode::Local),
QueryMode::Distributed => (
plan_root.gen_batch_distributed_plan()?,
QueryMode::Distributed,
),
};
Ok(BatchQueryPlanResult {
plan: batch_log_seq_scan,
query_mode,
schema,
stmt_type: StatementType::SELECT,
dependent_relations: table_catalog.dependent_relations.clone(),
})
}
// In the beginning (declare cur), we will give it an empty formats,
// this formats is not a real, when we fetch, We fill it with the formats returned from the pg client.
pub fn init_row_stream(
chunk_stream: &mut CursorDataChunkStream,
formats: &Vec<Format>,
from_snapshot: &bool,
fields: &Vec<Field>,
session: Arc<SessionImpl>,
) {
let mut formats = formats.clone();
let mut fields = fields.clone();
formats.pop();
fields.pop();
if *from_snapshot {
formats.pop();
fields.pop();
}
chunk_stream.init_row_stream(&fields, &formats, session);
}
}
#[derive(Default)]
pub struct CursorManager {
cursor_map: tokio::sync::Mutex<HashMap<String, Cursor>>,
}
impl CursorManager {
pub async fn add_subscription_cursor(
&self,
cursor_name: String,
start_timestamp: Option<u64>,
dependent_table_id: TableId,
subscription: Arc<SubscriptionCatalog>,
handle_args: &HandlerArgs,
) -> Result<()> {
let cursor = SubscriptionCursor::new(
cursor_name.clone(),
start_timestamp,
subscription,
dependent_table_id,
handle_args,
)
.await?;
let mut cursor_map = self.cursor_map.lock().await;
cursor_map.retain(|_, v| {
if let Cursor::Subscription(cursor) = v {
!matches!(cursor.state, State::Invalid)
} else {
true
}
});
cursor_map
.try_insert(cursor.cursor_name.clone(), Cursor::Subscription(cursor))
.map_err(|_| {
ErrorCode::CatalogError(format!("cursor `{}` already exists", cursor_name).into())
})?;
Ok(())
}
pub async fn add_query_cursor(
&self,
cursor_name: ObjectName,
chunk_stream: CursorDataChunkStream,
fields: Vec<Field>,
) -> Result<()> {
let cursor = QueryCursor::new(chunk_stream, fields)?;
self.cursor_map
.lock()
.await
.try_insert(cursor_name.to_string(), Cursor::Query(cursor))
.map_err(|_| {
ErrorCode::CatalogError(format!("cursor `{}` already exists", cursor_name).into())
})?;
Ok(())
}
pub async fn remove_cursor(&self, cursor_name: String) -> Result<()> {
self.cursor_map
.lock()
.await
.remove(&cursor_name)
.ok_or_else(|| {
ErrorCode::CatalogError(format!("cursor `{}` don't exists", cursor_name).into())
})?;
Ok(())
}
pub async fn remove_all_cursor(&self) {
self.cursor_map.lock().await.clear();
}
pub async fn remove_all_query_cursor(&self) {
self.cursor_map
.lock()
.await
.retain(|_, v| matches!(v, Cursor::Subscription(_)));
}
pub async fn get_rows_with_cursor(
&self,
cursor_name: String,
count: u32,
handle_args: HandlerArgs,
formats: &Vec<Format>,
) -> Result<(Vec<Row>, Vec<PgFieldDescriptor>)> {
if let Some(cursor) = self.cursor_map.lock().await.get_mut(&cursor_name) {
cursor.next(count, handle_args, formats).await
} else {
Err(ErrorCode::ItemNotFound(format!("Cannot find cursor `{}`", cursor_name)).into())
}
}
pub async fn get_fields_with_cursor(&self, cursor_name: String) -> Result<Vec<Field>> {
if let Some(cursor) = self.cursor_map.lock().await.get_mut(&cursor_name) {
Ok(cursor.get_fields())
} else {
Err(ErrorCode::ItemNotFound(format!("Cannot find cursor `{}`", cursor_name)).into())
}
}
pub async fn get_all_query_cursors(&self) -> (Vec<Row>, Vec<PgFieldDescriptor>) {
let cursor_names = self
.cursor_map
.lock()
.await
.iter()
.filter_map(|(currsor_name, cursor)| {
if let Cursor::Query(_cursor) = cursor {
let cursor_name = vec![Some(Bytes::from(currsor_name.clone().into_bytes()))];
Some(Row::new(cursor_name))
} else {
None
}
})
.collect();
(
cursor_names,
vec![PgFieldDescriptor::new(
"Name".to_string(),
DataType::Varchar.to_oid(),
DataType::Varchar.type_len(),
)],
)
}
pub async fn get_all_subscription_cursors(&self) -> (Vec<Row>, Vec<PgFieldDescriptor>) {
let cursors = self
.cursor_map
.lock()
.await
.iter()
.filter_map(|(cursor_name, cursor)| {
if let Cursor::Subscription(cursor) = cursor {
let cursors = vec![
Some(Bytes::from(cursor_name.clone().into_bytes())),
Some(Bytes::from(cursor.subscription.name.clone().into_bytes())),
];
Some(Row::new(cursors))
} else {
None
}
})
.collect();
(
cursors,
vec![
PgFieldDescriptor::new(
"Name".to_string(),
DataType::Varchar.to_oid(),
DataType::Varchar.type_len(),
),
PgFieldDescriptor::new(
"SubscriptionName".to_string(),
DataType::Varchar.to_oid(),
DataType::Varchar.type_len(),
),
],
)
}
}