18
18
use arrow_array:: RecordBatch ;
19
19
20
20
use crate :: io:: FileIO ;
21
- use crate :: spec:: { DataFile , PartitionKey } ;
21
+ use crate :: spec:: PartitionKey ;
22
22
use crate :: writer:: file_writer:: location_generator:: { FileNameGenerator , LocationGenerator } ;
23
- use crate :: writer:: { CurrentFileStatus , IcebergWriter , IcebergWriterBuilder } ;
23
+ use crate :: writer:: {
24
+ CurrentFileStatus , DefaultInput , DefaultOutput , IcebergWriter , IcebergWriterBuilder ,
25
+ } ;
24
26
use crate :: { Error , ErrorKind , Result } ;
25
27
26
28
/// A writer that automatically rolls over to a new file when the data size
@@ -29,11 +31,12 @@ use crate::{Error, ErrorKind, Result};
29
31
/// This writer wraps another writer that tracks the amount of data written.
30
32
/// When the data size exceeds the target size, it closes the current file and
31
33
/// starts writing to a new one.
32
- pub struct RollingWriter < B , L , F >
34
+ pub struct RollingWriter < B , L , F , I = DefaultInput , O = DefaultOutput >
33
35
where
34
- B : IcebergWriterBuilder ,
36
+ B : IcebergWriterBuilder < I , O > ,
35
37
L : LocationGenerator ,
36
38
F : FileNameGenerator ,
39
+ O : IntoIterator ,
37
40
{
38
41
inner : Option < B :: R > ,
39
42
inner_builder : B ,
@@ -42,15 +45,17 @@ where
42
45
file_name_generator : F ,
43
46
file_io : FileIO ,
44
47
partition_key : Option < PartitionKey > ,
45
- data_files : Vec < DataFile > , // todo this should be B::R::O? DefaultOutput?
48
+ data_files : Vec < O :: Item > ,
46
49
}
47
50
48
- impl < B , L , F > RollingWriter < B , L , F >
51
+ impl < B , L , F , I , O > RollingWriter < B , L , F , I , O >
49
52
where
50
- B : IcebergWriterBuilder ,
53
+ B : IcebergWriterBuilder < I , O > ,
51
54
B :: R : CurrentFileStatus ,
52
55
L : LocationGenerator ,
53
56
F : FileNameGenerator ,
57
+ O : IntoIterator ,
58
+ O :: Item : Clone ,
54
59
{
55
60
/// Creates a new `RollingWriter` with the specified inner builder and target size.
56
61
///
@@ -108,12 +113,12 @@ where
108
113
Ok ( writer)
109
114
}
110
115
111
- /// Write a record batch to the current file, rolling over to a new file if necessary.
116
+ /// Write input data to the current file, rolling over to a new file if necessary.
112
117
///
113
118
/// # Arguments
114
119
///
115
- /// * `batch ` - The record batch to write
116
- pub async fn write ( & mut self , batch : RecordBatch ) -> Result < ( ) > {
120
+ /// * `input ` - The input data to write
121
+ pub async fn write ( & mut self , input : I ) -> Result < ( ) > {
117
122
if self . inner . is_none ( ) {
118
123
// initialize inner writer
119
124
self . inner = Some ( self . create_new_writer ( ) . await ?) ;
@@ -131,7 +136,7 @@ where
131
136
132
137
// write the input
133
138
if let Some ( writer) = & mut self . inner {
134
- writer. write ( batch ) . await
139
+ writer. write ( input ) . await
135
140
} else {
136
141
Err ( Error :: new (
137
142
ErrorKind :: Unexpected ,
@@ -140,8 +145,8 @@ where
140
145
}
141
146
}
142
147
143
- /// Close the writer and return the data files for all files.
144
- pub async fn close ( & mut self ) -> Result < Vec < DataFile > > {
148
+ /// Close the writer and return the output items for all files.
149
+ pub async fn close ( & mut self ) -> Result < Vec < O :: Item > > {
145
150
// close the current writer and merge the output
146
151
if let Some ( mut current_writer) = self . inner . take ( ) {
147
152
self . data_files . extend ( current_writer. close ( ) . await ?) ;
@@ -151,12 +156,13 @@ where
151
156
}
152
157
}
153
158
154
- impl < B , L , F > CurrentFileStatus for RollingWriter < B , L , F >
159
+ impl < B , L , F , I , O > CurrentFileStatus for RollingWriter < B , L , F , I , O >
155
160
where
156
- B : IcebergWriterBuilder ,
157
- B :: R : IcebergWriter + CurrentFileStatus ,
161
+ B : IcebergWriterBuilder < I , O > ,
162
+ B :: R : IcebergWriter < I , O > + CurrentFileStatus ,
158
163
L : LocationGenerator ,
159
164
F : FileNameGenerator ,
165
+ O : IntoIterator ,
160
166
{
161
167
fn current_file_path ( & self ) -> String {
162
168
if let Some ( inner) = & self . inner {
0 commit comments