orc-rust 0.8.0

Implementation of Apache ORC file format using Apache Arrow in-memory format
Documentation
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
// 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.

use std::sync::Arc;

use arrow::array::{ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray};
use arrow::buffer::NullBuffer;
use arrow::datatypes::ArrowNativeTypeOp;
use arrow::datatypes::ArrowPrimitiveType;
use arrow::datatypes::DataType as ArrowDataType;
use arrow::datatypes::{
    Date32Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, SchemaRef,
};
use arrow::record_batch::{RecordBatch, RecordBatchOptions};
use snafu::{ensure, ResultExt};

use crate::column::Column;
use crate::encoding::boolean::BooleanDecoder;
use crate::encoding::byte::ByteRleDecoder;
use crate::encoding::float::FloatDecoder;
use crate::encoding::integer::get_signed_int_decoder;
use crate::encoding::PrimitiveValueDecoder;
use crate::error::{
    self, MismatchedSchemaSnafu, Result, UnexpectedSnafu, UnsupportedTypeVariantSnafu,
};
use crate::proto::stream::Kind;
use crate::schema::DataType;
use crate::stripe::Stripe;
use crate::RowSelection;

use self::decimal::new_decimal_decoder;
use self::list::ListArrayDecoder;
use self::map::MapArrayDecoder;
use self::string::{new_binary_decoder, new_string_decoder};
use self::struct_decoder::StructArrayDecoder;
use self::timestamp::{new_timestamp_decoder, new_timestamp_instant_decoder};
use self::union::UnionArrayDecoder;

mod decimal;
mod list;
mod map;
mod string;
mod struct_decoder;
mod timestamp;
mod union;

pub trait ArrayBatchDecoder: Send {
    /// Used as base for decoding ORC columns into Arrow arrays. Provide an input `batch_size`
    /// which specifies the upper limit of the number of values returned in the output array.
    ///
    /// If parent nested type (e.g. Struct) indicates a null in it's PRESENT stream,
    /// then the child doesn't have a value (similar to other nullability). So we need
    /// to take care to insert these null values as Arrow requires the child to hold
    /// data in the null slot of the child.
    // TODO: encode nullability in generic -> for a given column in a stripe, we will always know
    //       upfront if we need to bother with nulls or not, so we don't need to keep checking this
    //       for every invocation of next_batch
    // NOTE: null parent may have non-null child, so would still have to account for this
    fn next_batch(
        &mut self,
        batch_size: usize,
        parent_present: Option<&NullBuffer>,
    ) -> Result<ArrayRef>;

    /// Skip the next `n` values without decoding them, failing if it cannot skip the enough values.
    /// If parent nested type (e.g. Struct) indicates a null in it's PRESENT stream,
    /// then the child doesn't have a value (similar to other nullability). So we need
    /// to take care to insert these null values as Arrow requires the child to hold
    /// data in the null slot of the child.
    fn skip_values(&mut self, n: usize, parent_present: Option<&NullBuffer>) -> Result<()>;
}

struct PrimitiveArrayDecoder<T: ArrowPrimitiveType> {
    iter: Box<dyn PrimitiveValueDecoder<T::Native> + Send>,
    present: Option<PresentDecoder>,
}

impl<T: ArrowPrimitiveType> PrimitiveArrayDecoder<T> {
    pub fn new(
        iter: Box<dyn PrimitiveValueDecoder<T::Native> + Send>,
        present: Option<PresentDecoder>,
    ) -> Self {
        Self { iter, present }
    }

    fn next_primitive_batch(
        &mut self,
        batch_size: usize,
        parent_present: Option<&NullBuffer>,
    ) -> Result<PrimitiveArray<T>> {
        let present =
            derive_present_vec(&mut self.present, parent_present, batch_size).transpose()?;
        let mut data = vec![T::Native::ZERO; batch_size];
        match present {
            Some(present) => {
                self.iter.decode_spaced(data.as_mut_slice(), &present)?;
                let array = PrimitiveArray::<T>::new(data.into(), Some(present));
                Ok(array)
            }
            None => {
                self.iter.decode(data.as_mut_slice())?;
                let array = PrimitiveArray::<T>::from_iter_values(data);
                Ok(array)
            }
        }
    }
}

impl<T: ArrowPrimitiveType> ArrayBatchDecoder for PrimitiveArrayDecoder<T> {
    fn next_batch(
        &mut self,
        batch_size: usize,
        parent_present: Option<&NullBuffer>,
    ) -> Result<ArrayRef> {
        let array = self.next_primitive_batch(batch_size, parent_present)?;
        let array = Arc::new(array) as ArrayRef;
        Ok(array)
    }

    fn skip_values(&mut self, n: usize, parent_present: Option<&NullBuffer>) -> Result<()> {
        let non_null_count =
            skip_present_and_get_non_null_count(&mut self.present, parent_present, n)?;
        self.iter.skip(non_null_count)
    }
}

type Int64ArrayDecoder = PrimitiveArrayDecoder<Int64Type>;
type Int32ArrayDecoder = PrimitiveArrayDecoder<Int32Type>;
type Int16ArrayDecoder = PrimitiveArrayDecoder<Int16Type>;
type Int8ArrayDecoder = PrimitiveArrayDecoder<Int8Type>;
type Float32ArrayDecoder = PrimitiveArrayDecoder<Float32Type>;
type Float64ArrayDecoder = PrimitiveArrayDecoder<Float64Type>;
type DateArrayDecoder = PrimitiveArrayDecoder<Date32Type>; // TODO: does ORC encode as i64 or i32?

struct BooleanArrayDecoder {
    iter: Box<dyn PrimitiveValueDecoder<bool> + Send>,
    present: Option<PresentDecoder>,
}

impl BooleanArrayDecoder {
    pub fn new(
        iter: Box<dyn PrimitiveValueDecoder<bool> + Send>,
        present: Option<PresentDecoder>,
    ) -> Self {
        Self { iter, present }
    }
}

impl ArrayBatchDecoder for BooleanArrayDecoder {
    fn next_batch(
        &mut self,
        batch_size: usize,
        parent_present: Option<&NullBuffer>,
    ) -> Result<ArrayRef> {
        let present =
            derive_present_vec(&mut self.present, parent_present, batch_size).transpose()?;
        let mut data = vec![false; batch_size];
        let array = match present {
            Some(present) => {
                self.iter.decode_spaced(data.as_mut_slice(), &present)?;
                BooleanArray::new(data.into(), Some(present))
            }
            None => {
                self.iter.decode(data.as_mut_slice())?;
                BooleanArray::from(data)
            }
        };
        Ok(Arc::new(array))
    }

    fn skip_values(&mut self, n: usize, parent_present: Option<&NullBuffer>) -> Result<()> {
        let non_null_count =
            skip_present_and_get_non_null_count(&mut self.present, parent_present, n)?;
        self.iter.skip(non_null_count)
    }
}

struct PresentDecoder {
    // TODO: ideally directly reference BooleanDecoder, doing this way to avoid
    //       the generic propagation that would be required (BooleanDecoder<R: Read>)
    inner: Box<dyn PrimitiveValueDecoder<bool> + Send>,
}

impl PresentDecoder {
    fn from_stripe(stripe: &Stripe, column: &Column) -> Option<Self> {
        stripe
            .stream_map()
            .get_opt(column, Kind::Present)
            .map(|stream| {
                let inner = Box::new(BooleanDecoder::new(stream));
                PresentDecoder { inner }
            })
    }

    fn next_buffer(&mut self, size: usize) -> Result<NullBuffer> {
        let mut data = vec![false; size];
        self.inner.decode(&mut data)?;
        Ok(NullBuffer::from(data))
    }
}

fn merge_parent_present(
    parent_present: &NullBuffer,
    present: Result<NullBuffer>,
) -> Result<NullBuffer> {
    let present = present?;
    let non_null_count = parent_present.len() - parent_present.null_count();
    debug_assert!(present.len() == non_null_count);
    let mut builder = BooleanBufferBuilder::new(parent_present.len());
    builder.append_n(parent_present.len(), false);
    for (idx, p) in parent_present.valid_indices().zip(present.iter()) {
        builder.set_bit(idx, p);
    }
    Ok(builder.finish().into())
}

fn derive_present_vec(
    present: &mut Option<PresentDecoder>,
    parent_present: Option<&NullBuffer>,
    batch_size: usize,
) -> Option<Result<NullBuffer>> {
    let present = match (present, parent_present) {
        (Some(present), Some(parent_present)) => {
            let element_count = parent_present.len() - parent_present.null_count();
            let present = present.next_buffer(element_count);
            Some(merge_parent_present(parent_present, present))
        }
        (Some(present), None) => Some(present.next_buffer(batch_size)),
        (None, Some(parent_present)) => Some(Ok(parent_present.clone())),
        (None, None) => None,
    };

    // omit the null buffer if there are no nulls
    match present {
        Some(Ok(present)) if present.null_count() > 0 => Some(Ok(present)),
        _ => None,
    }
}

/// Skip n values and return the non-null count for the data stream
fn skip_present_and_get_non_null_count(
    present: &mut Option<PresentDecoder>,
    parent_present: Option<&NullBuffer>,
    n: usize,
) -> Result<usize> {
    match (present, parent_present) {
        (Some(present), Some(parent_present)) => {
            // Parent has nulls, so we need to decode parent present to know how many
            // of our present values to skip
            let non_null_in_parent = parent_present.len() - parent_present.null_count();

            // Skip our present values for non-null parents and count non-nulls
            let mut our_present = vec![false; non_null_in_parent];
            present.inner.decode(&mut our_present)?;
            let our_non_null_count = our_present.iter().filter(|&&v| v).count();

            Ok(our_non_null_count)
        }
        (Some(present), None) => {
            // No parent present, skip n values and count non-nulls
            let mut present_values = vec![false; n];
            present.inner.decode(&mut present_values)?;
            Ok(present_values.iter().filter(|&&v| v).count())
        }
        (None, Some(parent_present)) => {
            // No our present stream, all non-null parents have data
            Ok(parent_present.len() - parent_present.null_count())
        }
        (None, None) => {
            // No nulls at all, all n values have data
            Ok(n)
        }
    }
}

pub struct NaiveStripeDecoder {
    stripe: Stripe,
    schema_ref: SchemaRef,
    decoders: Vec<Box<dyn ArrayBatchDecoder>>,
    index: usize,
    batch_size: usize,
    number_of_rows: usize,
    row_selection: Option<RowSelection>,
    selection_index: usize,
}

impl NaiveStripeDecoder {
    /// Advance according to the configured row selection and return the next batch, if any.
    ///
    /// Behavior:
    /// - Iterates `RowSelection` segments (skip/select) starting at `selection_index`.
    /// - For skip segments: clamp to remaining rows in this stripe, advance decoders via
    ///   `skip_rows(actual_skip)`, and advance `index`. If the segment is fully consumed,
    ///   increment `selection_index`.
    /// - For select segments: decode up to `min(row_count, batch_size, remaining_in_stripe)`,
    ///   advance `index`, update `selection_index` if fully consumed, and return the batch.
    /// - If a segment requests rows beyond the end of the stripe, it is skipped (advancing
    ///   `selection_index`) without touching decoders.
    fn next_with_row_selection(&mut self) -> Option<Result<RecordBatch>> {
        // Process selectors until we produce a batch or exhaust selection
        loop {
            let (is_skip, row_count) = {
                let selectors = self.row_selection.as_ref().unwrap().selectors();
                if self.selection_index >= selectors.len() {
                    return None;
                }
                let selector = selectors[self.selection_index];
                (selector.skip, selector.row_count)
            };

            if is_skip {
                let remaining = self.number_of_rows - self.index;
                let actual_skip = row_count.min(remaining);

                if actual_skip == 0 {
                    // Nothing to skip in this stripe; try next selector
                    self.selection_index += 1;
                    continue;
                }

                // Keep decoders in sync by skipping values per column
                if let Err(e) = self.skip_rows(actual_skip) {
                    return Some(Err(e));
                }
                self.index += actual_skip;

                if actual_skip >= row_count {
                    self.selection_index += 1;
                }
            } else {
                let rows_to_read = row_count.min(self.batch_size);
                let remaining = self.number_of_rows - self.index;
                let actual_rows = rows_to_read.min(remaining);

                if actual_rows == 0 {
                    // Nothing to read from this selector in this stripe; advance selector
                    self.selection_index += 1;
                    continue;
                }

                let record = self.decode_next_batch(actual_rows).transpose()?;
                self.index += actual_rows;

                if actual_rows >= row_count {
                    self.selection_index += 1;
                }
                return Some(record);
            }
        }
    }
}

impl Iterator for NaiveStripeDecoder {
    type Item = Result<RecordBatch>;

    // TODO: check if we can make this more efficient
    fn next(&mut self) -> Option<Self::Item> {
        if self.index < self.number_of_rows {
            // Handle row selection if present
            if self.row_selection.is_some() {
                self.next_with_row_selection()
            } else {
                // No row selection - decode normally
                let record = self
                    .decode_next_batch(self.number_of_rows - self.index)
                    .transpose()?;
                self.index += self.batch_size;
                Some(record)
            }
        } else {
            None
        }
    }
}

pub fn array_decoder_factory(
    column: &Column,
    hinted_arrow_type: &ArrowDataType,
    stripe: &Stripe,
) -> Result<Box<dyn ArrayBatchDecoder>> {
    let decoder: Box<dyn ArrayBatchDecoder> = match (column.data_type(), hinted_arrow_type) {
        // TODO: try make branches more generic, reduce duplication
        (DataType::Boolean { .. }, ArrowDataType::Boolean) => {
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = Box::new(BooleanDecoder::new(iter));
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(BooleanArrayDecoder::new(iter, present))
        }
        (DataType::Byte { .. }, ArrowDataType::Int8) => {
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = Box::new(ByteRleDecoder::new(iter));
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(Int8ArrayDecoder::new(iter, present))
        }
        (DataType::Short { .. }, ArrowDataType::Int16) => {
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = get_signed_int_decoder(iter, column.rle_version());
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(Int16ArrayDecoder::new(iter, present))
        }
        (DataType::Int { .. }, ArrowDataType::Int32) => {
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = get_signed_int_decoder(iter, column.rle_version());
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(Int32ArrayDecoder::new(iter, present))
        }
        (DataType::Long { .. }, ArrowDataType::Int64) => {
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = get_signed_int_decoder(iter, column.rle_version());
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(Int64ArrayDecoder::new(iter, present))
        }
        (DataType::Float { .. }, ArrowDataType::Float32) => {
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = Box::new(FloatDecoder::new(iter));
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(Float32ArrayDecoder::new(iter, present))
        }
        (DataType::Double { .. }, ArrowDataType::Float64) => {
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = Box::new(FloatDecoder::new(iter));
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(Float64ArrayDecoder::new(iter, present))
        }
        (DataType::String { .. }, ArrowDataType::Utf8)
        | (DataType::Varchar { .. }, ArrowDataType::Utf8)
        | (DataType::Char { .. }, ArrowDataType::Utf8) => new_string_decoder(column, stripe)?,
        (DataType::Binary { .. }, ArrowDataType::Binary) => new_binary_decoder(column, stripe)?,
        (
            DataType::Decimal {
                precision, scale, ..
            },
            ArrowDataType::Decimal128(a_precision, a_scale),
        ) if *precision as u8 == *a_precision && *scale as i8 == *a_scale => {
            new_decimal_decoder(column, stripe, *precision, *scale)
        }
        (DataType::Timestamp { .. }, field_type) => {
            new_timestamp_decoder(column, field_type.clone(), stripe)?
        }
        (DataType::TimestampWithLocalTimezone { .. }, field_type) => {
            new_timestamp_instant_decoder(column, field_type.clone(), stripe)?
        }
        (DataType::Date { .. }, ArrowDataType::Date32) => {
            // TODO: allow Date64
            let iter = stripe.stream_map().get(column, Kind::Data);
            let iter = get_signed_int_decoder(iter, column.rle_version());
            let present = PresentDecoder::from_stripe(stripe, column);
            Box::new(DateArrayDecoder::new(iter, present))
        }
        (DataType::Struct { .. }, ArrowDataType::Struct(fields)) => {
            Box::new(StructArrayDecoder::new(column, fields.clone(), stripe)?)
        }
        (DataType::List { .. }, ArrowDataType::List(field)) => {
            // TODO: add support for ArrowDataType::LargeList
            Box::new(ListArrayDecoder::new(column, field.clone(), stripe)?)
        }
        (DataType::Map { .. }, ArrowDataType::Map(entries, sorted)) => {
            ensure!(!sorted, UnsupportedTypeVariantSnafu { msg: "Sorted map" });
            let ArrowDataType::Struct(entries) = entries.data_type() else {
                UnexpectedSnafu {
                    msg: "arrow Map with non-Struct entry type".to_owned(),
                }
                .fail()?
            };
            ensure!(
                entries.len() == 2,
                UnexpectedSnafu {
                    msg: format!(
                        "arrow Map with {} columns per entry (expected 2)",
                        entries.len()
                    )
                }
            );
            let keys_field = entries[0].clone();
            let values_field = entries[1].clone();

            Box::new(MapArrayDecoder::new(
                column,
                keys_field,
                values_field,
                stripe,
            )?)
        }
        (DataType::Union { .. }, ArrowDataType::Union(fields, _)) => {
            Box::new(UnionArrayDecoder::new(column, fields.clone(), stripe)?)
        }
        (data_type, field_type) => {
            return MismatchedSchemaSnafu {
                orc_type: data_type.clone(),
                arrow_type: field_type.clone(),
            }
            .fail()
        }
    };

    Ok(decoder)
}

impl NaiveStripeDecoder {
    fn inner_decode_next_batch(&mut self, remaining: usize) -> Result<Vec<ArrayRef>> {
        let chunk = self.batch_size.min(remaining);

        let mut fields = Vec::with_capacity(self.stripe.columns().len());

        for decoder in &mut self.decoders {
            let array = decoder.next_batch(chunk, None)?;
            if array.is_empty() {
                break;
            } else {
                fields.push(array);
            }
        }

        Ok(fields)
    }

    fn decode_next_batch(&mut self, remaining: usize) -> Result<Option<RecordBatch>> {
        let fields = self.inner_decode_next_batch(remaining)?;

        if fields.is_empty() {
            if remaining == 0 {
                Ok(None)
            } else {
                // In case of empty projection, we need to create a RecordBatch with `row_count` only
                // to reflect the row number
                Ok(Some(
                    RecordBatch::try_new_with_options(
                        Arc::clone(&self.schema_ref),
                        fields,
                        &RecordBatchOptions::new()
                            .with_row_count(Some(self.batch_size.min(remaining))),
                    )
                    .context(error::ConvertRecordBatchSnafu)?,
                ))
            }
        } else {
            //TODO(weny): any better way?
            let fields = self
                .schema_ref
                .fields
                .into_iter()
                .map(|field| field.name())
                .zip(fields)
                .collect::<Vec<_>>();

            Ok(Some(
                RecordBatch::try_from_iter(fields).context(error::ConvertRecordBatchSnafu)?,
            ))
        }
    }

    pub fn new(stripe: Stripe, schema_ref: SchemaRef, batch_size: usize) -> Result<Self> {
        Self::new_with_selection(stripe, schema_ref, batch_size, None)
    }

    pub fn new_with_selection(
        stripe: Stripe,
        schema_ref: SchemaRef,
        batch_size: usize,
        row_selection: Option<RowSelection>,
    ) -> Result<Self> {
        let number_of_rows = stripe.number_of_rows();
        let decoders = stripe
            .columns()
            .iter()
            .zip(schema_ref.fields.iter())
            .map(|(col, field)| array_decoder_factory(col, field.data_type(), &stripe))
            .collect::<Result<Vec<_>>>()?;

        Ok(Self {
            stripe,
            schema_ref,
            decoders,
            index: 0,
            batch_size,
            number_of_rows,
            row_selection,
            selection_index: 0,
        })
    }

    /// Skip the specified number of rows by calling skip_values on each decoder
    fn skip_rows(&mut self, count: usize) -> Result<()> {
        // Call skip_values on each decoder to efficiently skip rows
        // Top-level decoders don't have parent_present
        for decoder in &mut self.decoders {
            decoder.skip_values(count, None)?;
        }
        Ok(())
    }
}