arrow_schema/
fields.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use std::ops::Deref;
19use std::sync::Arc;
20
21use crate::{ArrowError, DataType, Field, FieldRef};
22
23/// A cheaply cloneable, owned slice of [`FieldRef`]
24///
25/// Similar to `Arc<Vec<FieldRef>>` or `Arc<[FieldRef]>`
26///
27/// Can be constructed in a number of ways
28///
29/// ```
30/// # use std::sync::Arc;
31/// # use arrow_schema::{DataType, Field, Fields, SchemaBuilder};
32/// // Can be constructed from Vec<Field>
33/// Fields::from(vec![Field::new("a", DataType::Boolean, false)]);
34/// // Can be constructed from Vec<FieldRef>
35/// Fields::from(vec![Arc::new(Field::new("a", DataType::Boolean, false))]);
36/// // Can be constructed from an iterator of Field
37/// std::iter::once(Field::new("a", DataType::Boolean, false)).collect::<Fields>();
38/// // Can be constructed from an iterator of FieldRef
39/// std::iter::once(Arc::new(Field::new("a", DataType::Boolean, false))).collect::<Fields>();
40/// ```
41///
42/// See [`SchemaBuilder`] for mutating or updating [`Fields`]
43///
44/// ```
45/// # use arrow_schema::{DataType, Field, SchemaBuilder};
46/// let mut builder = SchemaBuilder::new();
47/// builder.push(Field::new("a", DataType::Boolean, false));
48/// builder.push(Field::new("b", DataType::Boolean, false));
49/// let fields = builder.finish().fields;
50///
51/// let mut builder = SchemaBuilder::from(&fields);
52/// builder.remove(0);
53/// let new = builder.finish().fields;
54/// ```
55///
56/// [`SchemaBuilder`]: crate::SchemaBuilder
57#[derive(Clone, Eq, PartialEq, Ord, PartialOrd, Hash)]
58#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
59#[cfg_attr(feature = "serde", serde(transparent))]
60pub struct Fields(Arc<[FieldRef]>);
61
62impl std::fmt::Debug for Fields {
63    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
64        self.0.as_ref().fmt(f)
65    }
66}
67
68impl Fields {
69    /// Returns a new empty [`Fields`]
70    pub fn empty() -> Self {
71        Self(Arc::new([]))
72    }
73
74    /// Return size of this instance in bytes.
75    pub fn size(&self) -> usize {
76        self.iter()
77            .map(|field| field.size() + std::mem::size_of::<FieldRef>())
78            .sum()
79    }
80
81    /// Searches for a field by name, returning it along with its index if found
82    pub fn find(&self, name: &str) -> Option<(usize, &FieldRef)> {
83        self.0.iter().enumerate().find(|(_, b)| b.name() == name)
84    }
85
86    /// Check to see if `self` is a superset of `other`
87    ///
88    /// In particular returns true if both have the same number of fields, and [`Field::contains`]
89    /// for each field across self and other
90    ///
91    /// In other words, any record that conforms to `other` should also conform to `self`
92    pub fn contains(&self, other: &Fields) -> bool {
93        if Arc::ptr_eq(&self.0, &other.0) {
94            return true;
95        }
96        self.len() == other.len()
97            && self
98                .iter()
99                .zip(other.iter())
100                .all(|(a, b)| Arc::ptr_eq(a, b) || a.contains(b))
101    }
102
103    /// Returns a copy of this [`Fields`] containing only those [`FieldRef`] passing a predicate
104    ///
105    /// Performs a depth-first scan of [`Fields`] invoking `filter` for each [`FieldRef`]
106    /// containing no child [`FieldRef`], a leaf field, along with a count of the number
107    /// of such leaves encountered so far. Only [`FieldRef`] for which `filter`
108    /// returned `true` will be included in the result.
109    ///
110    /// This can therefore be used to select a subset of fields from nested types
111    /// such as [`DataType::Struct`] or [`DataType::List`].
112    ///
113    /// ```
114    /// # use arrow_schema::{DataType, Field, Fields};
115    /// let fields = Fields::from(vec![
116    ///     Field::new("a", DataType::Int32, true), // Leaf 0
117    ///     Field::new("b", DataType::Struct(Fields::from(vec![
118    ///         Field::new("c", DataType::Float32, false), // Leaf 1
119    ///         Field::new("d", DataType::Float64, false), // Leaf 2
120    ///         Field::new("e", DataType::Struct(Fields::from(vec![
121    ///             Field::new("f", DataType::Int32, false),   // Leaf 3
122    ///             Field::new("g", DataType::Float16, false), // Leaf 4
123    ///         ])), true),
124    ///     ])), false)
125    /// ]);
126    /// let filtered = fields.filter_leaves(|idx, _| [0, 2, 3, 4].contains(&idx));
127    /// let expected = Fields::from(vec![
128    ///     Field::new("a", DataType::Int32, true),
129    ///     Field::new("b", DataType::Struct(Fields::from(vec![
130    ///         Field::new("d", DataType::Float64, false),
131    ///         Field::new("e", DataType::Struct(Fields::from(vec![
132    ///             Field::new("f", DataType::Int32, false),
133    ///             Field::new("g", DataType::Float16, false),
134    ///         ])), true),
135    ///     ])), false)
136    /// ]);
137    /// assert_eq!(filtered, expected);
138    /// ```
139    pub fn filter_leaves<F: FnMut(usize, &FieldRef) -> bool>(&self, mut filter: F) -> Self {
140        self.try_filter_leaves(|idx, field| Ok(filter(idx, field)))
141            .unwrap()
142    }
143
144    /// Returns a copy of this [`Fields`] containing only those [`FieldRef`] passing a predicate
145    /// or an error if the predicate fails.
146    ///
147    /// See [`Fields::filter_leaves`] for more information.
148    pub fn try_filter_leaves<F: FnMut(usize, &FieldRef) -> Result<bool, ArrowError>>(
149        &self,
150        mut filter: F,
151    ) -> Result<Self, ArrowError> {
152        fn filter_field<F: FnMut(&FieldRef) -> Result<bool, ArrowError>>(
153            f: &FieldRef,
154            filter: &mut F,
155        ) -> Result<Option<FieldRef>, ArrowError> {
156            use DataType::*;
157
158            let v = match f.data_type() {
159                Dictionary(_, v) => v.as_ref(),       // Key must be integer
160                RunEndEncoded(_, v) => v.data_type(), // Run-ends must be integer
161                d => d,
162            };
163            let d = match v {
164                List(child) => {
165                    let fields = filter_field(child, filter)?;
166                    if let Some(fields) = fields {
167                        List(fields)
168                    } else {
169                        return Ok(None);
170                    }
171                }
172                LargeList(child) => {
173                    let fields = filter_field(child, filter)?;
174                    if let Some(fields) = fields {
175                        LargeList(fields)
176                    } else {
177                        return Ok(None);
178                    }
179                }
180                Map(child, ordered) => {
181                    let fields = filter_field(child, filter)?;
182                    if let Some(fields) = fields {
183                        Map(fields, *ordered)
184                    } else {
185                        return Ok(None);
186                    }
187                }
188                FixedSizeList(child, size) => {
189                    let fields = filter_field(child, filter)?;
190                    if let Some(fields) = fields {
191                        FixedSizeList(fields, *size)
192                    } else {
193                        return Ok(None);
194                    }
195                }
196                Struct(fields) => {
197                    let filtered: Result<Vec<_>, _> =
198                        fields.iter().map(|f| filter_field(f, filter)).collect();
199                    let filtered: Fields = filtered?
200                        .iter()
201                        .filter_map(|f| f.as_ref().cloned())
202                        .collect();
203
204                    if filtered.is_empty() {
205                        return Ok(None);
206                    }
207
208                    Struct(filtered)
209                }
210                Union(fields, mode) => {
211                    let filtered: Result<Vec<_>, _> = fields
212                        .iter()
213                        .map(|(id, f)| filter_field(f, filter).map(|f| f.map(|f| (id, f))))
214                        .collect();
215                    let filtered: UnionFields = filtered?
216                        .iter()
217                        .filter_map(|f| f.as_ref().cloned())
218                        .collect();
219
220                    if filtered.is_empty() {
221                        return Ok(None);
222                    }
223
224                    Union(filtered, *mode)
225                }
226                _ => {
227                    let filtered = filter(f)?;
228                    return Ok(filtered.then(|| f.clone()));
229                }
230            };
231            let d = match f.data_type() {
232                Dictionary(k, _) => Dictionary(k.clone(), Box::new(d)),
233                RunEndEncoded(v, f) => {
234                    RunEndEncoded(v.clone(), Arc::new(f.as_ref().clone().with_data_type(d)))
235                }
236                _ => d,
237            };
238            Ok(Some(Arc::new(f.as_ref().clone().with_data_type(d))))
239        }
240
241        let mut leaf_idx = 0;
242        let mut filter = |f: &FieldRef| {
243            let t = filter(leaf_idx, f)?;
244            leaf_idx += 1;
245            Ok(t)
246        };
247
248        let filtered: Result<Vec<_>, _> = self
249            .0
250            .iter()
251            .map(|f| filter_field(f, &mut filter))
252            .collect();
253        let filtered = filtered?
254            .iter()
255            .filter_map(|f| f.as_ref().cloned())
256            .collect();
257        Ok(filtered)
258    }
259}
260
261impl Default for Fields {
262    fn default() -> Self {
263        Self::empty()
264    }
265}
266
267impl FromIterator<Field> for Fields {
268    fn from_iter<T: IntoIterator<Item = Field>>(iter: T) -> Self {
269        iter.into_iter().map(Arc::new).collect()
270    }
271}
272
273impl FromIterator<FieldRef> for Fields {
274    fn from_iter<T: IntoIterator<Item = FieldRef>>(iter: T) -> Self {
275        Self(iter.into_iter().collect())
276    }
277}
278
279impl From<Vec<Field>> for Fields {
280    fn from(value: Vec<Field>) -> Self {
281        value.into_iter().collect()
282    }
283}
284
285impl From<Vec<FieldRef>> for Fields {
286    fn from(value: Vec<FieldRef>) -> Self {
287        Self(value.into())
288    }
289}
290
291impl From<&[FieldRef]> for Fields {
292    fn from(value: &[FieldRef]) -> Self {
293        Self(value.into())
294    }
295}
296
297impl<const N: usize> From<[FieldRef; N]> for Fields {
298    fn from(value: [FieldRef; N]) -> Self {
299        Self(Arc::new(value))
300    }
301}
302
303impl Deref for Fields {
304    type Target = [FieldRef];
305
306    fn deref(&self) -> &Self::Target {
307        self.0.as_ref()
308    }
309}
310
311impl<'a> IntoIterator for &'a Fields {
312    type Item = &'a FieldRef;
313    type IntoIter = std::slice::Iter<'a, FieldRef>;
314
315    fn into_iter(self) -> Self::IntoIter {
316        self.0.iter()
317    }
318}
319
320/// A cheaply cloneable, owned collection of [`FieldRef`] and their corresponding type ids
321#[derive(Clone, Eq, PartialEq, Ord, PartialOrd, Hash)]
322#[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))]
323#[cfg_attr(feature = "serde", serde(transparent))]
324pub struct UnionFields(Arc<[(i8, FieldRef)]>);
325
326impl std::fmt::Debug for UnionFields {
327    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
328        self.0.as_ref().fmt(f)
329    }
330}
331
332impl UnionFields {
333    /// Create a new [`UnionFields`] with no fields
334    pub fn empty() -> Self {
335        Self(Arc::from([]))
336    }
337
338    /// Create a new [`UnionFields`] from a [`Fields`] and array of type_ids
339    ///
340    /// See <https://arrow.apache.org/docs/format/Columnar.html#union-layout>
341    ///
342    /// ```
343    /// use arrow_schema::{DataType, Field, UnionFields};
344    /// // Create a new UnionFields with type id mapping
345    /// // 1 -> DataType::UInt8
346    /// // 3 -> DataType::Utf8
347    /// UnionFields::new(
348    ///     vec![1, 3],
349    ///     vec![
350    ///         Field::new("field1", DataType::UInt8, false),
351    ///         Field::new("field3", DataType::Utf8, false),
352    ///     ],
353    /// );
354    /// ```
355    pub fn new<F, T>(type_ids: T, fields: F) -> Self
356    where
357        F: IntoIterator,
358        F::Item: Into<FieldRef>,
359        T: IntoIterator<Item = i8>,
360    {
361        let fields = fields.into_iter().map(Into::into);
362        let mut set = 0_u128;
363        type_ids
364            .into_iter()
365            .inspect(|&idx| {
366                let mask = 1_u128 << idx;
367                if (set & mask) != 0 {
368                    panic!("duplicate type id: {}", idx);
369                } else {
370                    set |= mask;
371                }
372            })
373            .zip(fields)
374            .collect()
375    }
376
377    /// Return size of this instance in bytes.
378    pub fn size(&self) -> usize {
379        self.iter()
380            .map(|(_, field)| field.size() + std::mem::size_of::<(i8, FieldRef)>())
381            .sum()
382    }
383
384    /// Returns the number of fields in this [`UnionFields`]
385    pub fn len(&self) -> usize {
386        self.0.len()
387    }
388
389    /// Returns `true` if this is empty
390    pub fn is_empty(&self) -> bool {
391        self.0.is_empty()
392    }
393
394    /// Returns an iterator over the fields and type ids in this [`UnionFields`]
395    pub fn iter(&self) -> impl Iterator<Item = (i8, &FieldRef)> + '_ {
396        self.0.iter().map(|(id, f)| (*id, f))
397    }
398
399    /// Merge this field into self if it is compatible.
400    ///
401    /// See [`Field::try_merge`]
402    pub(crate) fn try_merge(&mut self, other: &Self) -> Result<(), ArrowError> {
403        // TODO: This currently may produce duplicate type IDs (#3982)
404        let mut output: Vec<_> = self.iter().map(|(id, f)| (id, f.clone())).collect();
405        for (field_type_id, from_field) in other.iter() {
406            let mut is_new_field = true;
407            for (self_type_id, self_field) in output.iter_mut() {
408                if from_field == self_field {
409                    // If the nested fields in two unions are the same, they must have same
410                    // type id.
411                    if *self_type_id != field_type_id {
412                        return Err(ArrowError::SchemaError(
413                            format!("Fail to merge schema field '{}' because the self_type_id = {} does not equal field_type_id = {}",
414                                    self_field.name(), self_type_id, field_type_id)
415                        ));
416                    }
417
418                    is_new_field = false;
419                    break;
420                }
421            }
422
423            if is_new_field {
424                output.push((field_type_id, from_field.clone()))
425            }
426        }
427        *self = output.into_iter().collect();
428        Ok(())
429    }
430}
431
432impl FromIterator<(i8, FieldRef)> for UnionFields {
433    fn from_iter<T: IntoIterator<Item = (i8, FieldRef)>>(iter: T) -> Self {
434        // TODO: Should this validate type IDs are unique (#3982)
435        Self(iter.into_iter().collect())
436    }
437}
438
439#[cfg(test)]
440mod tests {
441    use super::*;
442    use crate::UnionMode;
443
444    #[test]
445    fn test_filter() {
446        let floats = Fields::from(vec![
447            Field::new("a", DataType::Float32, false),
448            Field::new("b", DataType::Float32, false),
449        ]);
450        let fields = Fields::from(vec![
451            Field::new("a", DataType::Int32, true),
452            Field::new("floats", DataType::Struct(floats.clone()), true),
453            Field::new("b", DataType::Int16, true),
454            Field::new(
455                "c",
456                DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
457                false,
458            ),
459            Field::new(
460                "d",
461                DataType::Dictionary(
462                    Box::new(DataType::Int32),
463                    Box::new(DataType::Struct(floats.clone())),
464                ),
465                false,
466            ),
467            Field::new_list(
468                "e",
469                Field::new("floats", DataType::Struct(floats.clone()), true),
470                true,
471            ),
472            Field::new_fixed_size_list(
473                "f",
474                Field::new_list_field(DataType::Int32, false),
475                3,
476                false,
477            ),
478            Field::new_map(
479                "g",
480                "entries",
481                Field::new("keys", DataType::LargeUtf8, false),
482                Field::new("values", DataType::Int32, true),
483                false,
484                false,
485            ),
486            Field::new(
487                "h",
488                DataType::Union(
489                    UnionFields::new(
490                        vec![1, 3],
491                        vec![
492                            Field::new("field1", DataType::UInt8, false),
493                            Field::new("field3", DataType::Utf8, false),
494                        ],
495                    ),
496                    UnionMode::Dense,
497                ),
498                true,
499            ),
500            Field::new(
501                "i",
502                DataType::RunEndEncoded(
503                    Arc::new(Field::new("run_ends", DataType::Int32, false)),
504                    Arc::new(Field::new("values", DataType::Struct(floats.clone()), true)),
505                ),
506                false,
507            ),
508        ]);
509
510        let floats_a = DataType::Struct(vec![floats[0].clone()].into());
511
512        let r = fields.filter_leaves(|idx, _| idx == 0 || idx == 1);
513        assert_eq!(r.len(), 2);
514        assert_eq!(r[0], fields[0]);
515        assert_eq!(r[1].data_type(), &floats_a);
516
517        let r = fields.filter_leaves(|_, f| f.name() == "a");
518        assert_eq!(r.len(), 5);
519        assert_eq!(r[0], fields[0]);
520        assert_eq!(r[1].data_type(), &floats_a);
521        assert_eq!(
522            r[2].data_type(),
523            &DataType::Dictionary(Box::new(DataType::Int32), Box::new(floats_a.clone()))
524        );
525        assert_eq!(
526            r[3].as_ref(),
527            &Field::new_list("e", Field::new("floats", floats_a.clone(), true), true)
528        );
529        assert_eq!(
530            r[4].as_ref(),
531            &Field::new(
532                "i",
533                DataType::RunEndEncoded(
534                    Arc::new(Field::new("run_ends", DataType::Int32, false)),
535                    Arc::new(Field::new("values", floats_a.clone(), true)),
536                ),
537                false,
538            )
539        );
540
541        let r = fields.filter_leaves(|_, f| f.name() == "floats");
542        assert_eq!(r.len(), 0);
543
544        let r = fields.filter_leaves(|idx, _| idx == 9);
545        assert_eq!(r.len(), 1);
546        assert_eq!(r[0], fields[6]);
547
548        let r = fields.filter_leaves(|idx, _| idx == 10 || idx == 11);
549        assert_eq!(r.len(), 1);
550        assert_eq!(r[0], fields[7]);
551
552        let union = DataType::Union(
553            UnionFields::new(vec![1], vec![Field::new("field1", DataType::UInt8, false)]),
554            UnionMode::Dense,
555        );
556
557        let r = fields.filter_leaves(|idx, _| idx == 12);
558        assert_eq!(r.len(), 1);
559        assert_eq!(r[0].data_type(), &union);
560
561        let r = fields.filter_leaves(|idx, _| idx == 14 || idx == 15);
562        assert_eq!(r.len(), 1);
563        assert_eq!(r[0], fields[9]);
564
565        // Propagate error
566        let r = fields.try_filter_leaves(|_, _| Err(ArrowError::SchemaError("error".to_string())));
567        assert!(r.is_err());
568    }
569}