deltalake-core 0.32.0

Native Delta Lake implementation in Rust
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
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
//! Merge Barrier determines which files have modifications during the merge operation
//!
//! For every unique path in the input stream, a barrier is established. If any
//! single record for a file contains any delete, update, or insert operations
//! then the barrier for the file is opened and can be sent downstream.
//! To determine if a file contains zero changes, the input stream is
//! exhausted. Afterwards, records are then dropped.
//!
//! Bookkeeping is maintained to determine which files have modifications, so
//! they can be removed from the delta log.

use std::{
    collections::HashMap,
    pin::Pin,
    sync::Arc,
    task::{Context, Poll},
};

use arrow::array::{Array, ArrayRef, RecordBatch, builder::UInt64Builder};
use arrow::datatypes::SchemaRef;
use dashmap::DashSet;
use datafusion::common::{DataFusionError, Result as DataFusionResult};
use datafusion::logical_expr::{Expr, LogicalPlan, UserDefinedLogicalNodeCore};
use datafusion::physical_expr::{Distribution, PhysicalExpr};
use datafusion::physical_plan::{
    DisplayAs, DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream,
};
use futures::{Stream, StreamExt};

use crate::{
    DeltaTableError,
    delta_datafusion::get_path_column,
    operations::merge::{TARGET_DELETE_COLUMN, TARGET_INSERT_COLUMN, TARGET_UPDATE_COLUMN},
};

pub(crate) type BarrierSurvivorSet = Arc<DashSet<String>>;

#[derive(Debug)]
/// Physical Node for the MergeBarrier
/// Batches to this node must be repartitioned on col('deleta_rs_path').
/// Each record batch then undergoes further partitioning based on the file column to it's corresponding barrier
pub struct MergeBarrierExec {
    input: Arc<dyn ExecutionPlan>,
    file_column: Arc<String>,
    survivors: BarrierSurvivorSet,
    expr: Arc<dyn PhysicalExpr>,
}

impl MergeBarrierExec {
    /// Create a new MergeBarrierExec Node
    pub fn new(
        input: Arc<dyn ExecutionPlan>,
        file_column: Arc<String>,
        expr: Arc<dyn PhysicalExpr>,
    ) -> Self {
        MergeBarrierExec {
            input,
            file_column,
            survivors: Arc::new(DashSet::new()),
            expr,
        }
    }

    /// Files that have modifications to them and need to removed from the delta log
    pub fn survivors(&self) -> BarrierSurvivorSet {
        self.survivors.clone()
    }
}

impl ExecutionPlan for MergeBarrierExec {
    fn name(&self) -> &str {
        Self::static_name()
    }

    fn as_any(&self) -> &dyn std::any::Any {
        self
    }

    fn schema(&self) -> arrow_schema::SchemaRef {
        self.input.schema()
    }

    fn properties(&self) -> &Arc<datafusion::physical_plan::PlanProperties> {
        self.input.properties()
    }

    fn required_input_distribution(&self) -> Vec<Distribution> {
        vec![Distribution::HashPartitioned(vec![self.expr.clone()]); 1]
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        vec![&self.input]
    }

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> datafusion::common::Result<Arc<dyn ExecutionPlan>> {
        if children.len() != 1 {
            return Err(DataFusionError::Plan(
                "MergeBarrierExec wrong number of children".to_string(),
            ));
        }
        Ok(Arc::new(MergeBarrierExec::new(
            children[0].clone(),
            self.file_column.clone(),
            self.expr.clone(),
        )))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<datafusion::execution::TaskContext>,
    ) -> datafusion::common::Result<datafusion::physical_plan::SendableRecordBatchStream> {
        let input = self.input.execute(partition, context)?;
        Ok(Box::pin(MergeBarrierStream::new(
            input,
            self.schema(),
            self.survivors.clone(),
            self.file_column.clone(),
        )))
    }
}

impl DisplayAs for MergeBarrierExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default
            | DisplayFormatType::Verbose
            | DisplayFormatType::TreeRender => {
                write!(f, "MergeBarrier",)?;
                Ok(())
            }
        }
    }
}

#[derive(Debug)]
enum State {
    Feed,
    Drain,
    Finalize,
    Abort,
    Done,
}

#[derive(Debug)]
enum PartitionBarrierState {
    Closed,
    Open,
}

#[derive(Debug)]
struct MergeBarrierPartition {
    state: PartitionBarrierState,
    buffer: Vec<RecordBatch>,
    file_name: Option<String>,
}

impl MergeBarrierPartition {
    pub fn new(file_name: Option<String>) -> Self {
        MergeBarrierPartition {
            state: PartitionBarrierState::Closed,
            buffer: Vec::new(),
            file_name,
        }
    }

    pub fn feed(&mut self, batch: RecordBatch) -> DataFusionResult<()> {
        match self.state {
            PartitionBarrierState::Closed => {
                let delete_count = get_count(&batch, TARGET_DELETE_COLUMN)?;
                let update_count = get_count(&batch, TARGET_UPDATE_COLUMN)?;
                let insert_count = get_count(&batch, TARGET_INSERT_COLUMN)?;
                self.buffer.push(batch);

                if insert_count > 0 || update_count > 0 || delete_count > 0 {
                    self.state = PartitionBarrierState::Open;
                }
            }
            PartitionBarrierState::Open => {
                self.buffer.push(batch);
            }
        }
        Ok(())
    }

    pub fn drain(&mut self) -> Option<RecordBatch> {
        match self.state {
            PartitionBarrierState::Closed => None,
            PartitionBarrierState::Open => self.buffer.pop(),
        }
    }
}

struct MergeBarrierStream {
    schema: SchemaRef,
    state: State,
    input: SendableRecordBatchStream,
    file_column: Arc<String>,
    survivors: BarrierSurvivorSet,
    map: HashMap<String, usize>,
    file_partitions: Vec<MergeBarrierPartition>,
}

impl MergeBarrierStream {
    pub fn new(
        input: SendableRecordBatchStream,
        schema: SchemaRef,
        survivors: BarrierSurvivorSet,
        file_column: Arc<String>,
    ) -> Self {
        // Always allocate for a null bucket at index 0;
        let file_partitions = vec![MergeBarrierPartition::new(None)];

        MergeBarrierStream {
            schema,
            state: State::Feed,
            input,
            file_column,
            survivors,
            file_partitions,
            map: HashMap::new(),
        }
    }
}

fn get_count(batch: &RecordBatch, column: &str) -> DataFusionResult<usize> {
    batch
        .column_by_name(column)
        .map(|array| array.null_count())
        .ok_or_else(|| {
            DataFusionError::External(Box::new(DeltaTableError::Generic(
                "Required operation column is missing".to_string(),
            )))
        })
}

impl Stream for MergeBarrierStream {
    type Item = DataFusionResult<RecordBatch>;

    fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
        loop {
            match self.state {
                State::Feed => {
                    match self.input.poll_next_unpin(cx) {
                        Poll::Ready(Some(Ok(batch))) => {
                            let file_dictionary = get_path_column(&batch, &self.file_column)?;

                            // For each record batch, the key for a file path is not stable.
                            // We can iterate through the dictionary and lookup the correspond string for each record and then lookup the correct `file_partition` for that value.
                            // However this approach exposes the cost of hashing so we want to minimize that as much as possible.
                            // A map from an arrow dictionary key to the correct index of `file_partition` is created for each batch that's processed.
                            // This ensures we only need to hash each file path at most once per batch.
                            let mut key_map = Vec::with_capacity(file_dictionary.len());

                            for file_name in file_dictionary.values().into_iter() {
                                let key = match file_name {
                                    Some(name) => {
                                        if !self.map.contains_key(name) {
                                            let key = self.file_partitions.len();
                                            let part_stream =
                                                MergeBarrierPartition::new(Some(name.to_string()));
                                            self.file_partitions.push(part_stream);
                                            self.map.insert(name.to_string(), key);
                                        }
                                        // Safe unwrap due to the above
                                        *self.map.get(name).unwrap()
                                    }
                                    None => 0,
                                };
                                key_map.push(key)
                            }

                            let mut indices: Vec<_> =
                                Vec::with_capacity(self.file_partitions.len());
                            for _ in 0..self.file_partitions.len() {
                                indices.push(UInt64Builder::with_capacity(batch.num_rows()));
                            }

                            for (idx, key) in file_dictionary.keys().iter().enumerate() {
                                match key {
                                    Some(value) => {
                                        indices[key_map[value as usize]].append_value(idx as u64)
                                    }
                                    None => indices[0].append_value(idx as u64),
                                }
                            }

                            let batches: Vec<Result<(usize, RecordBatch), DataFusionError>> =
                                indices
                                    .into_iter()
                                    .enumerate()
                                    .filter_map(|(partition, mut indices)| {
                                        let indices = indices.finish();
                                        (!indices.is_empty()).then_some((partition, indices))
                                    })
                                    .map(move |(partition, indices)| {
                                        // Produce batches based on indices
                                        let columns = batch
                                            .columns()
                                            .iter()
                                            .map(|c| {
                                                Ok(arrow::compute::take(
                                                    c.as_ref(),
                                                    &indices,
                                                    None,
                                                )?)
                                            })
                                            .collect::<DataFusionResult<Vec<ArrayRef>>>()?;

                                        // This unwrap is safe since the processed batched has the same schema
                                        let batch =
                                            RecordBatch::try_new(batch.schema(), columns).unwrap();

                                        Ok((partition, batch))
                                    })
                                    .collect();

                            for batch in batches {
                                match batch {
                                    Ok((partition, batch)) => {
                                        self.file_partitions[partition].feed(batch)?;
                                    }
                                    Err(err) => {
                                        self.state = State::Abort;
                                        return Poll::Ready(Some(Err(err)));
                                    }
                                }
                            }

                            self.state = State::Drain;
                            continue;
                        }
                        Poll::Ready(Some(Err(err))) => {
                            self.state = State::Abort;
                            return Poll::Ready(Some(Err(err)));
                        }
                        Poll::Ready(None) => {
                            self.state = State::Finalize;
                            continue;
                        }
                        Poll::Pending => return Poll::Pending,
                    }
                }
                State::Drain => {
                    for part in &mut self.file_partitions {
                        if let Some(batch) = part.drain() {
                            return Poll::Ready(Some(Ok(batch)));
                        }
                    }

                    self.state = State::Feed;
                    continue;
                }
                State::Finalize => {
                    for part in &mut self.file_partitions {
                        if let Some(batch) = part.drain() {
                            return Poll::Ready(Some(Ok(batch)));
                        }
                    }

                    {
                        for part in &self.file_partitions {
                            match part.state {
                                PartitionBarrierState::Closed => {}
                                PartitionBarrierState::Open => {
                                    if let Some(file_name) = &part.file_name {
                                        self.survivors.insert(file_name.to_owned());
                                    }
                                }
                            }
                        }
                    }

                    self.state = State::Done;
                    continue;
                }
                State::Abort => return Poll::Ready(None),
                State::Done => return Poll::Ready(None),
            }
        }
    }

    fn size_hint(&self) -> (usize, Option<usize>) {
        (0, self.input.size_hint().1)
    }
}

impl RecordBatchStream for MergeBarrierStream {
    fn schema(&self) -> SchemaRef {
        self.schema.clone()
    }
}

#[derive(Debug, Hash, Eq, PartialEq, PartialOrd)]
pub(crate) struct MergeBarrier {
    pub input: LogicalPlan,
    pub expr: Expr,
    pub file_column: Arc<String>,
}

impl UserDefinedLogicalNodeCore for MergeBarrier {
    fn name(&self) -> &str {
        "MergeBarrier"
    }

    fn inputs(&self) -> Vec<&datafusion::logical_expr::LogicalPlan> {
        vec![&self.input]
    }

    fn schema(&self) -> &datafusion::common::DFSchemaRef {
        self.input.schema()
    }

    fn expressions(&self) -> Vec<datafusion::logical_expr::Expr> {
        vec![self.expr.clone()]
    }

    fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
        write!(f, "MergeBarrier")
    }

    fn with_exprs_and_inputs(
        &self,
        exprs: Vec<datafusion::logical_expr::Expr>,
        inputs: Vec<datafusion::logical_expr::LogicalPlan>,
    ) -> DataFusionResult<Self> {
        Ok(MergeBarrier {
            input: inputs[0].clone(),
            file_column: self.file_column.clone(),
            expr: exprs[0].clone(),
        })
    }
}

pub(crate) fn find_node<T: 'static>(
    parent: &Arc<dyn ExecutionPlan>,
) -> Option<Arc<dyn ExecutionPlan>> {
    //! Used to locate a Node::<T> after the planner converts the logical node
    if parent.as_any().downcast_ref::<T>().is_some() {
        return Some(parent.to_owned());
    }

    for child in &parent.children() {
        let res = find_node::<T>(child);
        if res.is_some() {
            return res;
        }
    }

    None
}

#[cfg(test)]
mod tests {
    use super::BarrierSurvivorSet;
    use crate::operations::merge::MergeBarrierExec;
    use crate::operations::merge::{
        TARGET_DELETE_COLUMN, TARGET_INSERT_COLUMN, TARGET_UPDATE_COLUMN,
    };
    use arrow::datatypes::Schema as ArrowSchema;
    use arrow_array::RecordBatch;
    use arrow_array::StringArray;
    use arrow_array::{DictionaryArray, UInt16Array};
    use arrow_schema::DataType as ArrowDataType;
    use arrow_schema::Field;
    use datafusion::assert_batches_sorted_eq;
    use datafusion::datasource::memory::MemorySourceConfig;
    use datafusion::execution::TaskContext;
    use datafusion::physical_expr::expressions::Column;
    use datafusion::physical_plan::ExecutionPlan;
    use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec;
    use futures::StreamExt;
    use std::sync::Arc;

    #[tokio::test]
    async fn test_barrier() {
        // Validate that files without modifications are dropped and that files with changes passthrough
        // File 0: No Changes
        // File 1: Contains an update
        // File 2: Contains a delete
        // null (id: 3): is a insert

        let schema = get_schema();
        let keys = UInt16Array::from(vec![Some(0), Some(1), Some(2), None]);
        let values = StringArray::from(vec![Some("file0"), Some("file1"), Some("file2")]);
        let dict = DictionaryArray::new(keys, Arc::new(values));

        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(arrow::array::StringArray::from(vec!["0", "1", "2", "3"])),
                Arc::new(dict),
                //insert column
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    Some(false),
                    Some(false),
                    None,
                ])),
                //update column
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    None,
                    Some(false),
                    Some(false),
                ])),
                //delete column
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    Some(false),
                    None,
                    Some(false),
                ])),
            ],
        )
        .unwrap();

        let (actual, survivors) = execute(vec![batch]).await;
        let expected = vec![
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
            "| id | __delta_rs_path | __delta_rs_target_insert | __delta_rs_target_update | __delta_rs_target_delete |",
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
            "| 1  | file1           | false                    |                          | false                    |",
            "| 2  | file2           | false                    | false                    |                          |",
            "| 3  |                 |                          | false                    | false                    |",
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
        ];
        assert_batches_sorted_eq!(&expected, &actual);

        assert!(!survivors.contains(&"file0".to_string()));
        assert!(survivors.contains(&"file1".to_string()));
        assert!(survivors.contains(&"file2".to_string()));
        assert_eq!(survivors.len(), 2);
    }

    #[tokio::test]
    async fn test_barrier_changing_indices() {
        // Validate implementation can handle different dictionary indices between batches

        let schema = get_schema();
        let mut batches = vec![];

        // Batch 1
        let keys = UInt16Array::from(vec![Some(0), Some(1)]);
        let values = StringArray::from(vec![Some("file0"), Some("file1")]);
        let dict = DictionaryArray::new(keys, Arc::new(values));

        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(arrow::array::StringArray::from(vec!["0", "1"])),
                Arc::new(dict),
                //insert column
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    Some(false),
                ])),
                //update column
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    Some(false),
                ])),
                //delete column
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    Some(false),
                ])),
            ],
        )
        .unwrap();
        batches.push(batch);
        // Batch 2

        let keys = UInt16Array::from(vec![Some(0), Some(1)]);
        let values = StringArray::from(vec![Some("file1"), Some("file0")]);
        let dict = DictionaryArray::new(keys, Arc::new(values));
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(arrow::array::StringArray::from(vec!["2", "3"])),
                Arc::new(dict),
                //insert column
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    Some(false),
                ])),
                //update column
                Arc::new(arrow::array::BooleanArray::from(vec![None, Some(false)])),
                //delete column
                Arc::new(arrow::array::BooleanArray::from(vec![Some(false), None])),
            ],
        )
        .unwrap();
        batches.push(batch);

        let (actual, _survivors) = execute(batches).await;
        let expected = vec![
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
            "| id | __delta_rs_path | __delta_rs_target_insert | __delta_rs_target_update | __delta_rs_target_delete |",
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
            "| 0  | file0           | false                    | false                    | false                    |",
            "| 1  | file1           | false                    | false                    | false                    |",
            "| 2  | file1           | false                    |                          | false                    |",
            "| 3  | file0           | false                    | false                    |                          |",
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
        ];
        assert_batches_sorted_eq!(&expected, &actual);
    }

    #[tokio::test]
    async fn test_barrier_null_paths() {
        // Arrow dictionaries are interesting since a null value can be either in the keys of the dict or in the values.
        // Validate they can be processed without issue

        let schema = get_schema();
        let keys = UInt16Array::from(vec![Some(0), None, Some(1)]);
        let values = StringArray::from(vec![Some("file1"), None]);
        let dict = DictionaryArray::new(keys, Arc::new(values));

        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(arrow::array::StringArray::from(vec!["1", "2", "3"])),
                Arc::new(dict),
                Arc::new(arrow::array::BooleanArray::from(vec![
                    Some(false),
                    None,
                    None,
                ])),
                Arc::new(arrow::array::BooleanArray::from(vec![false, false, false])),
                Arc::new(arrow::array::BooleanArray::from(vec![false, false, false])),
            ],
        )
        .unwrap();

        let (actual, _) = execute(vec![batch]).await;
        let expected = vec![
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
            "| id | __delta_rs_path | __delta_rs_target_insert | __delta_rs_target_update | __delta_rs_target_delete |",
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
            "| 2  |                 |                          | false                    | false                    |",
            "| 3  |                 |                          | false                    | false                    |",
            "+----+-----------------+--------------------------+--------------------------+--------------------------+",
        ];
        assert_batches_sorted_eq!(&expected, &actual);
    }

    async fn execute(input: Vec<RecordBatch>) -> (Vec<RecordBatch>, BarrierSurvivorSet) {
        let schema = get_schema();
        let repartition = Arc::new(Column::new("__delta_rs_path", 2));
        let exec = MemorySourceConfig::try_new_exec(&[input], schema.clone(), None).unwrap();

        let task_ctx = Arc::new(TaskContext::default());
        let merge =
            MergeBarrierExec::new(exec, Arc::new("__delta_rs_path".to_string()), repartition);

        let survivors = merge.survivors();
        let coalescence = CoalesceBatchesExec::new(Arc::new(merge), 100);
        let mut stream = coalescence.execute(0, task_ctx).unwrap();
        (vec![stream.next().await.unwrap().unwrap()], survivors)
    }

    fn get_schema() -> Arc<ArrowSchema> {
        Arc::new(ArrowSchema::new(vec![
            Field::new("id", ArrowDataType::Utf8, true),
            Field::new(
                "__delta_rs_path",
                ArrowDataType::Dictionary(
                    Box::new(ArrowDataType::UInt16),
                    Box::new(ArrowDataType::Utf8),
                ),
                true,
            ),
            Field::new(TARGET_INSERT_COLUMN, ArrowDataType::Boolean, true),
            Field::new(TARGET_UPDATE_COLUMN, ArrowDataType::Boolean, true),
            Field::new(TARGET_DELETE_COLUMN, ArrowDataType::Boolean, true),
        ]))
    }
}