-
Notifications
You must be signed in to change notification settings - Fork 833
/
filter.rs
2047 lines (1760 loc) · 72.3 KB
/
filter.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
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.
//! Defines filter kernels
use std::ops::AddAssign;
use std::sync::Arc;
use arrow_array::builder::BooleanBufferBuilder;
use arrow_array::cast::AsArray;
use arrow_array::types::{
ArrowDictionaryKeyType, ArrowPrimitiveType, ByteArrayType, ByteViewType, RunEndIndexType,
};
use arrow_array::*;
use arrow_buffer::{bit_util, ArrowNativeType, BooleanBuffer, NullBuffer, RunEndBuffer};
use arrow_buffer::{Buffer, MutableBuffer};
use arrow_data::bit_iterator::{BitIndexIterator, BitSliceIterator};
use arrow_data::transform::MutableArrayData;
use arrow_data::{ArrayData, ArrayDataBuilder};
use arrow_schema::*;
/// If the filter selects more than this fraction of rows, use
/// [`SlicesIterator`] to copy ranges of values. Otherwise iterate
/// over individual rows using [`IndexIterator`]
///
/// Threshold of 0.8 chosen based on <https://dl.acm.org/doi/abs/10.1145/3465998.3466009>
///
const FILTER_SLICES_SELECTIVITY_THRESHOLD: f64 = 0.8;
/// An iterator of `(usize, usize)` each representing an interval
/// `[start, end)` whose slots of a bitmap [Buffer] are true.
///
/// Each interval corresponds to a contiguous region of memory to be
/// "taken" from an array to be filtered.
///
/// ## Notes:
///
/// 1. Ignores the validity bitmap (ignores nulls)
///
/// 2. Only performant for filters that copy across long contiguous runs
#[derive(Debug)]
pub struct SlicesIterator<'a>(BitSliceIterator<'a>);
impl<'a> SlicesIterator<'a> {
/// Creates a new iterator from a [BooleanArray]
pub fn new(filter: &'a BooleanArray) -> Self {
Self(filter.values().set_slices())
}
}
impl Iterator for SlicesIterator<'_> {
type Item = (usize, usize);
fn next(&mut self) -> Option<Self::Item> {
self.0.next()
}
}
/// An iterator of `usize` whose index in [`BooleanArray`] is true
///
/// This provides the best performance on most predicates, apart from those which keep
/// large runs and therefore favour [`SlicesIterator`]
struct IndexIterator<'a> {
remaining: usize,
iter: BitIndexIterator<'a>,
}
impl<'a> IndexIterator<'a> {
fn new(filter: &'a BooleanArray, remaining: usize) -> Self {
assert_eq!(filter.null_count(), 0);
let iter = filter.values().set_indices();
Self { remaining, iter }
}
}
impl Iterator for IndexIterator<'_> {
type Item = usize;
fn next(&mut self) -> Option<Self::Item> {
if self.remaining != 0 {
// Fascinatingly swapping these two lines around results in a 50%
// performance regression for some benchmarks
let next = self.iter.next().expect("IndexIterator exhausted early");
self.remaining -= 1;
// Must panic if exhausted early as trusted length iterator
return Some(next);
}
None
}
fn size_hint(&self) -> (usize, Option<usize>) {
(self.remaining, Some(self.remaining))
}
}
/// Counts the number of set bits in `filter`
fn filter_count(filter: &BooleanArray) -> usize {
filter.values().count_set_bits()
}
/// Function that can filter arbitrary arrays
///
/// Deprecated: Use [`FilterPredicate`] instead
#[deprecated]
pub type Filter<'a> = Box<dyn Fn(&ArrayData) -> ArrayData + 'a>;
/// Returns a prepared function optimized to filter multiple arrays.
///
/// Creating this function requires time, but using it is faster than [filter] when the
/// same filter needs to be applied to multiple arrays (e.g. a multi-column `RecordBatch`).
/// WARNING: the nulls of `filter` are ignored and the value on its slot is considered.
/// Therefore, it is considered undefined behavior to pass `filter` with null values.
///
/// Deprecated: Use [`FilterBuilder`] instead
#[deprecated]
#[allow(deprecated)]
pub fn build_filter(filter: &BooleanArray) -> Result<Filter, ArrowError> {
let iter = SlicesIterator::new(filter);
let filter_count = filter_count(filter);
let chunks = iter.collect::<Vec<_>>();
Ok(Box::new(move |array: &ArrayData| {
match filter_count {
// return all
len if len == array.len() => array.clone(),
0 => ArrayData::new_empty(array.data_type()),
_ => {
let mut mutable = MutableArrayData::new(vec![array], false, filter_count);
chunks
.iter()
.for_each(|(start, end)| mutable.extend(0, *start, *end));
mutable.freeze()
}
}
}))
}
/// Remove null values by do a bitmask AND operation with null bits and the boolean bits.
pub fn prep_null_mask_filter(filter: &BooleanArray) -> BooleanArray {
let nulls = filter.nulls().unwrap();
let mask = filter.values() & nulls.inner();
BooleanArray::new(mask, None)
}
/// Returns a filtered `values` [Array] where the corresponding elements of
/// `predicate` are `true`.
///
/// See also [`FilterBuilder`] for more control over the filtering process.
///
/// # Example
/// ```rust
/// # use arrow_array::{Int32Array, BooleanArray};
/// # use arrow_select::filter::filter;
/// let array = Int32Array::from(vec![5, 6, 7, 8, 9]);
/// let filter_array = BooleanArray::from(vec![true, false, false, true, false]);
/// let c = filter(&array, &filter_array).unwrap();
/// let c = c.as_any().downcast_ref::<Int32Array>().unwrap();
/// assert_eq!(c, &Int32Array::from(vec![5, 8]));
/// ```
pub fn filter(values: &dyn Array, predicate: &BooleanArray) -> Result<ArrayRef, ArrowError> {
let mut filter_builder = FilterBuilder::new(predicate);
if multiple_arrays(values.data_type()) {
// Only optimize if filtering more than one array
// Otherwise, the overhead of optimization can be more than the benefit
filter_builder = filter_builder.optimize();
}
let predicate = filter_builder.build();
filter_array(values, &predicate)
}
fn multiple_arrays(data_type: &DataType) -> bool {
match data_type {
DataType::Struct(fields) => {
fields.len() > 1 || fields.len() == 1 && multiple_arrays(fields[0].data_type())
}
DataType::Union(fields, UnionMode::Sparse) => !fields.is_empty(),
_ => false,
}
}
/// Returns a filtered [RecordBatch] where the corresponding elements of
/// `predicate` are true.
///
/// This is the equivalent of calling [filter] on each column of the [RecordBatch].
pub fn filter_record_batch(
record_batch: &RecordBatch,
predicate: &BooleanArray,
) -> Result<RecordBatch, ArrowError> {
let mut filter_builder = FilterBuilder::new(predicate);
if record_batch.num_columns() > 1 {
// Only optimize if filtering more than one column
// Otherwise, the overhead of optimization can be more than the benefit
filter_builder = filter_builder.optimize();
}
let filter = filter_builder.build();
let filtered_arrays = record_batch
.columns()
.iter()
.map(|a| filter_array(a, &filter))
.collect::<Result<Vec<_>, _>>()?;
let options = RecordBatchOptions::default().with_row_count(Some(filter.count()));
RecordBatch::try_new_with_options(record_batch.schema(), filtered_arrays, &options)
}
/// A builder to construct [`FilterPredicate`]
#[derive(Debug)]
pub struct FilterBuilder {
filter: BooleanArray,
count: usize,
strategy: IterationStrategy,
}
impl FilterBuilder {
/// Create a new [`FilterBuilder`] that can be used to construct a [`FilterPredicate`]
pub fn new(filter: &BooleanArray) -> Self {
let filter = match filter.null_count() {
0 => filter.clone(),
_ => prep_null_mask_filter(filter),
};
let count = filter_count(&filter);
let strategy = IterationStrategy::default_strategy(filter.len(), count);
Self {
filter,
count,
strategy,
}
}
/// Compute an optimised representation of the provided `filter` mask that can be
/// applied to an array more quickly.
///
/// Note: There is limited benefit to calling this to then filter a single array
/// Note: This will likely have a larger memory footprint than the original mask
pub fn optimize(mut self) -> Self {
match self.strategy {
IterationStrategy::SlicesIterator => {
let slices = SlicesIterator::new(&self.filter).collect();
self.strategy = IterationStrategy::Slices(slices)
}
IterationStrategy::IndexIterator => {
let indices = IndexIterator::new(&self.filter, self.count).collect();
self.strategy = IterationStrategy::Indices(indices)
}
_ => {}
}
self
}
/// Construct the final `FilterPredicate`
pub fn build(self) -> FilterPredicate {
FilterPredicate {
filter: self.filter,
count: self.count,
strategy: self.strategy,
}
}
}
/// The iteration strategy used to evaluate [`FilterPredicate`]
#[derive(Debug)]
enum IterationStrategy {
/// A lazily evaluated iterator of ranges
SlicesIterator,
/// A lazily evaluated iterator of indices
IndexIterator,
/// A precomputed list of indices
Indices(Vec<usize>),
/// A precomputed array of ranges
Slices(Vec<(usize, usize)>),
/// Select all rows
All,
/// Select no rows
None,
}
impl IterationStrategy {
/// The default [`IterationStrategy`] for a filter of length `filter_length`
/// and selecting `filter_count` rows
fn default_strategy(filter_length: usize, filter_count: usize) -> Self {
if filter_length == 0 || filter_count == 0 {
return IterationStrategy::None;
}
if filter_count == filter_length {
return IterationStrategy::All;
}
// Compute the selectivity of the predicate by dividing the number of true
// bits in the predicate by the predicate's total length
//
// This can then be used as a heuristic for the optimal iteration strategy
let selectivity_frac = filter_count as f64 / filter_length as f64;
if selectivity_frac > FILTER_SLICES_SELECTIVITY_THRESHOLD {
return IterationStrategy::SlicesIterator;
}
IterationStrategy::IndexIterator
}
}
/// A filtering predicate that can be applied to an [`Array`]
#[derive(Debug)]
pub struct FilterPredicate {
filter: BooleanArray,
count: usize,
strategy: IterationStrategy,
}
impl FilterPredicate {
/// Selects rows from `values` based on this [`FilterPredicate`]
pub fn filter(&self, values: &dyn Array) -> Result<ArrayRef, ArrowError> {
filter_array(values, self)
}
/// Number of rows being selected based on this [`FilterPredicate`]
pub fn count(&self) -> usize {
self.count
}
}
fn filter_array(values: &dyn Array, predicate: &FilterPredicate) -> Result<ArrayRef, ArrowError> {
if predicate.filter.len() > values.len() {
return Err(ArrowError::InvalidArgumentError(format!(
"Filter predicate of length {} is larger than target array of length {}",
predicate.filter.len(),
values.len()
)));
}
match predicate.strategy {
IterationStrategy::None => Ok(new_empty_array(values.data_type())),
IterationStrategy::All => Ok(values.slice(0, predicate.count)),
// actually filter
_ => downcast_primitive_array! {
values => Ok(Arc::new(filter_primitive(values, predicate))),
DataType::Boolean => {
let values = values.as_any().downcast_ref::<BooleanArray>().unwrap();
Ok(Arc::new(filter_boolean(values, predicate)))
}
DataType::Utf8 => {
Ok(Arc::new(filter_bytes(values.as_string::<i32>(), predicate)))
}
DataType::LargeUtf8 => {
Ok(Arc::new(filter_bytes(values.as_string::<i64>(), predicate)))
}
DataType::Utf8View => {
Ok(Arc::new(filter_byte_view(values.as_string_view(), predicate)))
}
DataType::Binary => {
Ok(Arc::new(filter_bytes(values.as_binary::<i32>(), predicate)))
}
DataType::LargeBinary => {
Ok(Arc::new(filter_bytes(values.as_binary::<i64>(), predicate)))
}
DataType::BinaryView => {
Ok(Arc::new(filter_byte_view(values.as_binary_view(), predicate)))
}
DataType::FixedSizeBinary(_) => {
Ok(Arc::new(filter_fixed_size_binary(values.as_fixed_size_binary(), predicate)))
}
DataType::RunEndEncoded(_, _) => {
downcast_run_array!{
values => Ok(Arc::new(filter_run_end_array(values, predicate)?)),
t => unimplemented!("Filter not supported for RunEndEncoded type {:?}", t)
}
}
DataType::Dictionary(_, _) => downcast_dictionary_array! {
values => Ok(Arc::new(filter_dict(values, predicate))),
t => unimplemented!("Filter not supported for dictionary type {:?}", t)
}
DataType::Struct(_) => {
Ok(Arc::new(filter_struct(values.as_struct(), predicate)?))
}
DataType::Union(_, UnionMode::Sparse) => {
Ok(Arc::new(filter_sparse_union(values.as_union(), predicate)?))
}
_ => {
let data = values.to_data();
// fallback to using MutableArrayData
let mut mutable = MutableArrayData::new(
vec![&data],
false,
predicate.count,
);
match &predicate.strategy {
IterationStrategy::Slices(slices) => {
slices
.iter()
.for_each(|(start, end)| mutable.extend(0, *start, *end));
}
_ => {
let iter = SlicesIterator::new(&predicate.filter);
iter.for_each(|(start, end)| mutable.extend(0, start, end));
}
}
let data = mutable.freeze();
Ok(make_array(data))
}
},
}
}
/// Filter any supported [`RunArray`] based on a [`FilterPredicate`]
fn filter_run_end_array<R: RunEndIndexType>(
re_arr: &RunArray<R>,
pred: &FilterPredicate,
) -> Result<RunArray<R>, ArrowError>
where
R::Native: Into<i64> + From<bool>,
R::Native: AddAssign,
{
let run_ends: &RunEndBuffer<R::Native> = re_arr.run_ends();
let mut values_filter = BooleanBufferBuilder::new(run_ends.len());
let mut new_run_ends = vec![R::default_value(); run_ends.len()];
let mut start = 0i64;
let mut i = 0;
let mut count = R::default_value();
let filter_values = pred.filter.values();
for end in run_ends.inner().into_iter().map(|i| (*i).into()) {
let mut keep = false;
for pred in filter_values
.iter()
.skip(start as usize)
.take((end - start) as usize)
{
count += R::Native::from(pred);
keep |= pred
}
// this is to avoid branching
new_run_ends[i] = count;
i += keep as usize;
values_filter.append(keep);
start = end;
}
new_run_ends.truncate(i);
if values_filter.is_empty() {
new_run_ends.clear();
}
let values = re_arr.values();
let pred = BooleanArray::new(values_filter.finish(), None);
let values = filter(&values, &pred)?;
let run_ends = PrimitiveArray::<R>::new(new_run_ends.into(), None);
RunArray::try_new(&run_ends, &values)
}
/// Computes a new null mask for `data` based on `predicate`
///
/// If the predicate selected no null-rows, returns `None`, otherwise returns
/// `Some((null_count, null_buffer))` where `null_count` is the number of nulls
/// in the filtered output, and `null_buffer` is the filtered null buffer
///
fn filter_null_mask(
nulls: Option<&NullBuffer>,
predicate: &FilterPredicate,
) -> Option<(usize, Buffer)> {
let nulls = nulls?;
if nulls.null_count() == 0 {
return None;
}
let nulls = filter_bits(nulls.inner(), predicate);
// The filtered `nulls` has a length of `predicate.count` bits and
// therefore the null count is this minus the number of valid bits
let null_count = predicate.count - nulls.count_set_bits_offset(0, predicate.count);
if null_count == 0 {
return None;
}
Some((null_count, nulls))
}
/// Filter the packed bitmask `buffer`, with `predicate` starting at bit offset `offset`
fn filter_bits(buffer: &BooleanBuffer, predicate: &FilterPredicate) -> Buffer {
let src = buffer.values();
let offset = buffer.offset();
match &predicate.strategy {
IterationStrategy::IndexIterator => {
let bits = IndexIterator::new(&predicate.filter, predicate.count)
.map(|src_idx| bit_util::get_bit(src, src_idx + offset));
// SAFETY: `IndexIterator` reports its size correctly
unsafe { MutableBuffer::from_trusted_len_iter_bool(bits).into() }
}
IterationStrategy::Indices(indices) => {
let bits = indices
.iter()
.map(|src_idx| bit_util::get_bit(src, *src_idx + offset));
// SAFETY: `Vec::iter()` reports its size correctly
unsafe { MutableBuffer::from_trusted_len_iter_bool(bits).into() }
}
IterationStrategy::SlicesIterator => {
let mut builder = BooleanBufferBuilder::new(bit_util::ceil(predicate.count, 8));
for (start, end) in SlicesIterator::new(&predicate.filter) {
builder.append_packed_range(start + offset..end + offset, src)
}
builder.into()
}
IterationStrategy::Slices(slices) => {
let mut builder = BooleanBufferBuilder::new(bit_util::ceil(predicate.count, 8));
for (start, end) in slices {
builder.append_packed_range(*start + offset..*end + offset, src)
}
builder.into()
}
IterationStrategy::All | IterationStrategy::None => unreachable!(),
}
}
/// `filter` implementation for boolean buffers
fn filter_boolean(array: &BooleanArray, predicate: &FilterPredicate) -> BooleanArray {
let values = filter_bits(array.values(), predicate);
let mut builder = ArrayDataBuilder::new(DataType::Boolean)
.len(predicate.count)
.add_buffer(values);
if let Some((null_count, nulls)) = filter_null_mask(array.nulls(), predicate) {
builder = builder.null_count(null_count).null_bit_buffer(Some(nulls));
}
let data = unsafe { builder.build_unchecked() };
BooleanArray::from(data)
}
#[inline(never)]
fn filter_native<T: ArrowNativeType>(values: &[T], predicate: &FilterPredicate) -> Buffer {
assert!(values.len() >= predicate.filter.len());
let buffer = match &predicate.strategy {
IterationStrategy::SlicesIterator => {
let mut buffer = MutableBuffer::with_capacity(predicate.count * T::get_byte_width());
for (start, end) in SlicesIterator::new(&predicate.filter) {
buffer.extend_from_slice(&values[start..end]);
}
buffer
}
IterationStrategy::Slices(slices) => {
let mut buffer = MutableBuffer::with_capacity(predicate.count * T::get_byte_width());
for (start, end) in slices {
buffer.extend_from_slice(&values[*start..*end]);
}
buffer
}
IterationStrategy::IndexIterator => {
let iter = IndexIterator::new(&predicate.filter, predicate.count).map(|x| values[x]);
// SAFETY: IndexIterator is trusted length
unsafe { MutableBuffer::from_trusted_len_iter(iter) }
}
IterationStrategy::Indices(indices) => {
let iter = indices.iter().map(|x| values[*x]);
// SAFETY: `Vec::iter` is trusted length
unsafe { MutableBuffer::from_trusted_len_iter(iter) }
}
IterationStrategy::All | IterationStrategy::None => unreachable!(),
};
buffer.into()
}
/// `filter` implementation for primitive arrays
fn filter_primitive<T>(array: &PrimitiveArray<T>, predicate: &FilterPredicate) -> PrimitiveArray<T>
where
T: ArrowPrimitiveType,
{
let values = array.values();
let buffer = filter_native(values, predicate);
let mut builder = ArrayDataBuilder::new(array.data_type().clone())
.len(predicate.count)
.add_buffer(buffer);
if let Some((null_count, nulls)) = filter_null_mask(array.nulls(), predicate) {
builder = builder.null_count(null_count).null_bit_buffer(Some(nulls));
}
let data = unsafe { builder.build_unchecked() };
PrimitiveArray::from(data)
}
/// [`FilterBytes`] is created from a source [`GenericByteArray`] and can be
/// used to build a new [`GenericByteArray`] by copying values from the source
///
/// TODO(raphael): Could this be used for the take kernel as well?
struct FilterBytes<'a, OffsetSize> {
src_offsets: &'a [OffsetSize],
src_values: &'a [u8],
dst_offsets: Vec<OffsetSize>,
dst_values: Vec<u8>,
cur_offset: OffsetSize,
}
impl<'a, OffsetSize> FilterBytes<'a, OffsetSize>
where
OffsetSize: OffsetSizeTrait,
{
fn new<T>(capacity: usize, array: &'a GenericByteArray<T>) -> Self
where
T: ByteArrayType<Offset = OffsetSize>,
{
let dst_values = Vec::new();
let mut dst_offsets: Vec<OffsetSize> = Vec::with_capacity(capacity + 1);
let cur_offset = OffsetSize::from_usize(0).unwrap();
dst_offsets.push(cur_offset);
Self {
src_offsets: array.value_offsets(),
src_values: array.value_data(),
dst_offsets,
dst_values,
cur_offset,
}
}
/// Returns the byte offset at `idx`
#[inline]
fn get_value_offset(&self, idx: usize) -> usize {
self.src_offsets[idx].as_usize()
}
/// Returns the start and end of the value at index `idx` along with its length
#[inline]
fn get_value_range(&self, idx: usize) -> (usize, usize, OffsetSize) {
// These can only fail if `array` contains invalid data
let start = self.get_value_offset(idx);
let end = self.get_value_offset(idx + 1);
let len = OffsetSize::from_usize(end - start).expect("illegal offset range");
(start, end, len)
}
/// Extends the in-progress array by the indexes in the provided iterator
fn extend_idx(&mut self, iter: impl Iterator<Item = usize>) {
self.dst_offsets.extend(iter.map(|idx| {
let start = self.src_offsets[idx].as_usize();
let end = self.src_offsets[idx + 1].as_usize();
let len = OffsetSize::from_usize(end - start).expect("illegal offset range");
self.cur_offset += len;
self.dst_values
.extend_from_slice(&self.src_values[start..end]);
self.cur_offset
}));
}
/// Extends the in-progress array by the ranges in the provided iterator
fn extend_slices(&mut self, iter: impl Iterator<Item = (usize, usize)>) {
for (start, end) in iter {
// These can only fail if `array` contains invalid data
for idx in start..end {
let (_, _, len) = self.get_value_range(idx);
self.cur_offset += len;
self.dst_offsets.push(self.cur_offset); // push_unchecked?
}
let value_start = self.get_value_offset(start);
let value_end = self.get_value_offset(end);
self.dst_values
.extend_from_slice(&self.src_values[value_start..value_end]);
}
}
}
/// `filter` implementation for byte arrays
///
/// Note: NULLs with a non-zero slot length in `array` will have the corresponding
/// data copied across. This allows handling the null mask separately from the data
fn filter_bytes<T>(array: &GenericByteArray<T>, predicate: &FilterPredicate) -> GenericByteArray<T>
where
T: ByteArrayType,
{
let mut filter = FilterBytes::new(predicate.count, array);
match &predicate.strategy {
IterationStrategy::SlicesIterator => {
filter.extend_slices(SlicesIterator::new(&predicate.filter))
}
IterationStrategy::Slices(slices) => filter.extend_slices(slices.iter().cloned()),
IterationStrategy::IndexIterator => {
filter.extend_idx(IndexIterator::new(&predicate.filter, predicate.count))
}
IterationStrategy::Indices(indices) => filter.extend_idx(indices.iter().cloned()),
IterationStrategy::All | IterationStrategy::None => unreachable!(),
}
let mut builder = ArrayDataBuilder::new(T::DATA_TYPE)
.len(predicate.count)
.add_buffer(filter.dst_offsets.into())
.add_buffer(filter.dst_values.into());
if let Some((null_count, nulls)) = filter_null_mask(array.nulls(), predicate) {
builder = builder.null_count(null_count).null_bit_buffer(Some(nulls));
}
let data = unsafe { builder.build_unchecked() };
GenericByteArray::from(data)
}
/// `filter` implementation for byte view arrays.
fn filter_byte_view<T: ByteViewType>(
array: &GenericByteViewArray<T>,
predicate: &FilterPredicate,
) -> GenericByteViewArray<T> {
let new_view_buffer = filter_native(array.views(), predicate);
let mut builder = ArrayDataBuilder::new(T::DATA_TYPE)
.len(predicate.count)
.add_buffer(new_view_buffer)
.add_buffers(array.data_buffers().to_vec());
if let Some((null_count, nulls)) = filter_null_mask(array.nulls(), predicate) {
builder = builder.null_count(null_count).null_bit_buffer(Some(nulls));
}
GenericByteViewArray::from(unsafe { builder.build_unchecked() })
}
fn filter_fixed_size_binary(
array: &FixedSizeBinaryArray,
predicate: &FilterPredicate,
) -> FixedSizeBinaryArray {
let values: &[u8] = array.values();
let value_length = array.value_length() as usize;
let calculate_offset_from_index = |index: usize| index * value_length;
let buffer = match &predicate.strategy {
IterationStrategy::SlicesIterator => {
let mut buffer = MutableBuffer::with_capacity(predicate.count * value_length);
for (start, end) in SlicesIterator::new(&predicate.filter) {
buffer.extend_from_slice(
&values[calculate_offset_from_index(start)..calculate_offset_from_index(end)],
);
}
buffer
}
IterationStrategy::Slices(slices) => {
let mut buffer = MutableBuffer::with_capacity(predicate.count * value_length);
for (start, end) in slices {
buffer.extend_from_slice(
&values[calculate_offset_from_index(*start)..calculate_offset_from_index(*end)],
);
}
buffer
}
IterationStrategy::IndexIterator => {
let iter = IndexIterator::new(&predicate.filter, predicate.count).map(|x| {
&values[calculate_offset_from_index(x)..calculate_offset_from_index(x + 1)]
});
let mut buffer = MutableBuffer::new(predicate.count * value_length);
iter.for_each(|item| buffer.extend_from_slice(item));
buffer
}
IterationStrategy::Indices(indices) => {
let iter = indices.iter().map(|x| {
&values[calculate_offset_from_index(*x)..calculate_offset_from_index(*x + 1)]
});
let mut buffer = MutableBuffer::new(predicate.count * value_length);
iter.for_each(|item| buffer.extend_from_slice(item));
buffer
}
IterationStrategy::All | IterationStrategy::None => unreachable!(),
};
let mut builder = ArrayDataBuilder::new(array.data_type().clone())
.len(predicate.count)
.add_buffer(buffer.into());
if let Some((null_count, nulls)) = filter_null_mask(array.nulls(), predicate) {
builder = builder.null_count(null_count).null_bit_buffer(Some(nulls));
}
let data = unsafe { builder.build_unchecked() };
FixedSizeBinaryArray::from(data)
}
/// `filter` implementation for dictionaries
fn filter_dict<T>(array: &DictionaryArray<T>, predicate: &FilterPredicate) -> DictionaryArray<T>
where
T: ArrowDictionaryKeyType,
T::Native: num::Num,
{
let builder = filter_primitive::<T>(array.keys(), predicate)
.into_data()
.into_builder()
.data_type(array.data_type().clone())
.child_data(vec![array.values().to_data()]);
// SAFETY:
// Keys were valid before, filtered subset is therefore still valid
DictionaryArray::from(unsafe { builder.build_unchecked() })
}
/// `filter` implementation for structs
fn filter_struct(
array: &StructArray,
predicate: &FilterPredicate,
) -> Result<StructArray, ArrowError> {
let columns = array
.columns()
.iter()
.map(|column| filter_array(column, predicate))
.collect::<Result<_, _>>()?;
let nulls = if let Some((null_count, nulls)) = filter_null_mask(array.nulls(), predicate) {
let buffer = BooleanBuffer::new(nulls, 0, predicate.count);
Some(unsafe { NullBuffer::new_unchecked(buffer, null_count) })
} else {
None
};
Ok(unsafe { StructArray::new_unchecked(array.fields().clone(), columns, nulls) })
}
/// `filter` implementation for sparse unions
fn filter_sparse_union(
array: &UnionArray,
predicate: &FilterPredicate,
) -> Result<UnionArray, ArrowError> {
let DataType::Union(fields, UnionMode::Sparse) = array.data_type() else {
unreachable!()
};
let type_ids = filter_primitive(&Int8Array::new(array.type_ids().clone(), None), predicate);
let children = fields
.iter()
.map(|(child_type_id, _)| filter_array(array.child(child_type_id), predicate))
.collect::<Result<_, _>>()?;
Ok(unsafe {
UnionArray::new_unchecked(fields.clone(), type_ids.into_parts().1, None, children)
})
}
#[cfg(test)]
mod tests {
use arrow_array::builder::*;
use arrow_array::cast::as_run_array;
use arrow_array::types::*;
use rand::distributions::{Alphanumeric, Standard};
use rand::prelude::*;
use super::*;
macro_rules! def_temporal_test {
($test:ident, $array_type: ident, $data: expr) => {
#[test]
fn $test() {
let a = $data;
let b = BooleanArray::from(vec![true, false, true, false]);
let c = filter(&a, &b).unwrap();
let d = c.as_ref().as_any().downcast_ref::<$array_type>().unwrap();
assert_eq!(2, d.len());
assert_eq!(1, d.value(0));
assert_eq!(3, d.value(1));
}
};
}
def_temporal_test!(
test_filter_date32,
Date32Array,
Date32Array::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_date64,
Date64Array,
Date64Array::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_time32_second,
Time32SecondArray,
Time32SecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_time32_millisecond,
Time32MillisecondArray,
Time32MillisecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_time64_microsecond,
Time64MicrosecondArray,
Time64MicrosecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_time64_nanosecond,
Time64NanosecondArray,
Time64NanosecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_duration_second,
DurationSecondArray,
DurationSecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_duration_millisecond,
DurationMillisecondArray,
DurationMillisecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_duration_microsecond,
DurationMicrosecondArray,
DurationMicrosecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_duration_nanosecond,
DurationNanosecondArray,
DurationNanosecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_timestamp_second,
TimestampSecondArray,
TimestampSecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_timestamp_millisecond,
TimestampMillisecondArray,
TimestampMillisecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_timestamp_microsecond,
TimestampMicrosecondArray,
TimestampMicrosecondArray::from(vec![1, 2, 3, 4])
);
def_temporal_test!(
test_filter_timestamp_nanosecond,
TimestampNanosecondArray,
TimestampNanosecondArray::from(vec![1, 2, 3, 4])
);
#[test]
fn test_filter_array_slice() {
let a = Int32Array::from(vec![5, 6, 7, 8, 9]).slice(1, 4);
let b = BooleanArray::from(vec![true, false, false, true]);
// filtering with sliced filter array is not currently supported
// let b_slice = BooleanArray::from(vec![true, false, false, true, false]).slice(1, 4);
// let b = b_slice.as_any().downcast_ref().unwrap();
let c = filter(&a, &b).unwrap();
let d = c.as_ref().as_any().downcast_ref::<Int32Array>().unwrap();
assert_eq!(2, d.len());
assert_eq!(6, d.value(0));
assert_eq!(9, d.value(1));
}
#[test]
fn test_filter_array_low_density() {
// this test exercises the all 0's branch of the filter algorithm
let mut data_values = (1..=65).collect::<Vec<i32>>();
let mut filter_values = (1..=65).map(|i| matches!(i % 65, 0)).collect::<Vec<bool>>();
// set up two more values after the batch
data_values.extend_from_slice(&[66, 67]);
filter_values.extend_from_slice(&[false, true]);
let a = Int32Array::from(data_values);
let b = BooleanArray::from(filter_values);
let c = filter(&a, &b).unwrap();
let d = c.as_ref().as_any().downcast_ref::<Int32Array>().unwrap();
assert_eq!(2, d.len());
assert_eq!(65, d.value(0));
assert_eq!(67, d.value(1));
}
#[test]
fn test_filter_array_high_density() {
// this test exercises the all 1's branch of the filter algorithm
let mut data_values = (1..=65).map(Some).collect::<Vec<_>>();
let mut filter_values = (1..=65)
.map(|i| !matches!(i % 65, 0))
.collect::<Vec<bool>>();