16
16
// under the License.
17
17
18
18
use std:: collections:: HashMap ;
19
- use std:: future:: Future ;
20
- use std:: ops:: Deref ;
21
- use std:: pin:: Pin ;
22
- use std:: sync:: { Arc , RwLock } ;
23
- use std:: task:: { Context , Poll } ;
19
+ use std:: sync:: { Arc , OnceLock } ;
24
20
25
21
use futures:: channel:: mpsc:: { channel, Sender } ;
26
22
use futures:: StreamExt ;
23
+ use tokio:: sync:: Notify ;
27
24
28
25
use crate :: runtime:: spawn;
29
26
use crate :: scan:: { DeleteFileContext , FileScanTaskDeleteFile } ;
30
27
use crate :: spec:: { DataContentType , DataFile , Struct } ;
31
- use crate :: { Error , ErrorKind , Result } ;
28
+ use crate :: Result ;
32
29
33
30
/// Index of delete files
34
31
#[ derive( Clone , Debug ) ]
35
32
pub ( crate ) struct DeleteFileIndex {
36
- state : Arc < RwLock < DeleteFileIndexState > > ,
37
- }
38
-
39
- #[ derive( Debug ) ]
40
- enum DeleteFileIndexState {
41
- Populating ,
42
- Populated ( PopulatedDeleteFileIndex ) ,
33
+ index : Arc < OnceLock < PopulatedDeleteFileIndex > > ,
34
+ ready_notify : Arc < Notify > ,
43
35
}
44
36
45
37
#[ derive( Debug ) ]
@@ -59,36 +51,50 @@ impl DeleteFileIndex {
59
51
pub ( crate ) fn new ( ) -> ( DeleteFileIndex , Sender < DeleteFileContext > ) {
60
52
// TODO: what should the channel limit be?
61
53
let ( tx, rx) = channel ( 10 ) ;
62
- let state = Arc :: new ( RwLock :: new ( DeleteFileIndexState :: Populating ) ) ;
54
+ let index = Arc :: new ( OnceLock :: new ( ) ) ;
55
+ let ready_notify = Arc :: new ( Notify :: new ( ) ) ;
63
56
let delete_file_stream = rx. boxed ( ) ;
64
57
65
58
spawn ( {
66
- let state = state. clone ( ) ;
59
+ let index = index. clone ( ) ;
60
+ let ready_notify = ready_notify. clone ( ) ;
67
61
async move {
68
62
let delete_files = delete_file_stream. collect :: < Vec < _ > > ( ) . await ;
69
63
70
64
let populated_delete_file_index = PopulatedDeleteFileIndex :: new ( delete_files) ;
71
65
72
- let mut guard = state. write ( ) . unwrap ( ) ;
73
- * guard = DeleteFileIndexState :: Populated ( populated_delete_file_index) ;
66
+ index
67
+ . set ( populated_delete_file_index)
68
+ . expect ( "delete file index should not be written by another thread" ) ;
69
+ ready_notify. notify_waiters ( ) ;
74
70
}
75
71
} ) ;
76
72
77
- ( DeleteFileIndex { state } , tx)
73
+ (
74
+ DeleteFileIndex {
75
+ index,
76
+ ready_notify,
77
+ } ,
78
+ tx,
79
+ )
78
80
}
79
81
80
82
/// Gets all the delete files that apply to the specified data file.
81
- ///
82
- /// Returns a future that resolves to a Result<Vec<FileScanTaskDeleteFile>>
83
- pub ( crate ) fn get_deletes_for_data_file < ' a > (
83
+ pub ( crate ) async fn get_deletes_for_data_file (
84
84
& self ,
85
- data_file : & ' a DataFile ,
85
+ data_file : & DataFile ,
86
86
seq_num : Option < i64 > ,
87
- ) -> DeletesForDataFile < ' a > {
88
- DeletesForDataFile {
89
- state : self . state . clone ( ) ,
90
- data_file,
91
- seq_num,
87
+ ) -> Result < Vec < FileScanTaskDeleteFile > > {
88
+ match self . index . get ( ) {
89
+ Some ( idx) => Ok ( idx. get_deletes_for_data_file ( data_file, seq_num) ) ,
90
+ None => {
91
+ self . ready_notify . notified ( ) . await ;
92
+ Ok ( self
93
+ . index
94
+ . get ( )
95
+ . unwrap ( )
96
+ . get_deletes_for_data_file ( data_file, seq_num) )
97
+ }
92
98
}
93
99
}
94
100
}
@@ -193,26 +199,3 @@ impl PopulatedDeleteFileIndex {
193
199
results
194
200
}
195
201
}
196
-
197
- /// Future for the `DeleteFileIndex::get_deletes_for_data_file` method
198
- pub ( crate ) struct DeletesForDataFile < ' a > {
199
- state : Arc < RwLock < DeleteFileIndexState > > ,
200
- data_file : & ' a DataFile ,
201
- seq_num : Option < i64 > ,
202
- }
203
-
204
- impl Future for DeletesForDataFile < ' _ > {
205
- type Output = Result < Vec < FileScanTaskDeleteFile > > ;
206
-
207
- fn poll ( self : Pin < & mut Self > , _cx : & mut Context < ' _ > ) -> Poll < Self :: Output > {
208
- match self . state . try_read ( ) {
209
- Ok ( guard) => match guard. deref ( ) {
210
- DeleteFileIndexState :: Populated ( idx) => Poll :: Ready ( Ok (
211
- idx. get_deletes_for_data_file ( self . data_file , self . seq_num )
212
- ) ) ,
213
- _ => Poll :: Pending ,
214
- } ,
215
- Err ( err) => Poll :: Ready ( Err ( Error :: new ( ErrorKind :: Unexpected , err. to_string ( ) ) ) ) ,
216
- }
217
- }
218
- }
0 commit comments