polars_plan/dsl/function_expr/
boolean.rs

1use std::ops::{BitAnd, BitOr};
2
3use polars_core::POOL;
4use rayon::iter::{IntoParallelRefIterator, ParallelIterator};
5
6use super::*;
7#[cfg(feature = "is_in")]
8use crate::wrap;
9use crate::{map, map_as_slice};
10
11#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
12#[derive(Clone, PartialEq, Debug, Eq, Hash)]
13pub enum BooleanFunction {
14    Any {
15        ignore_nulls: bool,
16    },
17    All {
18        ignore_nulls: bool,
19    },
20    IsNull,
21    IsNotNull,
22    IsFinite,
23    IsInfinite,
24    IsNan,
25    IsNotNan,
26    #[cfg(feature = "is_first_distinct")]
27    IsFirstDistinct,
28    #[cfg(feature = "is_last_distinct")]
29    IsLastDistinct,
30    #[cfg(feature = "is_unique")]
31    IsUnique,
32    #[cfg(feature = "is_unique")]
33    IsDuplicated,
34    #[cfg(feature = "is_between")]
35    IsBetween {
36        closed: ClosedInterval,
37    },
38    #[cfg(feature = "is_in")]
39    IsIn,
40    AllHorizontal,
41    AnyHorizontal,
42    // Also bitwise negate
43    Not,
44}
45
46impl BooleanFunction {
47    pub(super) fn get_field(&self, mapper: FieldsMapper) -> PolarsResult<Field> {
48        match self {
49            BooleanFunction::Not => {
50                mapper.try_map_dtype(|dtype| {
51                    match dtype {
52                        DataType::Boolean => Ok(DataType::Boolean),
53                        dt if dt.is_integer() => Ok(dt.clone()),
54                        dt => polars_bail!(InvalidOperation: "dtype {:?} not supported in 'not' operation", dt) 
55                    }
56                })
57
58            },
59            _ => mapper.with_dtype(DataType::Boolean),
60        }
61    }
62}
63
64impl Display for BooleanFunction {
65    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
66        use BooleanFunction::*;
67        let s = match self {
68            All { .. } => "all",
69            Any { .. } => "any",
70            IsNull => "is_null",
71            IsNotNull => "is_not_null",
72            IsFinite => "is_finite",
73            IsInfinite => "is_infinite",
74            IsNan => "is_nan",
75            IsNotNan => "is_not_nan",
76            #[cfg(feature = "is_first_distinct")]
77            IsFirstDistinct => "is_first_distinct",
78            #[cfg(feature = "is_last_distinct")]
79            IsLastDistinct => "is_last_distinct",
80            #[cfg(feature = "is_unique")]
81            IsUnique => "is_unique",
82            #[cfg(feature = "is_unique")]
83            IsDuplicated => "is_duplicated",
84            #[cfg(feature = "is_between")]
85            IsBetween { .. } => "is_between",
86            #[cfg(feature = "is_in")]
87            IsIn => "is_in",
88            AnyHorizontal => "any_horizontal",
89            AllHorizontal => "all_horizontal",
90            Not => "not",
91        };
92        write!(f, "{s}")
93    }
94}
95
96impl From<BooleanFunction> for SpecialEq<Arc<dyn ColumnsUdf>> {
97    fn from(func: BooleanFunction) -> Self {
98        use BooleanFunction::*;
99        match func {
100            Any { ignore_nulls } => map!(any, ignore_nulls),
101            All { ignore_nulls } => map!(all, ignore_nulls),
102            IsNull => map!(is_null),
103            IsNotNull => map!(is_not_null),
104            IsFinite => map!(is_finite),
105            IsInfinite => map!(is_infinite),
106            IsNan => map!(is_nan),
107            IsNotNan => map!(is_not_nan),
108            #[cfg(feature = "is_first_distinct")]
109            IsFirstDistinct => map!(is_first_distinct),
110            #[cfg(feature = "is_last_distinct")]
111            IsLastDistinct => map!(is_last_distinct),
112            #[cfg(feature = "is_unique")]
113            IsUnique => map!(is_unique),
114            #[cfg(feature = "is_unique")]
115            IsDuplicated => map!(is_duplicated),
116            #[cfg(feature = "is_between")]
117            IsBetween { closed } => map_as_slice!(is_between, closed),
118            #[cfg(feature = "is_in")]
119            IsIn => wrap!(is_in),
120            Not => map!(not),
121            AllHorizontal => map_as_slice!(all_horizontal),
122            AnyHorizontal => map_as_slice!(any_horizontal),
123        }
124    }
125}
126
127impl From<BooleanFunction> for FunctionExpr {
128    fn from(func: BooleanFunction) -> Self {
129        FunctionExpr::Boolean(func)
130    }
131}
132
133fn any(s: &Column, ignore_nulls: bool) -> PolarsResult<Column> {
134    let ca = s.bool()?;
135    if ignore_nulls {
136        Ok(Column::new(s.name().clone(), [ca.any()]))
137    } else {
138        Ok(Column::new(s.name().clone(), [ca.any_kleene()]))
139    }
140}
141
142fn all(s: &Column, ignore_nulls: bool) -> PolarsResult<Column> {
143    let ca = s.bool()?;
144    if ignore_nulls {
145        Ok(Column::new(s.name().clone(), [ca.all()]))
146    } else {
147        Ok(Column::new(s.name().clone(), [ca.all_kleene()]))
148    }
149}
150
151fn is_null(s: &Column) -> PolarsResult<Column> {
152    Ok(s.is_null().into_column())
153}
154
155fn is_not_null(s: &Column) -> PolarsResult<Column> {
156    Ok(s.is_not_null().into_column())
157}
158
159fn is_finite(s: &Column) -> PolarsResult<Column> {
160    s.is_finite().map(|ca| ca.into_column())
161}
162
163fn is_infinite(s: &Column) -> PolarsResult<Column> {
164    s.is_infinite().map(|ca| ca.into_column())
165}
166
167pub(super) fn is_nan(s: &Column) -> PolarsResult<Column> {
168    s.is_nan().map(|ca| ca.into_column())
169}
170
171pub(super) fn is_not_nan(s: &Column) -> PolarsResult<Column> {
172    s.is_not_nan().map(|ca| ca.into_column())
173}
174
175#[cfg(feature = "is_first_distinct")]
176fn is_first_distinct(s: &Column) -> PolarsResult<Column> {
177    polars_ops::prelude::is_first_distinct(s.as_materialized_series()).map(|ca| ca.into_column())
178}
179
180#[cfg(feature = "is_last_distinct")]
181fn is_last_distinct(s: &Column) -> PolarsResult<Column> {
182    polars_ops::prelude::is_last_distinct(s.as_materialized_series()).map(|ca| ca.into_column())
183}
184
185#[cfg(feature = "is_unique")]
186fn is_unique(s: &Column) -> PolarsResult<Column> {
187    polars_ops::prelude::is_unique(s.as_materialized_series()).map(|ca| ca.into_column())
188}
189
190#[cfg(feature = "is_unique")]
191fn is_duplicated(s: &Column) -> PolarsResult<Column> {
192    polars_ops::prelude::is_duplicated(s.as_materialized_series()).map(|ca| ca.into_column())
193}
194
195#[cfg(feature = "is_between")]
196fn is_between(s: &[Column], closed: ClosedInterval) -> PolarsResult<Column> {
197    let ser = &s[0];
198    let lower = &s[1];
199    let upper = &s[2];
200    polars_ops::prelude::is_between(
201        ser.as_materialized_series(),
202        lower.as_materialized_series(),
203        upper.as_materialized_series(),
204        closed,
205    )
206    .map(|ca| ca.into_column())
207}
208
209#[cfg(feature = "is_in")]
210fn is_in(s: &mut [Column]) -> PolarsResult<Option<Column>> {
211    let left = &s[0];
212    let other = &s[1];
213    polars_ops::prelude::is_in(
214        left.as_materialized_series(),
215        other.as_materialized_series(),
216    )
217    .map(|ca| Some(ca.into_column()))
218}
219
220fn not(s: &Column) -> PolarsResult<Column> {
221    polars_ops::series::negate_bitwise(s.as_materialized_series()).map(Column::from)
222}
223
224// We shouldn't hit these often only on very wide dataframes where we don't reduce to & expressions.
225fn any_horizontal(s: &[Column]) -> PolarsResult<Column> {
226    let out = POOL
227        .install(|| {
228            s.par_iter()
229                .try_fold(
230                    || BooleanChunked::new(PlSmallStr::EMPTY, &[false]),
231                    |acc, b| {
232                        let b = b.cast(&DataType::Boolean)?;
233                        let b = b.bool()?;
234                        PolarsResult::Ok((&acc).bitor(b))
235                    },
236                )
237                .try_reduce(
238                    || BooleanChunked::new(PlSmallStr::EMPTY, [false]),
239                    |a, b| Ok(a.bitor(b)),
240                )
241        })?
242        .with_name(s[0].name().clone());
243    Ok(out.into_column())
244}
245
246// We shouldn't hit these often only on very wide dataframes where we don't reduce to & expressions.
247fn all_horizontal(s: &[Column]) -> PolarsResult<Column> {
248    let out = POOL
249        .install(|| {
250            s.par_iter()
251                .try_fold(
252                    || BooleanChunked::new(PlSmallStr::EMPTY, &[true]),
253                    |acc, b| {
254                        let b = b.cast(&DataType::Boolean)?;
255                        let b = b.bool()?;
256                        PolarsResult::Ok((&acc).bitand(b))
257                    },
258                )
259                .try_reduce(
260                    || BooleanChunked::new(PlSmallStr::EMPTY, [true]),
261                    |a, b| Ok(a.bitand(b)),
262                )
263        })?
264        .with_name(s[0].name().clone());
265    Ok(out.into_column())
266}