12
12
// See the License for the specific language governing permissions and
13
13
// limitations under the License.
14
14
15
- use std:: sync:: Arc ;
16
-
17
15
use futures_async_stream:: try_stream;
18
16
use futures_util:: stream:: StreamExt ;
19
17
use itertools:: Itertools ;
20
- use risingwave_common:: array:: arrow:: IcebergArrowConvert ;
21
- use risingwave_common:: array:: { ArrayImpl , DataChunk , I64Array , Utf8Array } ;
18
+ use risingwave_common:: array:: DataChunk ;
22
19
use risingwave_common:: catalog:: {
23
20
Field , Schema , ICEBERG_FILE_PATH_COLUMN_NAME , ICEBERG_SEQUENCE_NUM_COLUMN_NAME ,
24
21
} ;
25
22
use risingwave_common:: types:: { DataType , ScalarImpl } ;
26
- use risingwave_common_estimate_size:: EstimateSize ;
27
- use risingwave_connector:: source:: iceberg:: { IcebergFileScanTask , IcebergProperties , IcebergSplit } ;
23
+ use risingwave_connector:: source:: iceberg:: {
24
+ scan_task_to_chunk, IcebergFileScanTask , IcebergProperties , IcebergScanOpts , IcebergSplit ,
25
+ } ;
28
26
use risingwave_connector:: source:: { ConnectorProperties , SplitImpl , SplitMetaData } ;
29
27
use risingwave_connector:: WithOptionsSecResolved ;
30
28
use risingwave_expr:: expr:: LiteralExpression ;
@@ -92,7 +90,6 @@ impl IcebergScanExecutor {
92
90
async fn do_execute ( mut self : Box < Self > ) {
93
91
let table = self . iceberg_config . load_table ( ) . await ?;
94
92
let data_types = self . schema . data_types ( ) ;
95
- let table_name = table. identifier ( ) . name ( ) . to_owned ( ) ;
96
93
97
94
let data_file_scan_tasks = match Option :: take ( & mut self . file_scan_tasks ) {
98
95
Some ( IcebergFileScanTask :: Data ( data_file_scan_tasks) ) => data_file_scan_tasks,
@@ -110,58 +107,20 @@ impl IcebergScanExecutor {
110
107
}
111
108
} ;
112
109
113
- let mut read_bytes = 0 ;
114
- let _metrics_report_guard = scopeguard:: guard (
115
- ( read_bytes, table_name, self . metrics . clone ( ) ) ,
116
- |( read_bytes, table_name, metrics) | {
117
- if let Some ( metrics) = metrics {
118
- metrics
119
- . iceberg_scan_metrics ( )
120
- . iceberg_read_bytes
121
- . with_guarded_label_values ( & [ & table_name] )
122
- . inc_by ( read_bytes as _ ) ;
123
- }
124
- } ,
125
- ) ;
126
110
for data_file_scan_task in data_file_scan_tasks {
127
- let data_file_path = data_file_scan_task. data_file_path . clone ( ) ;
128
- let data_sequence_number = data_file_scan_task. sequence_number ;
129
-
130
- let reader = table
131
- . reader_builder ( )
132
- . with_batch_size ( self . batch_size )
133
- . build ( ) ;
134
- let file_scan_stream = tokio_stream:: once ( Ok ( data_file_scan_task) ) ;
135
-
136
- let mut record_batch_stream =
137
- reader. read ( Box :: pin ( file_scan_stream) ) . await ?. enumerate ( ) ;
138
-
139
- while let Some ( ( index, record_batch) ) = record_batch_stream. next ( ) . await {
140
- let record_batch = record_batch?;
141
-
142
- // iceberg_t1_source
143
- let mut chunk = IcebergArrowConvert . chunk_from_record_batch ( & record_batch) ?;
144
- if self . need_seq_num {
145
- let ( mut columns, visibility) = chunk. into_parts ( ) ;
146
- columns. push ( Arc :: new ( ArrayImpl :: Int64 ( I64Array :: from_iter (
147
- vec ! [ data_sequence_number; visibility. len( ) ] ,
148
- ) ) ) ) ;
149
- chunk = DataChunk :: from_parts ( columns. into ( ) , visibility)
150
- } ;
151
- if self . need_file_path_and_pos {
152
- let ( mut columns, visibility) = chunk. into_parts ( ) ;
153
- columns. push ( Arc :: new ( ArrayImpl :: Utf8 ( Utf8Array :: from_iter (
154
- vec ! [ data_file_path. as_str( ) ; visibility. len( ) ] ,
155
- ) ) ) ) ;
156
- let index_start = ( index * self . batch_size ) as i64 ;
157
- columns. push ( Arc :: new ( ArrayImpl :: Int64 ( I64Array :: from_iter (
158
- ( index_start..( index_start + visibility. len ( ) as i64 ) )
159
- . collect :: < Vec < i64 > > ( ) ,
160
- ) ) ) ) ;
161
- chunk = DataChunk :: from_parts ( columns. into ( ) , visibility)
162
- }
111
+ #[ for_await]
112
+ for chunk in scan_task_to_chunk (
113
+ table. clone ( ) ,
114
+ data_file_scan_task,
115
+ IcebergScanOpts {
116
+ batch_size : self . batch_size ,
117
+ need_seq_num : self . need_seq_num ,
118
+ need_file_path_and_pos : self . need_file_path_and_pos ,
119
+ } ,
120
+ self . metrics . as_ref ( ) . map ( |m| m. iceberg_scan_metrics ( ) ) ,
121
+ ) {
122
+ let chunk = chunk?;
163
123
assert_eq ! ( chunk. data_types( ) , data_types) ;
164
- read_bytes += chunk. estimated_heap_size ( ) as u64 ;
165
124
yield chunk;
166
125
}
167
126
}
0 commit comments