forked from apache/iceberg-rust
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathreader.rs
More file actions
4012 lines (3479 loc) · 151 KB
/
reader.rs
File metadata and controls
4012 lines (3479 loc) · 151 KB
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
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you 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.
//! Parquet file data reader
use std::collections::{HashMap, HashSet};
use std::ops::Range;
use std::str::FromStr;
use std::sync::Arc;
use arrow_arith::boolean::{and, and_kleene, is_not_null, is_null, not, or, or_kleene};
use arrow_array::{
Array, ArrayRef, BooleanArray, Datum as ArrowDatum, Int32Array, RecordBatch, RunArray, Scalar,
StringArray,
};
use arrow_cast::cast::cast;
use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq};
use arrow_schema::{
ArrowError, DataType, Field, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef,
};
use arrow_string::like::starts_with;
use bytes::Bytes;
use fnv::FnvHashSet;
use futures::future::BoxFuture;
use futures::{FutureExt, StreamExt, TryFutureExt, TryStreamExt, try_join};
use parquet::arrow::arrow_reader::{
ArrowPredicateFn, ArrowReaderOptions, RowFilter, RowSelection, RowSelector,
};
use parquet::arrow::async_reader::AsyncFileReader;
use parquet::arrow::{PARQUET_FIELD_ID_META_KEY, ParquetRecordBatchStreamBuilder, ProjectionMask};
use parquet::file::metadata::{
PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData,
};
use parquet::schema::types::{SchemaDescriptor, Type as ParquetType};
use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader;
use crate::arrow::record_batch_transformer::RecordBatchTransformer;
use crate::arrow::{arrow_schema_to_schema, get_arrow_datum};
use crate::delete_vector::DeleteVector;
use crate::error::Result;
use crate::expr::visitors::bound_predicate_visitor::{BoundPredicateVisitor, visit};
use crate::expr::visitors::page_index_evaluator::PageIndexEvaluator;
use crate::expr::visitors::row_group_metrics_evaluator::RowGroupMetricsEvaluator;
use crate::expr::{BoundPredicate, BoundReference};
use crate::io::{FileIO, FileMetadata, FileRead};
use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskStream};
use crate::spec::{Datum, NestedField, PrimitiveType, Schema, Type};
use crate::utils::available_parallelism;
use crate::{Error, ErrorKind};
/// Reserved field ID for the file path (_file) column per Iceberg spec
pub(crate) const RESERVED_FIELD_ID_FILE: i32 = 2147483646;
/// Column name for the file path metadata column per Iceberg spec
pub(crate) const RESERVED_COL_NAME_FILE: &str = "_file";
/// Builder to create ArrowReader
pub struct ArrowReaderBuilder {
batch_size: Option<usize>,
file_io: FileIO,
concurrency_limit_data_files: usize,
row_group_filtering_enabled: bool,
row_selection_enabled: bool,
}
impl ArrowReaderBuilder {
/// Create a new ArrowReaderBuilder
pub fn new(file_io: FileIO) -> Self {
let num_cpus = available_parallelism().get();
ArrowReaderBuilder {
batch_size: None,
file_io,
concurrency_limit_data_files: num_cpus,
row_group_filtering_enabled: true,
row_selection_enabled: false,
}
}
/// Sets the max number of in flight data files that are being fetched
pub fn with_data_file_concurrency_limit(mut self, val: usize) -> Self {
self.concurrency_limit_data_files = val;
self
}
/// Sets the desired size of batches in the response
/// to something other than the default
pub fn with_batch_size(mut self, batch_size: usize) -> Self {
self.batch_size = Some(batch_size);
self
}
/// Determines whether to enable row group filtering.
pub fn with_row_group_filtering_enabled(mut self, row_group_filtering_enabled: bool) -> Self {
self.row_group_filtering_enabled = row_group_filtering_enabled;
self
}
/// Determines whether to enable row selection.
pub fn with_row_selection_enabled(mut self, row_selection_enabled: bool) -> Self {
self.row_selection_enabled = row_selection_enabled;
self
}
/// Build the ArrowReader.
pub fn build(self) -> ArrowReader {
ArrowReader {
batch_size: self.batch_size,
file_io: self.file_io.clone(),
delete_file_loader: CachingDeleteFileLoader::new(
self.file_io.clone(),
self.concurrency_limit_data_files,
),
concurrency_limit_data_files: self.concurrency_limit_data_files,
row_group_filtering_enabled: self.row_group_filtering_enabled,
row_selection_enabled: self.row_selection_enabled,
}
}
}
/// Reads data from Parquet files
#[derive(Clone)]
pub struct ArrowReader {
batch_size: Option<usize>,
file_io: FileIO,
delete_file_loader: CachingDeleteFileLoader,
/// the maximum number of data files that can be fetched at the same time
concurrency_limit_data_files: usize,
row_group_filtering_enabled: bool,
row_selection_enabled: bool,
}
impl ArrowReader {
/// Take a stream of FileScanTasks and reads all the files.
/// Returns a stream of Arrow RecordBatches containing the data from the files
pub fn read(self, tasks: FileScanTaskStream) -> Result<ArrowRecordBatchStream> {
let file_io = self.file_io.clone();
let batch_size = self.batch_size;
let concurrency_limit_data_files = self.concurrency_limit_data_files;
let row_group_filtering_enabled = self.row_group_filtering_enabled;
let row_selection_enabled = self.row_selection_enabled;
let stream = tasks
.map_ok(move |task| {
let file_io = file_io.clone();
Self::process_file_scan_task(
task,
batch_size,
file_io,
self.delete_file_loader.clone(),
row_group_filtering_enabled,
row_selection_enabled,
)
})
.map_err(|err| {
Error::new(ErrorKind::Unexpected, "file scan task generate failed").with_source(err)
})
.try_buffer_unordered(concurrency_limit_data_files)
.try_flatten_unordered(concurrency_limit_data_files);
Ok(Box::pin(stream) as ArrowRecordBatchStream)
}
#[allow(clippy::too_many_arguments)]
async fn process_file_scan_task(
task: FileScanTask,
batch_size: Option<usize>,
file_io: FileIO,
delete_file_loader: CachingDeleteFileLoader,
row_group_filtering_enabled: bool,
row_selection_enabled: bool,
) -> Result<ArrowRecordBatchStream> {
let should_load_page_index =
(row_selection_enabled && task.predicate.is_some()) || !task.deletes.is_empty();
let delete_filter_rx = delete_file_loader.load_deletes(&task.deletes, task.schema.clone());
// Migrated tables lack field IDs, requiring us to inspect the schema to choose
// between field-ID-based or position-based projection
let initial_stream_builder = Self::create_parquet_record_batch_stream_builder(
&task.data_file_path,
file_io.clone(),
should_load_page_index,
None,
)
.await?;
// Parquet files from Hive/Spark migrations lack field IDs in their metadata
let missing_field_ids = initial_stream_builder
.schema()
.fields()
.iter()
.next()
.is_some_and(|f| f.metadata().get(PARQUET_FIELD_ID_META_KEY).is_none());
// Adding position-based fallback IDs at schema level (not per-batch) enables projection
// on files that lack embedded field IDs. We recreate the builder to apply the modified schema.
let mut record_batch_stream_builder = if missing_field_ids {
let arrow_schema =
add_fallback_field_ids_to_arrow_schema(initial_stream_builder.schema());
let options = ArrowReaderOptions::new().with_schema(arrow_schema);
Self::create_parquet_record_batch_stream_builder(
&task.data_file_path,
file_io.clone(),
should_load_page_index,
Some(options),
)
.await?
} else {
initial_stream_builder
};
// Check if _file column is requested and filter it out for projection
let project_field_ids_without_virtual: Vec<i32> = task
.project_field_ids
.iter()
.filter_map(|&field_id| {
if field_id == RESERVED_FIELD_ID_FILE {
None
} else {
Some(field_id)
}
})
.collect();
// Fallback IDs don't match Parquet's embedded field IDs (since they don't exist),
// so we must use position-based projection instead of field-ID matching
let projection_mask = Self::get_arrow_projection_mask(
&project_field_ids_without_virtual,
&task.schema,
record_batch_stream_builder.parquet_schema(),
record_batch_stream_builder.schema(),
missing_field_ids, // Whether to use position-based (true) or field-ID-based (false) projection
)?;
record_batch_stream_builder =
record_batch_stream_builder.with_projection(projection_mask.clone());
// RecordBatchTransformer performs any transformations required on the RecordBatches
// that come back from the file, such as type promotion, default column insertion
// and column re-ordering
let mut record_batch_transformer =
RecordBatchTransformer::build(task.schema_ref(), &project_field_ids_without_virtual);
if let Some(batch_size) = batch_size {
record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size);
}
let delete_filter = delete_filter_rx.await.unwrap()?;
let delete_predicate = delete_filter.build_equality_delete_predicate(&task).await?;
// In addition to the optional predicate supplied in the `FileScanTask`,
// we also have an optional predicate resulting from equality delete files.
// If both are present, we logical-AND them together to form a single filter
// predicate that we can pass to the `RecordBatchStreamBuilder`.
let final_predicate = match (&task.predicate, delete_predicate) {
(None, None) => None,
(Some(predicate), None) => Some(predicate.clone()),
(None, Some(ref predicate)) => Some(predicate.clone()),
(Some(filter_predicate), Some(delete_predicate)) => {
Some(filter_predicate.clone().and(delete_predicate))
}
};
// There are three possible sources for potential lists of selected RowGroup indices,
// and two for `RowSelection`s.
// Selected RowGroup index lists can come from three sources:
// * When task.start and task.length specify a byte range (file splitting);
// * When there are equality delete files that are applicable;
// * When there is a scan predicate and row_group_filtering_enabled = true.
// `RowSelection`s can be created in either or both of the following cases:
// * When there are positional delete files that are applicable;
// * When there is a scan predicate and row_selection_enabled = true
// Note that row group filtering from predicates only happens when
// there is a scan predicate AND row_group_filtering_enabled = true,
// but we perform row selection filtering if there are applicable
// equality delete files OR (there is a scan predicate AND row_selection_enabled),
// since the only implemented method of applying positional deletes is
// by using a `RowSelection`.
let mut selected_row_group_indices = None;
let mut row_selection = None;
// Filter row groups based on byte range from task.start and task.length.
// If both start and length are 0, read the entire file (backwards compatibility).
if task.start != 0 || task.length != 0 {
let byte_range_filtered_row_groups = Self::filter_row_groups_by_byte_range(
record_batch_stream_builder.metadata(),
task.start,
task.length,
)?;
selected_row_group_indices = Some(byte_range_filtered_row_groups);
}
if let Some(predicate) = final_predicate {
let (iceberg_field_ids, field_id_map) = Self::build_field_id_set_and_map(
record_batch_stream_builder.parquet_schema(),
&predicate,
)?;
let row_filter = Self::get_row_filter(
&predicate,
record_batch_stream_builder.parquet_schema(),
&iceberg_field_ids,
&field_id_map,
)?;
record_batch_stream_builder = record_batch_stream_builder.with_row_filter(row_filter);
if row_group_filtering_enabled {
let predicate_filtered_row_groups = Self::get_selected_row_group_indices(
&predicate,
record_batch_stream_builder.metadata(),
&field_id_map,
&task.schema,
)?;
// Merge predicate-based filtering with byte range filtering (if present)
// by taking the intersection of both filters
selected_row_group_indices = match selected_row_group_indices {
Some(byte_range_filtered) => {
// Keep only row groups that are in both filters
let intersection: Vec<usize> = byte_range_filtered
.into_iter()
.filter(|idx| predicate_filtered_row_groups.contains(idx))
.collect();
Some(intersection)
}
None => Some(predicate_filtered_row_groups),
};
}
if row_selection_enabled {
row_selection = Some(Self::get_row_selection_for_filter_predicate(
&predicate,
record_batch_stream_builder.metadata(),
&selected_row_group_indices,
&field_id_map,
&task.schema,
)?);
}
}
let positional_delete_indexes = delete_filter.get_delete_vector(&task);
if let Some(positional_delete_indexes) = positional_delete_indexes {
let delete_row_selection = {
let positional_delete_indexes = positional_delete_indexes.lock().unwrap();
Self::build_deletes_row_selection(
record_batch_stream_builder.metadata().row_groups(),
&selected_row_group_indices,
&positional_delete_indexes,
)
}?;
// merge the row selection from the delete files with the row selection
// from the filter predicate, if there is one from the filter predicate
row_selection = match row_selection {
None => Some(delete_row_selection),
Some(filter_row_selection) => {
Some(filter_row_selection.intersection(&delete_row_selection))
}
};
}
if let Some(row_selection) = row_selection {
record_batch_stream_builder =
record_batch_stream_builder.with_row_selection(row_selection);
}
if let Some(selected_row_group_indices) = selected_row_group_indices {
record_batch_stream_builder =
record_batch_stream_builder.with_row_groups(selected_row_group_indices);
}
// Clone data_file_path for use in the closure
let data_file_path = task.data_file_path.clone();
// Build the batch stream and send all the RecordBatches that it generates
// to the requester.
let record_batch_stream =
record_batch_stream_builder
.build()?
.map(move |batch| match batch {
Ok(batch) => {
let mut processed_batch =
record_batch_transformer.process_record_batch(batch)?;
// Add the _file column at each requested position
// We insert them back at their original positions since we're reconstructing
// the original column order
for (position, field_id) in task.project_field_ids.iter().enumerate() {
if *field_id == RESERVED_FIELD_ID_FILE {
processed_batch = Self::add_file_path_column_ree_at_position(
processed_batch,
&data_file_path,
RESERVED_COL_NAME_FILE,
RESERVED_FIELD_ID_FILE,
position,
)?;
}
}
Ok(processed_batch)
}
Err(err) => Err(err.into()),
});
Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream)
}
pub(crate) async fn create_parquet_record_batch_stream_builder(
data_file_path: &str,
file_io: FileIO,
should_load_page_index: bool,
arrow_reader_options: Option<ArrowReaderOptions>,
) -> Result<ParquetRecordBatchStreamBuilder<ArrowFileReader<impl FileRead + Sized>>> {
// Get the metadata for the Parquet file we need to read and build
// a reader for the data within
let parquet_file = file_io.new_input(data_file_path)?;
let (parquet_metadata, parquet_reader) =
try_join!(parquet_file.metadata(), parquet_file.reader())?;
let parquet_file_reader = ArrowFileReader::new(parquet_metadata, parquet_reader)
.with_preload_column_index(true)
.with_preload_offset_index(true)
.with_preload_page_index(should_load_page_index);
// Create the record batch stream builder, which wraps the parquet file reader
let options = arrow_reader_options.unwrap_or_default();
let record_batch_stream_builder =
ParquetRecordBatchStreamBuilder::new_with_options(parquet_file_reader, options).await?;
Ok(record_batch_stream_builder)
}
/// computes a `RowSelection` from positional delete indices.
///
/// Using the Parquet page index, we build a `RowSelection` that rejects rows that are indicated
/// as having been deleted by a positional delete, taking into account any row groups that have
/// been skipped entirely by the filter predicate
fn build_deletes_row_selection(
row_group_metadata_list: &[RowGroupMetaData],
selected_row_groups: &Option<Vec<usize>>,
positional_deletes: &DeleteVector,
) -> Result<RowSelection> {
let mut results: Vec<RowSelector> = Vec::new();
let mut selected_row_groups_idx = 0;
let mut current_row_group_base_idx: u64 = 0;
let mut delete_vector_iter = positional_deletes.iter();
let mut next_deleted_row_idx_opt = delete_vector_iter.next();
for (idx, row_group_metadata) in row_group_metadata_list.iter().enumerate() {
let row_group_num_rows = row_group_metadata.num_rows() as u64;
let next_row_group_base_idx = current_row_group_base_idx + row_group_num_rows;
// if row group selection is enabled,
if let Some(selected_row_groups) = selected_row_groups {
// if we've consumed all the selected row groups, we're done
if selected_row_groups_idx == selected_row_groups.len() {
break;
}
if idx == selected_row_groups[selected_row_groups_idx] {
// we're in a selected row group. Increment selected_row_groups_idx
// so that next time around the for loop we're looking for the next
// selected row group
selected_row_groups_idx += 1;
} else {
// Advance iterator past all deletes in the skipped row group.
// advance_to() positions the iterator to the first delete >= next_row_group_base_idx.
// However, if our cached next_deleted_row_idx_opt is in the skipped range,
// we need to call next() to update the cache with the newly positioned value.
delete_vector_iter.advance_to(next_row_group_base_idx);
// Only update the cache if the cached value is stale (in the skipped range)
if let Some(cached_idx) = next_deleted_row_idx_opt {
if cached_idx < next_row_group_base_idx {
next_deleted_row_idx_opt = delete_vector_iter.next();
}
}
// still increment the current page base index but then skip to the next row group
// in the file
current_row_group_base_idx += row_group_num_rows;
continue;
}
}
let mut next_deleted_row_idx = match next_deleted_row_idx_opt {
Some(next_deleted_row_idx) => {
// if the index of the next deleted row is beyond this row group, add a selection for
// the remainder of this row group and skip to the next row group
if next_deleted_row_idx >= next_row_group_base_idx {
results.push(RowSelector::select(row_group_num_rows as usize));
current_row_group_base_idx += row_group_num_rows;
continue;
}
next_deleted_row_idx
}
// If there are no more pos deletes, add a selector for the entirety of this row group.
_ => {
results.push(RowSelector::select(row_group_num_rows as usize));
current_row_group_base_idx += row_group_num_rows;
continue;
}
};
let mut current_idx = current_row_group_base_idx;
'chunks: while next_deleted_row_idx < next_row_group_base_idx {
// `select` all rows that precede the next delete index
if current_idx < next_deleted_row_idx {
let run_length = next_deleted_row_idx - current_idx;
results.push(RowSelector::select(run_length as usize));
current_idx += run_length;
}
// `skip` all consecutive deleted rows in the current row group
let mut run_length = 0;
while next_deleted_row_idx == current_idx
&& next_deleted_row_idx < next_row_group_base_idx
{
run_length += 1;
current_idx += 1;
next_deleted_row_idx_opt = delete_vector_iter.next();
next_deleted_row_idx = match next_deleted_row_idx_opt {
Some(next_deleted_row_idx) => next_deleted_row_idx,
_ => {
// We've processed the final positional delete.
// Conclude the skip and then break so that we select the remaining
// rows in the row group and move on to the next row group
results.push(RowSelector::skip(run_length));
break 'chunks;
}
};
}
if run_length > 0 {
results.push(RowSelector::skip(run_length));
}
}
if current_idx < next_row_group_base_idx {
results.push(RowSelector::select(
(next_row_group_base_idx - current_idx) as usize,
));
}
current_row_group_base_idx += row_group_num_rows;
}
Ok(results.into())
}
/// Helper function to add a `_file` column to a RecordBatch at a specific position.
/// Takes the array, field to add, and position where to insert.
fn create_file_field_at_position(
batch: RecordBatch,
file_array: ArrayRef,
file_field: Field,
field_id: i32,
position: usize,
) -> Result<RecordBatch> {
let file_field_with_metadata = Arc::new(file_field.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
field_id.to_string(),
)])));
// Build columns vector in a single pass without insert
let original_columns = batch.columns();
let mut columns = Vec::with_capacity(original_columns.len() + 1);
columns.extend_from_slice(&original_columns[..position]);
columns.push(file_array);
columns.extend_from_slice(&original_columns[position..]);
// Build fields vector in a single pass without insert
let schema = batch.schema();
let original_fields = schema.fields();
let mut fields = Vec::with_capacity(original_fields.len() + 1);
fields.extend(original_fields[..position].iter().cloned());
fields.push(file_field_with_metadata);
fields.extend(original_fields[position..].iter().cloned());
let schema = Arc::new(ArrowSchema::new(fields));
RecordBatch::try_new(schema, columns).map_err(|e| {
Error::new(
ErrorKind::Unexpected,
"Failed to add _file column to RecordBatch",
)
.with_source(e)
})
}
/// Adds a `_file` column to the RecordBatch at a specific position.
/// Uses Run-End Encoding (REE) for maximum memory efficiency.
pub(crate) fn add_file_path_column_ree_at_position(
batch: RecordBatch,
file_path: &str,
field_name: &str,
field_id: i32,
position: usize,
) -> Result<RecordBatch> {
let num_rows = batch.num_rows();
// Use Run-End Encoded array for optimal memory efficiency
let run_ends = if num_rows == 0 {
Int32Array::from(Vec::<i32>::new())
} else {
Int32Array::from(vec![num_rows as i32])
};
let values = if num_rows == 0 {
StringArray::from(Vec::<&str>::new())
} else {
StringArray::from(vec![file_path])
};
let file_array = RunArray::try_new(&run_ends, &values).map_err(|e| {
Error::new(
ErrorKind::Unexpected,
"Failed to create RunArray for _file column",
)
.with_source(e)
})?;
let run_ends_field = Arc::new(Field::new("run_ends", DataType::Int32, false));
let values_field = Arc::new(Field::new("values", DataType::Utf8, true));
let file_field = Field::new(
field_name,
DataType::RunEndEncoded(run_ends_field, values_field),
false,
);
Self::create_file_field_at_position(
batch,
Arc::new(file_array),
file_field,
field_id,
position,
)
}
fn build_field_id_set_and_map(
parquet_schema: &SchemaDescriptor,
predicate: &BoundPredicate,
) -> Result<(HashSet<i32>, HashMap<i32, usize>)> {
// Collects all Iceberg field IDs referenced in the filter predicate
let mut collector = CollectFieldIdVisitor {
field_ids: HashSet::default(),
};
visit(&mut collector, predicate)?;
let iceberg_field_ids = collector.field_ids();
// Without embedded field IDs, we fall back to position-based mapping for compatibility
let field_id_map = match build_field_id_map(parquet_schema)? {
Some(map) => map,
None => build_fallback_field_id_map(parquet_schema),
};
Ok((iceberg_field_ids, field_id_map))
}
/// Recursively extract leaf field IDs because Parquet projection works at the leaf column level.
/// Nested types (struct/list/map) are flattened in Parquet's columnar format.
fn include_leaf_field_id(field: &NestedField, field_ids: &mut Vec<i32>) {
match field.field_type.as_ref() {
Type::Primitive(_) => {
field_ids.push(field.id);
}
Type::Struct(struct_type) => {
for nested_field in struct_type.fields() {
Self::include_leaf_field_id(nested_field, field_ids);
}
}
Type::List(list_type) => {
Self::include_leaf_field_id(&list_type.element_field, field_ids);
}
Type::Map(map_type) => {
Self::include_leaf_field_id(&map_type.key_field, field_ids);
Self::include_leaf_field_id(&map_type.value_field, field_ids);
}
}
}
fn get_arrow_projection_mask(
field_ids: &[i32],
iceberg_schema_of_task: &Schema,
parquet_schema: &SchemaDescriptor,
arrow_schema: &ArrowSchemaRef,
use_fallback: bool, // Whether file lacks embedded field IDs (e.g., migrated from Hive/Spark)
) -> Result<ProjectionMask> {
fn type_promotion_is_valid(
file_type: Option<&PrimitiveType>,
projected_type: Option<&PrimitiveType>,
) -> bool {
match (file_type, projected_type) {
(Some(lhs), Some(rhs)) if lhs == rhs => true,
(Some(PrimitiveType::Int), Some(PrimitiveType::Long)) => true,
(Some(PrimitiveType::Float), Some(PrimitiveType::Double)) => true,
(
Some(PrimitiveType::Decimal {
precision: file_precision,
scale: file_scale,
}),
Some(PrimitiveType::Decimal {
precision: requested_precision,
scale: requested_scale,
}),
) if requested_precision >= file_precision && file_scale == requested_scale => true,
// Uuid will be store as Fixed(16) in parquet file, so the read back type will be Fixed(16).
(Some(PrimitiveType::Fixed(16)), Some(PrimitiveType::Uuid)) => true,
_ => false,
}
}
if field_ids.is_empty() {
return Ok(ProjectionMask::all());
}
if use_fallback {
// Position-based projection necessary because file lacks embedded field IDs
Self::get_arrow_projection_mask_fallback(field_ids, parquet_schema)
} else {
// Field-ID-based projection using embedded field IDs from Parquet metadata
// Parquet's columnar format requires leaf-level (not top-level struct/list/map) projection
let mut leaf_field_ids = vec![];
for field_id in field_ids {
let field = iceberg_schema_of_task.field_by_id(*field_id);
if let Some(field) = field {
Self::include_leaf_field_id(field, &mut leaf_field_ids);
}
}
Self::get_arrow_projection_mask_with_field_ids(
&leaf_field_ids,
iceberg_schema_of_task,
parquet_schema,
arrow_schema,
type_promotion_is_valid,
)
}
}
/// Standard projection using embedded field IDs from Parquet metadata.
/// For iceberg-java compatibility with ParquetSchemaUtil.pruneColumns().
fn get_arrow_projection_mask_with_field_ids(
leaf_field_ids: &[i32],
iceberg_schema_of_task: &Schema,
parquet_schema: &SchemaDescriptor,
arrow_schema: &ArrowSchemaRef,
type_promotion_is_valid: fn(Option<&PrimitiveType>, Option<&PrimitiveType>) -> bool,
) -> Result<ProjectionMask> {
let mut column_map = HashMap::new();
let fields = arrow_schema.fields();
// Pre-project only the fields that have been selected, possibly avoiding converting
// some Arrow types that are not yet supported.
let mut projected_fields: HashMap<FieldRef, i32> = HashMap::new();
let projected_arrow_schema = ArrowSchema::new_with_metadata(
fields.filter_leaves(|_, f| {
f.metadata()
.get(PARQUET_FIELD_ID_META_KEY)
.and_then(|field_id| i32::from_str(field_id).ok())
.is_some_and(|field_id| {
projected_fields.insert((*f).clone(), field_id);
leaf_field_ids.contains(&field_id)
})
}),
arrow_schema.metadata().clone(),
);
let iceberg_schema = arrow_schema_to_schema(&projected_arrow_schema)?;
fields.filter_leaves(|idx, field| {
let Some(field_id) = projected_fields.get(field).cloned() else {
return false;
};
let iceberg_field = iceberg_schema_of_task.field_by_id(field_id);
let parquet_iceberg_field = iceberg_schema.field_by_id(field_id);
if iceberg_field.is_none() || parquet_iceberg_field.is_none() {
return false;
}
if !type_promotion_is_valid(
parquet_iceberg_field
.unwrap()
.field_type
.as_primitive_type(),
iceberg_field.unwrap().field_type.as_primitive_type(),
) {
return false;
}
column_map.insert(field_id, idx);
true
});
// Schema evolution: New columns may not exist in old Parquet files.
// We only project existing columns; RecordBatchTransformer adds default/NULL values.
let mut indices = vec![];
for field_id in leaf_field_ids {
if let Some(col_idx) = column_map.get(field_id) {
indices.push(*col_idx);
}
}
if indices.is_empty() {
// Edge case: All requested columns are new (don't exist in file).
// Project all columns so RecordBatchTransformer has a batch to transform.
Ok(ProjectionMask::all())
} else {
Ok(ProjectionMask::leaves(parquet_schema, indices))
}
}
/// Fallback projection for Parquet files without field IDs.
/// Uses position-based matching: field ID N → column position N-1.
/// Projects entire top-level columns (including nested content) for iceberg-java compatibility.
fn get_arrow_projection_mask_fallback(
field_ids: &[i32],
parquet_schema: &SchemaDescriptor,
) -> Result<ProjectionMask> {
// Position-based: field_id N → column N-1 (field IDs are 1-indexed)
let parquet_root_fields = parquet_schema.root_schema().get_fields();
let mut root_indices = vec![];
for field_id in field_ids.iter() {
let parquet_pos = (*field_id - 1) as usize;
if parquet_pos < parquet_root_fields.len() {
root_indices.push(parquet_pos);
}
// RecordBatchTransformer adds missing columns with NULL values
}
if root_indices.is_empty() {
Ok(ProjectionMask::all())
} else {
Ok(ProjectionMask::roots(parquet_schema, root_indices))
}
}
fn get_row_filter(
predicates: &BoundPredicate,
parquet_schema: &SchemaDescriptor,
iceberg_field_ids: &HashSet<i32>,
field_id_map: &HashMap<i32, usize>,
) -> Result<RowFilter> {
// Collect Parquet column indices from field ids.
// If the field id is not found in Parquet schema, it will be ignored due to schema evolution.
let mut column_indices = iceberg_field_ids
.iter()
.filter_map(|field_id| field_id_map.get(field_id).cloned())
.collect::<Vec<_>>();
column_indices.sort();
// The converter that converts `BoundPredicates` to `ArrowPredicates`
let mut converter = PredicateConverter {
parquet_schema,
column_map: field_id_map,
column_indices: &column_indices,
};
// After collecting required leaf column indices used in the predicate,
// creates the projection mask for the Arrow predicates.
let projection_mask = ProjectionMask::leaves(parquet_schema, column_indices.clone());
let predicate_func = visit(&mut converter, predicates)?;
let arrow_predicate = ArrowPredicateFn::new(projection_mask, predicate_func);
Ok(RowFilter::new(vec![Box::new(arrow_predicate)]))
}
fn get_selected_row_group_indices(
predicate: &BoundPredicate,
parquet_metadata: &Arc<ParquetMetaData>,
field_id_map: &HashMap<i32, usize>,
snapshot_schema: &Schema,
) -> Result<Vec<usize>> {
let row_groups_metadata = parquet_metadata.row_groups();
let mut results = Vec::with_capacity(row_groups_metadata.len());
for (idx, row_group_metadata) in row_groups_metadata.iter().enumerate() {
if RowGroupMetricsEvaluator::eval(
predicate,
row_group_metadata,
field_id_map,
snapshot_schema,
)? {
results.push(idx);
}
}
Ok(results)
}
fn get_row_selection_for_filter_predicate(
predicate: &BoundPredicate,
parquet_metadata: &Arc<ParquetMetaData>,
selected_row_groups: &Option<Vec<usize>>,
field_id_map: &HashMap<i32, usize>,
snapshot_schema: &Schema,
) -> Result<RowSelection> {
let Some(column_index) = parquet_metadata.column_index() else {
return Err(Error::new(
ErrorKind::Unexpected,
"Parquet file metadata does not contain a column index",
));
};
let Some(offset_index) = parquet_metadata.offset_index() else {
return Err(Error::new(
ErrorKind::Unexpected,
"Parquet file metadata does not contain an offset index",
));
};
// If all row groups were filtered out, return an empty RowSelection (select no rows)
if let Some(selected_row_groups) = selected_row_groups {
if selected_row_groups.is_empty() {
return Ok(RowSelection::from(Vec::new()));
}
}
let mut selected_row_groups_idx = 0;
let page_index = column_index
.iter()
.enumerate()
.zip(offset_index)
.zip(parquet_metadata.row_groups());
let mut results = Vec::new();
for (((idx, column_index), offset_index), row_group_metadata) in page_index {
if let Some(selected_row_groups) = selected_row_groups {
// skip row groups that aren't present in selected_row_groups
if idx == selected_row_groups[selected_row_groups_idx] {
selected_row_groups_idx += 1;
} else {
continue;
}
}
let selections_for_page = PageIndexEvaluator::eval(
predicate,
column_index,
offset_index,
row_group_metadata,
field_id_map,
snapshot_schema,
)?;
results.push(selections_for_page);
if let Some(selected_row_groups) = selected_row_groups {
if selected_row_groups_idx == selected_row_groups.len() {
break;
}
}
}
Ok(results.into_iter().flatten().collect::<Vec<_>>().into())
}
/// Filters row groups by byte range to support Iceberg's file splitting.
///
/// Iceberg splits large files at row group boundaries, so we only read row groups
/// whose byte ranges overlap with [start, start+length).
fn filter_row_groups_by_byte_range(
parquet_metadata: &Arc<ParquetMetaData>,
start: u64,
length: u64,
) -> Result<Vec<usize>> {
let row_groups = parquet_metadata.row_groups();
let mut selected = Vec::new();
let end = start + length;
// Row groups are stored sequentially after the 4-byte magic header.
let mut current_byte_offset = 4u64;
for (idx, row_group) in row_groups.iter().enumerate() {
let row_group_size = row_group.compressed_size() as u64;
let row_group_end = current_byte_offset + row_group_size;