lance 0.19.2

A columnar data format that is 100x faster than Parquet for random access.
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
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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::{collections::HashSet, sync::Arc};

use crate::io::commit::commit_transaction;
use crate::{io::exec::Planner, Error, Result};
use arrow::compute::CastOptions;
use arrow_array::{RecordBatch, RecordBatchReader};
use arrow_schema::{DataType, Field as ArrowField, Schema as ArrowSchema};
use datafusion::execution::SendableRecordBatchStream;
use futures::stream::{StreamExt, TryStreamExt};
use lance_arrow::SchemaExt;
use lance_core::datatypes::{Field, Schema};
use lance_datafusion::utils::reader_to_stream;
use lance_table::format::Fragment;
use snafu::{location, Location};

use super::fragment::FileFragment;
use super::{
    transaction::{Operation, Transaction},
    Dataset,
};

#[derive(Debug, Clone, PartialEq)]
pub struct BatchInfo {
    pub fragment_id: u32,
    pub batch_index: usize,
}

/// A mechanism for saving UDF results.
///
/// This is used to determine if a UDF has already been run on a given input,
/// and to store the results of a UDF for future use.
pub trait UDFCheckpointStore: Send + Sync {
    fn get_batch(&self, info: &BatchInfo) -> Result<Option<RecordBatch>>;
    fn insert_batch(&self, info: BatchInfo, batch: RecordBatch) -> Result<()>;
    fn get_fragment(&self, fragment_id: u32) -> Result<Option<Fragment>>;
    fn insert_fragment(&self, fragment: Fragment) -> Result<()>;
}

pub struct BatchUDF {
    #[allow(clippy::type_complexity)]
    pub mapper: Box<dyn Fn(&RecordBatch) -> Result<RecordBatch> + Send + Sync>,
    /// The schema of the returned RecordBatch
    pub output_schema: Arc<ArrowSchema>,
    /// A checkpoint store for the UDF results
    pub result_checkpoint: Option<Arc<dyn UDFCheckpointStore>>,
}

/// A way to define one or more new columns in a dataset
pub enum NewColumnTransform {
    /// A UDF that takes a RecordBatch of existing data and returns a
    /// RecordBatch with the new columns for those corresponding rows. The returned
    /// batch must return the same number of rows as the input batch.
    BatchUDF(BatchUDF),
    /// A set of SQL expressions that define new columns.
    SqlExpressions(Vec<(String, String)>),
    /// A stream of RecordBatches that define new columns.
    Stream(SendableRecordBatchStream),
    /// An iterator of RecordBatches that define new columns.
    Reader(Box<dyn RecordBatchReader + Send>),
}

/// Definition of a change to a column in a dataset
pub struct ColumnAlteration {
    /// Path to the existing column to be altered.
    pub path: String,
    /// The new name of the column. If None, the column name will not be changed.
    pub rename: Option<String>,
    /// Whether the column is nullable. If None, the nullability will not be changed.
    pub nullable: Option<bool>,
    /// The new data type of the column. If None, the data type will not be changed.
    pub data_type: Option<DataType>,
}

impl ColumnAlteration {
    pub fn new(path: String) -> Self {
        Self {
            path,
            rename: None,
            nullable: None,
            data_type: None,
        }
    }

    pub fn rename(mut self, name: String) -> Self {
        self.rename = Some(name);
        self
    }

    pub fn set_nullable(mut self, nullable: bool) -> Self {
        self.nullable = Some(nullable);
        self
    }

    pub fn cast_to(mut self, data_type: DataType) -> Self {
        self.data_type = Some(data_type);
        self
    }
}

/// Limit casts to same type. This is mostly to filter out weird casts like
/// casting a string to a boolean or float to string.
fn is_upcast_downcast(from_type: &DataType, to_type: &DataType) -> bool {
    use DataType::*;
    match from_type {
        from_type if from_type.is_integer() => to_type.is_integer(),
        from_type if from_type.is_floating() => to_type.is_floating(),
        from_type if from_type.is_temporal() => to_type.is_temporal(),
        Boolean => matches!(to_type, Boolean),
        Utf8 | LargeUtf8 => matches!(to_type, Utf8 | LargeUtf8),
        Binary | LargeBinary => matches!(to_type, Binary | LargeBinary),
        Decimal128(_, _) | Decimal256(_, _) => {
            matches!(to_type, Decimal128(_, _) | Decimal256(_, _))
        }
        List(from_field) | LargeList(from_field) | FixedSizeList(from_field, _) => match to_type {
            List(to_field) | LargeList(to_field) | FixedSizeList(to_field, _) => {
                is_upcast_downcast(from_field.data_type(), to_field.data_type())
            }
            _ => false,
        },
        _ => false,
    }
}

pub(super) async fn add_columns_to_fragments(
    dataset: &Dataset,
    transforms: NewColumnTransform,
    read_columns: Option<Vec<String>>,
    fragments: &[FileFragment],
    batch_size: Option<u32>,
) -> Result<(Vec<Fragment>, Schema)> {
    // Check names early (before calling add_columns_impl) to avoid extra work if
    // the names are wrong.
    let check_names = |output_schema: &ArrowSchema| {
        let new_names = output_schema.field_names();
        for field in &dataset.schema().fields {
            if new_names.contains(&&field.name) {
                return Err(Error::invalid_input(
                    format!("Column {} already exists in the dataset", field.name),
                    location!(),
                ));
            }
        }
        Ok(())
    };

    let (output_schema, fragments) = match transforms {
        NewColumnTransform::BatchUDF(udf) => {
            check_names(udf.output_schema.as_ref())?;
            let fragments = add_columns_impl(
                fragments,
                read_columns,
                udf.mapper,
                batch_size,
                udf.result_checkpoint,
                None,
            )
            .await?;
            Result::Ok((udf.output_schema, fragments))
        }
        NewColumnTransform::SqlExpressions(expressions) => {
            // We just transform the SQL expression into a UDF backed by DataFusion
            // physical expressions.
            let arrow_schema = Arc::new(ArrowSchema::from(dataset.schema()));
            let planner = Planner::new(arrow_schema);
            let exprs = expressions
                .into_iter()
                .map(|(name, expr)| {
                    let expr = planner.parse_expr(&expr)?;
                    let expr = planner.optimize_expr(expr)?;
                    Ok((name, expr))
                })
                .collect::<Result<Vec<_>>>()?;

            let needed_columns = exprs
                .iter()
                .flat_map(|(_, expr)| Planner::column_names_in_expr(expr))
                .collect::<HashSet<_>>()
                .into_iter()
                .collect::<Vec<_>>();
            let read_schema = dataset.schema().project(&needed_columns)?;
            let read_schema = Arc::new(ArrowSchema::from(&read_schema));
            // Need to re-create the planner with the read schema because physical
            // expressions use positional column references.
            let planner = Planner::new(read_schema.clone());
            let exprs = exprs
                .into_iter()
                .map(|(name, expr)| {
                    let expr = planner.create_physical_expr(&expr)?;
                    Ok((name, expr))
                })
                .collect::<Result<Vec<_>>>()?;

            let output_schema = Arc::new(ArrowSchema::new(
                exprs
                    .iter()
                    .map(|(name, expr)| {
                        Ok(ArrowField::new(
                            name,
                            expr.data_type(read_schema.as_ref())?,
                            expr.nullable(read_schema.as_ref())?,
                        ))
                    })
                    .collect::<Result<Vec<_>>>()?,
            ));
            check_names(output_schema.as_ref())?;

            let schema_ref = output_schema.clone();
            let mapper = move |batch: &RecordBatch| {
                let num_rows = batch.num_rows();
                let columns = exprs
                    .iter()
                    .map(|(_, expr)| Ok(expr.evaluate(batch)?.into_array(num_rows)?))
                    .collect::<Result<Vec<_>>>()?;

                let batch = RecordBatch::try_new(schema_ref.clone(), columns)?;
                Ok(batch)
            };
            let mapper = Box::new(mapper);

            let read_columns = Some(read_schema.field_names().into_iter().cloned().collect());
            let fragments =
                add_columns_impl(fragments, read_columns, mapper, batch_size, None, None).await?;
            Ok((output_schema, fragments))
        }
        NewColumnTransform::Stream(stream) => {
            let output_schema = stream.schema();
            check_names(output_schema.as_ref())?;
            let fragments = add_columns_from_stream(fragments, stream, None, batch_size).await?;
            Ok((output_schema, fragments))
        }
        NewColumnTransform::Reader(reader) => {
            let output_schema = reader.schema();
            check_names(output_schema.as_ref())?;
            let stream = reader_to_stream(reader);
            let fragments = add_columns_from_stream(fragments, stream, None, batch_size).await?;
            Ok((output_schema, fragments))
        }
    }?;

    let mut schema = dataset.schema().merge(output_schema.as_ref())?;
    schema.set_field_id(Some(dataset.manifest.max_field_id()));

    Ok((fragments, schema))
}

pub(super) async fn add_columns(
    dataset: &mut Dataset,
    transforms: NewColumnTransform,
    read_columns: Option<Vec<String>>,
    batch_size: Option<u32>,
) -> Result<()> {
    let (fragments, schema) = add_columns_to_fragments(
        dataset,
        transforms,
        read_columns,
        &dataset.get_fragments(),
        batch_size,
    )
    .await?;

    let operation = Operation::Merge { fragments, schema };
    let transaction = Transaction::new(
        dataset.manifest.version,
        operation,
        // TODO: Make it possible to add new blob columns
        /*blob_op= */ None,
        None,
    );
    let new_manifest = commit_transaction(
        dataset,
        &dataset.object_store,
        dataset.commit_handler.as_ref(),
        &transaction,
        &Default::default(),
        &Default::default(),
        dataset.manifest_naming_scheme,
    )
    .await?;

    dataset.manifest = Arc::new(new_manifest);

    Ok(())
}

#[allow(clippy::type_complexity)]
async fn add_columns_impl(
    fragments: &[FileFragment],
    read_columns: Option<Vec<String>>,
    mapper: Box<dyn Fn(&RecordBatch) -> Result<RecordBatch> + Send + Sync>,
    batch_size: Option<u32>,
    result_cache: Option<Arc<dyn UDFCheckpointStore>>,
    schemas: Option<(Schema, Schema)>,
) -> Result<Vec<Fragment>> {
    let read_columns_ref = read_columns.as_deref();
    let mapper_ref = mapper.as_ref();
    let fragments = futures::stream::iter(fragments)
        .then(|fragment| {
            let cache_ref = result_cache.clone();
            let schemas_ref = &schemas;
            async move {
                if let Some(cache) = &cache_ref {
                    let fragment_id = fragment.id() as u32;
                    let fragment = cache.get_fragment(fragment_id)?;
                    if let Some(fragment) = fragment {
                        return Ok(fragment);
                    }
                }

                let mut updater = fragment
                    .updater(read_columns_ref, schemas_ref.clone(), batch_size)
                    .await?;

                let mut batch_index = 0;
                // TODO: the structure of the updater prevents batch-level parallelism here,
                //       but there is no reason why we couldn't do this in parallel.
                while let Some(batch) = updater.next().await? {
                    let batch_info = BatchInfo {
                        fragment_id: fragment.id() as u32,
                        batch_index,
                    };

                    let new_batch = if let Some(cache) = &cache_ref {
                        if let Some(batch) = cache.get_batch(&batch_info)? {
                            batch
                        } else {
                            let new_batch = mapper_ref(batch)?;
                            cache.insert_batch(batch_info, new_batch.clone())?;
                            new_batch
                        }
                    } else {
                        mapper_ref(batch)?
                    };

                    updater.update(new_batch).await?;
                    batch_index += 1;
                }

                let fragment = updater.finish().await?;

                if let Some(cache) = &cache_ref {
                    cache.insert_fragment(fragment.clone())?;
                }

                Ok::<_, Error>(fragment)
            }
        })
        .try_collect::<Vec<_>>()
        .await?;
    Ok(fragments)
}

async fn add_columns_from_stream(
    fragments: &[FileFragment],
    mut stream: SendableRecordBatchStream,
    schemas: Option<(Schema, Schema)>,
    batch_size: Option<u32>,
) -> Result<Vec<Fragment>> {
    let mut new_fragments = Vec::with_capacity(fragments.len());
    let mut last_seen_batch: Option<RecordBatch> = None;
    for fragment in fragments {
        let mut updater = fragment
            .updater::<String>(Some(&[]), schemas.clone(), batch_size)
            .await?;
        while let Some(batch) = updater.next().await? {
            debug_assert_eq!(batch.num_columns(), 1);
            let mut rows_remaining = batch.num_rows();

            let mut batches = Vec::new();

            while rows_remaining > 0 {
                let next_batch = if let Some(last_seen_batch) = last_seen_batch {
                    last_seen_batch
                } else {
                    stream.next().await.ok_or_else(|| {
                        Error::invalid_input(
                            "Stream ended before producing values for all rows in dataset",
                            location!(),
                        )
                    })??
                };
                let num_rows = next_batch.num_rows();
                if num_rows > rows_remaining {
                    let new_batch = next_batch.slice(0, rows_remaining);
                    batches.push(new_batch);
                    last_seen_batch =
                        Some(next_batch.slice(rows_remaining, num_rows - rows_remaining));
                    rows_remaining = 0;
                } else {
                    batches.push(next_batch);
                    rows_remaining -= num_rows;
                    last_seen_batch = None;
                }
            }

            let new_batch =
                arrow_select::concat::concat_batches(&batches[0].schema(), batches.iter())?;

            updater.update(new_batch).await?;
        }
        new_fragments.push(updater.finish().await?);
    }

    // Ensure the stream is fully consumed
    if last_seen_batch.is_some() || stream.next().await.is_some() {
        return Err(Error::InvalidInput {
            source: "Stream produced more values than expected for dataset".into(),
            location: location!(),
        });
    }

    Ok(new_fragments)
}

/// Modify columns in the dataset, changing their name, type, or nullability.
///
/// If a column has an index, it's index will be preserved.
pub(super) async fn alter_columns(
    dataset: &mut Dataset,
    alterations: &[ColumnAlteration],
) -> Result<()> {
    // Validate we aren't making nullable columns non-nullable and that all
    // the referenced columns actually exist.
    let mut new_schema = dataset.schema().clone();

    // Mapping of old to new fields that need to be casted.
    let mut cast_fields: Vec<(Field, Field)> = Vec::new();

    let mut next_field_id = dataset.manifest.max_field_id() + 1;

    for alteration in alterations {
        let field_src = dataset.schema().field(&alteration.path).ok_or_else(|| {
            Error::invalid_input(
                format!(
                    "Column \"{}\" does not exist in the dataset",
                    alteration.path
                ),
                location!(),
            )
        })?;

        if !field_src.is_default_storage() {
            return Err(Error::NotSupported {
                source: format!(
                    "Column \"{}\" is not a default storage column and cannot yet be altered",
                    alteration.path
                )
                .into(),
                location: location!(),
            });
        }

        if let Some(nullable) = alteration.nullable {
            // TODO: in the future, we could check the values of the column to see if
            //       they are all non-null and thus the column could be made non-nullable.
            if field_src.nullable && !nullable {
                return Err(Error::invalid_input(
                    format!(
                        "Column \"{}\" is already nullable and thus cannot be made non-nullable",
                        alteration.path
                    ),
                    location!(),
                ));
            }
        }

        let field_dest = new_schema.mut_field_by_id(field_src.id).unwrap();
        if let Some(rename) = &alteration.rename {
            field_dest.name.clone_from(rename);
        }
        if let Some(nullable) = alteration.nullable {
            field_dest.nullable = nullable;
        }

        if let Some(data_type) = &alteration.data_type {
            if !(lance_arrow::cast::can_cast_types(&field_src.data_type(), data_type)
                && is_upcast_downcast(&field_src.data_type(), data_type))
            {
                return Err(Error::invalid_input(
                    format!(
                        "Cannot cast column \"{}\" from {:?} to {:?}",
                        alteration.path,
                        field_src.data_type(),
                        data_type
                    ),
                    location!(),
                ));
            }

            let arrow_field = ArrowField::new(
                field_dest.name.clone(),
                data_type.clone(),
                field_dest.nullable,
            );
            *field_dest = Field::try_from(&arrow_field)?;
            field_dest.set_id(field_src.parent_id, &mut next_field_id);

            cast_fields.push((field_src.clone(), field_dest.clone()));
        }
    }

    new_schema.validate()?;

    // If we aren't casting a column, we don't need to touch the fragments.
    let transaction = if cast_fields.is_empty() {
        Transaction::new(
            dataset.manifest.version,
            Operation::Project { schema: new_schema },
            // TODO: Make it possible to alter blob columns
            /*blob_op= */ None,
            None,
        )
    } else {
        // Otherwise, we need to re-write the relevant fields.
        let read_columns = cast_fields
            .iter()
            .map(|(old, _new)| {
                let parts = dataset.schema().field_ancestry_by_id(old.id).unwrap();
                let part_names = parts.iter().map(|p| p.name.clone()).collect::<Vec<_>>();
                part_names.join(".")
            })
            .collect::<Vec<_>>();

        let new_ids = cast_fields
            .iter()
            .map(|(_old, new)| new.id)
            .collect::<Vec<_>>();
        // This schema contains the exact field ids we want to write the new fields with.
        let new_col_schema = new_schema.project_by_ids(&new_ids, true);

        let mapper = move |batch: &RecordBatch| {
            let mut fields = Vec::with_capacity(cast_fields.len());
            let mut columns = Vec::with_capacity(batch.num_columns());
            for (old, new) in &cast_fields {
                let old_column = batch[&old.name].clone();
                let new_column = lance_arrow::cast::cast_with_options(
                    &old_column,
                    &new.data_type(),
                    // Safe: false means it will error if the cast is lossy.
                    &CastOptions {
                        safe: false,
                        ..Default::default()
                    },
                )?;
                columns.push(new_column);
                fields.push(Arc::new(ArrowField::from(new)));
            }
            let schema = Arc::new(ArrowSchema::new(fields));
            Ok(RecordBatch::try_new(schema, columns)?)
        };
        let mapper = Box::new(mapper);

        let fragments = add_columns_impl(
            &dataset.get_fragments(),
            Some(read_columns),
            mapper,
            None,
            None,
            Some((new_col_schema, new_schema.clone())),
        )
        .await?;

        // Some data files may no longer contain any columns in the dataset (e.g. if every
        // remaining column has been altered into a different data file) and so we remove them
        let schema_field_ids = new_schema.field_ids().into_iter().collect::<Vec<_>>();
        let fragments = fragments
            .into_iter()
            .map(|mut frag| {
                frag.files.retain(|f| {
                    f.fields
                        .iter()
                        .any(|field| schema_field_ids.contains(field))
                });
                frag
            })
            .collect::<Vec<_>>();

        Transaction::new(
            dataset.manifest.version,
            Operation::Merge {
                schema: new_schema,
                fragments,
            },
            /*blob_op= */ None,
            None,
        )
    };

    // TODO: adjust the indices here for the new schema

    let manifest = commit_transaction(
        dataset,
        &dataset.object_store,
        dataset.commit_handler.as_ref(),
        &transaction,
        &Default::default(),
        &Default::default(),
        dataset.manifest_naming_scheme,
    )
    .await?;

    dataset.manifest = Arc::new(manifest);

    Ok(())
}

/// Remove columns from the dataset.
///
/// This is a metadata-only operation and does not remove the data from the
/// underlying storage. In order to remove the data, you must subsequently
/// call `compact_files` to rewrite the data without the removed columns and
/// then call `cleanup_old_versions` to remove the old files.
pub(super) async fn drop_columns(dataset: &mut Dataset, columns: &[&str]) -> Result<()> {
    // Check if columns are present in the dataset and construct the new schema.
    for col in columns {
        if let Some(field) = dataset.schema().field(col) {
            if !field.is_default_storage() {
                return Err(Error::NotSupported {
                    source: format!(
                        "Column \"{}\" is not a default storage column and cannot yet be dropped",
                        col
                    )
                    .into(),
                    location: location!(),
                });
            }
        } else {
            return Err(Error::invalid_input(
                format!("Column {} does not exist in the dataset", col),
                location!(),
            ));
        }
    }

    let columns_to_remove = dataset.manifest.schema.project(columns)?;
    let new_schema = dataset.manifest.schema.exclude(columns_to_remove)?;

    if new_schema.fields.is_empty() {
        return Err(Error::invalid_input(
            "Cannot drop all columns from a dataset",
            location!(),
        ));
    }

    let transaction = Transaction::new(
        dataset.manifest.version,
        Operation::Project { schema: new_schema },
        /*blob_op= */ None,
        None,
    );

    let manifest = commit_transaction(
        dataset,
        &dataset.object_store,
        dataset.commit_handler.as_ref(),
        &transaction,
        &Default::default(),
        &Default::default(),
        dataset.manifest_naming_scheme,
    )
    .await?;

    dataset.manifest = Arc::new(manifest);

    Ok(())
}

#[cfg(test)]
mod test {
    use std::sync::Mutex;

    use crate::dataset::WriteParams;

    use super::*;
    use arrow_array::{Int32Array, RecordBatchIterator};
    use arrow_schema::Fields as ArrowFields;
    use lance_file::version::LanceFileVersion;
    use rstest::rstest;

    // Used to validate that futures returned are Send.
    fn require_send<T: Send>(t: T) -> T {
        t
    }

    #[tokio::test]
    async fn test_append_columns_exprs() -> Result<()> {
        let num_rows = 5;
        let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "id",
            DataType::Int32,
            false,
        )]));
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(Int32Array::from_iter_values(0..num_rows as i32))],
        )?;
        let reader = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());

        let test_dir = tempfile::tempdir()?;
        let test_uri = test_dir.path().to_str().unwrap();
        let mut dataset = Dataset::write(
            reader,
            test_uri,
            Some(WriteParams {
                data_storage_version: Some(LanceFileVersion::Legacy),
                ..Default::default()
            }),
        )
        .await?;
        dataset.validate().await?;

        // Adding a duplicate column name will break
        let fut = dataset.add_columns(
            NewColumnTransform::SqlExpressions(vec![("id".into(), "id + 1".into())]),
            None,
            None,
        );
        // (Quick validation that the future is Send)
        let res = require_send(fut).await;
        assert!(matches!(res, Err(Error::InvalidInput { .. })));

        // Can add a column that is independent of any existing ones
        dataset
            .add_columns(
                NewColumnTransform::SqlExpressions(vec![("value".into(), "2 * random()".into())]),
                None,
                None,
            )
            .await?;

        // Can add a column derived from an existing one.
        dataset
            .add_columns(
                NewColumnTransform::SqlExpressions(vec![("double_id".into(), "2 * id".into())]),
                None,
                None,
            )
            .await?;

        // Can derive a column from existing ones across multiple data files.
        dataset
            .add_columns(
                NewColumnTransform::SqlExpressions(vec![(
                    "triple_id".into(),
                    "id + double_id".into(),
                )]),
                None,
                None,
            )
            .await?;

        // These can be read back, the dataset is valid
        dataset.validate().await?;

        let data = dataset.scan().try_into_batch().await?;
        let expected_schema = ArrowSchema::new(vec![
            ArrowField::new("id", DataType::Int32, false),
            ArrowField::new("value", DataType::Float64, true),
            ArrowField::new("double_id", DataType::Int32, false),
            ArrowField::new("triple_id", DataType::Int32, false),
        ]);
        assert_eq!(data.schema().as_ref(), &expected_schema);
        assert_eq!(data.num_rows(), num_rows);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn test_append_columns_udf(
        #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
        data_storage_version: LanceFileVersion,
    ) -> Result<()> {
        use arrow_array::Float64Array;

        let num_rows = 5;
        let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "id",
            DataType::Int32,
            false,
        )]));

        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(Int32Array::from_iter_values(0..num_rows as i32))],
        )?;
        let reader = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());

        let test_dir = tempfile::tempdir()?;
        let test_uri = test_dir.path().to_str().unwrap();
        let mut dataset = Dataset::write(
            reader,
            test_uri,
            Some(WriteParams {
                data_storage_version: Some(data_storage_version),
                ..Default::default()
            }),
        )
        .await?;
        dataset.validate().await?;

        // Adding a duplicate column name will break
        let transforms = NewColumnTransform::BatchUDF(BatchUDF {
            mapper: Box::new(|_| unimplemented!()),
            output_schema: Arc::new(ArrowSchema::new(vec![ArrowField::new(
                "id",
                DataType::Int32,
                false,
            )])),
            result_checkpoint: None,
        });
        let res = dataset.add_columns(transforms, None, None).await;
        assert!(matches!(res, Err(Error::InvalidInput { .. })));

        // Can add a column that independent (empty read_schema)
        let output_schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "value",
            DataType::Float64,
            true,
        )]));
        let output_schema_ref = output_schema.clone();
        let mapper = move |batch: &RecordBatch| {
            Ok(RecordBatch::try_new(
                output_schema_ref.clone(),
                vec![Arc::new(Float64Array::from_iter_values(
                    (0..batch.num_rows()).map(|i| i as f64),
                ))],
            )?)
        };
        let transforms = NewColumnTransform::BatchUDF(BatchUDF {
            mapper: Box::new(mapper),
            output_schema,
            result_checkpoint: None,
        });
        dataset.add_columns(transforms, None, None).await?;

        // Can add a column that depends on another column (double id)
        let output_schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "double_id",
            DataType::Int32,
            false,
        )]));
        let output_schema_ref = output_schema.clone();
        let mapper = move |batch: &RecordBatch| {
            let id = batch
                .column(0)
                .as_any()
                .downcast_ref::<Int32Array>()
                .unwrap();
            Ok(RecordBatch::try_new(
                output_schema_ref.clone(),
                vec![Arc::new(Int32Array::from_iter_values(
                    id.values().iter().map(|i| i * 2),
                ))],
            )?)
        };
        let transforms = NewColumnTransform::BatchUDF(BatchUDF {
            mapper: Box::new(mapper),
            output_schema,
            result_checkpoint: None,
        });
        dataset.add_columns(transforms, None, None).await?;
        // These can be read back, the dataset is valid
        dataset.validate().await?;

        let data = dataset.scan().try_into_batch().await?;
        let expected_schema = ArrowSchema::new(vec![
            ArrowField::new("id", DataType::Int32, false),
            ArrowField::new("value", DataType::Float64, true),
            ArrowField::new("double_id", DataType::Int32, false),
        ]);
        assert_eq!(data.schema().as_ref(), &expected_schema);
        assert_eq!(data.num_rows(), num_rows);

        Ok(())
    }

    #[tokio::test]
    async fn test_append_columns_udf_cache() -> Result<()> {
        let num_rows = 100;
        let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "id",
            DataType::Int32,
            false,
        )]));

        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(Int32Array::from_iter_values(0..num_rows))],
        )?;
        let reader = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());

        let test_dir = tempfile::tempdir()?;
        let test_uri = test_dir.path().to_str().unwrap();
        let mut dataset = Dataset::write(
            reader,
            test_uri,
            Some(WriteParams {
                max_rows_per_file: 50,
                max_rows_per_group: 25,
                data_storage_version: Some(LanceFileVersion::Legacy),
                ..Default::default()
            }),
        )
        .await?;
        dataset.validate().await?;

        #[derive(Default)]
        struct RequestCounter {
            pub get_batch_requests: Mutex<Vec<BatchInfo>>,
            pub insert_batch_requests: Mutex<Vec<BatchInfo>>,
            pub get_fragment_requests: Mutex<Vec<u32>>,
            pub insert_fragment_requests: Mutex<Vec<u32>>,
        }

        impl UDFCheckpointStore for RequestCounter {
            fn get_batch(&self, info: &BatchInfo) -> Result<Option<RecordBatch>> {
                self.get_batch_requests.lock().unwrap().push(info.clone());

                if info.fragment_id == 1 && info.batch_index == 0 {
                    Ok(Some(RecordBatch::try_new(
                        Arc::new(ArrowSchema::new(vec![ArrowField::new(
                            "double_id",
                            DataType::Int32,
                            false,
                        )])),
                        vec![Arc::new(Int32Array::from_iter_values(50..75))],
                    )?))
                } else {
                    Ok(None)
                }
            }

            fn insert_batch(&self, info: BatchInfo, _value: RecordBatch) -> Result<()> {
                self.insert_batch_requests.lock().unwrap().push(info);
                Ok(())
            }

            fn get_fragment(&self, fragment_id: u32) -> Result<Option<Fragment>> {
                self.get_fragment_requests.lock().unwrap().push(fragment_id);
                if fragment_id == 0 {
                    Ok(Some(Fragment {
                        files: vec![],
                        id: 0,
                        deletion_file: None,
                        row_id_meta: None,
                        physical_rows: Some(50),
                    }))
                } else {
                    Ok(None)
                }
            }

            fn insert_fragment(&self, fragment: Fragment) -> Result<()> {
                self.insert_fragment_requests
                    .lock()
                    .unwrap()
                    .push(fragment.id as u32);
                Ok(())
            }
        }

        let request_counter = Arc::new(RequestCounter::default());

        let output_schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "double_id",
            DataType::Int32,
            false,
        )]));
        let output_schema_ref = output_schema.clone();
        let mapper = move |batch: &RecordBatch| {
            let id = batch
                .column(0)
                .as_any()
                .downcast_ref::<Int32Array>()
                .unwrap();
            Ok(RecordBatch::try_new(
                output_schema_ref.clone(),
                vec![Arc::new(Int32Array::from_iter_values(
                    id.values().iter().map(|i| i * 2),
                ))],
            )?)
        };
        let transforms = NewColumnTransform::BatchUDF(BatchUDF {
            mapper: Box::new(mapper),
            output_schema,
            result_checkpoint: Some(request_counter.clone()),
        });
        dataset.add_columns(transforms, None, None).await?;

        // Should have requested both fragments
        assert_eq!(
            request_counter
                .get_fragment_requests
                .lock()
                .unwrap()
                .as_slice(),
            &[0, 1]
        );
        // Should have only inserted the second fragment, since the first one was already cached
        assert_eq!(
            request_counter
                .insert_fragment_requests
                .lock()
                .unwrap()
                .as_slice(),
            &[1]
        );

        // Should have only requested the second two batches, since the first fragment was already cached
        assert_eq!(
            request_counter
                .get_batch_requests
                .lock()
                .unwrap()
                .as_slice(),
            &[
                BatchInfo {
                    fragment_id: 1,
                    batch_index: 0,
                },
                BatchInfo {
                    fragment_id: 1,
                    batch_index: 1,
                },
            ]
        );
        // Should have only saved the last batch, since the first batch of second fragment was already cached
        assert_eq!(
            request_counter
                .insert_batch_requests
                .lock()
                .unwrap()
                .as_slice(),
            &[BatchInfo {
                fragment_id: 1,
                batch_index: 1,
            },]
        );

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn test_rename_columns(
        #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
        data_storage_version: LanceFileVersion,
    ) -> Result<()> {
        use std::collections::HashMap;

        use arrow_array::{ArrayRef, StructArray};

        let metadata: HashMap<String, String> = [("k1".into(), "v1".into())].into();

        let schema = Arc::new(ArrowSchema::new_with_metadata(
            vec![
                ArrowField::new("a", DataType::Int32, false),
                ArrowField::new(
                    "b",
                    DataType::Struct(ArrowFields::from(vec![ArrowField::new(
                        "c",
                        DataType::Int32,
                        true,
                    )])),
                    true,
                ),
            ],
            metadata.clone(),
        ));

        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(Int32Array::from(vec![1, 2])),
                Arc::new(StructArray::from(vec![(
                    Arc::new(ArrowField::new("c", DataType::Int32, true)),
                    Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef,
                )])),
            ],
        )?;

        let test_dir = tempfile::tempdir()?;
        let test_uri = test_dir.path().to_str().unwrap();

        let batches = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());
        let mut dataset = Dataset::write(
            batches,
            test_uri,
            Some(WriteParams {
                data_storage_version: Some(data_storage_version),
                ..Default::default()
            }),
        )
        .await?;

        let original_fragments = dataset.fragments().to_vec();

        // Rename a top-level column
        dataset
            .alter_columns(&[ColumnAlteration::new("a".into())
                .rename("x".into())
                .set_nullable(true)])
            .await?;
        dataset.validate().await?;
        assert_eq!(dataset.manifest.version, 2);
        assert_eq!(dataset.fragments().as_ref(), &original_fragments);

        let expected_schema = ArrowSchema::new_with_metadata(
            vec![
                ArrowField::new("x", DataType::Int32, true),
                ArrowField::new(
                    "b",
                    DataType::Struct(ArrowFields::from(vec![ArrowField::new(
                        "c",
                        DataType::Int32,
                        true,
                    )])),
                    true,
                ),
            ],
            metadata.clone(),
        );
        assert_eq!(&ArrowSchema::from(dataset.schema()), &expected_schema);

        // Rename to duplicate name fails
        let err = dataset
            .alter_columns(&[ColumnAlteration::new("b".into()).rename("x".into())])
            .await
            .unwrap_err();
        assert!(err.to_string().contains("Duplicate field name \"x\""));

        // Rename a nested column.
        dataset
            .alter_columns(&[ColumnAlteration::new("b.c".into()).rename("d".into())])
            .await?;
        dataset.validate().await?;
        assert_eq!(dataset.manifest.version, 3);
        assert_eq!(dataset.fragments().as_ref(), &original_fragments);

        let expected_schema = ArrowSchema::new_with_metadata(
            vec![
                ArrowField::new("x", DataType::Int32, true),
                ArrowField::new(
                    "b",
                    DataType::Struct(ArrowFields::from(vec![ArrowField::new(
                        "d",
                        DataType::Int32,
                        true,
                    )])),
                    true,
                ),
            ],
            metadata.clone(),
        );
        assert_eq!(&ArrowSchema::from(dataset.schema()), &expected_schema);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn test_cast_column(
        #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
        data_storage_version: LanceFileVersion,
    ) -> Result<()> {
        // Create a table with 2 scalar columns, 1 vector column

        use arrow::datatypes::{Int32Type, Int64Type};
        use arrow_array::{Float16Array, Float32Array, Int64Array, ListArray};
        use half::f16;
        use lance_arrow::FixedSizeListArrayExt;
        use lance_index::{scalar::ScalarIndexParams, DatasetIndexExt, IndexType};
        use lance_linalg::distance::MetricType;
        use lance_testing::datagen::generate_random_array;

        use crate::index::vector::VectorIndexParams;
        let schema = Arc::new(ArrowSchema::new(vec![
            ArrowField::new("i", DataType::Int32, false),
            ArrowField::new("f", DataType::Float32, false),
            ArrowField::new(
                "vec",
                DataType::FixedSizeList(
                    Arc::new(ArrowField::new("item", DataType::Float32, true)),
                    128,
                ),
                false,
            ),
            ArrowField::new("l", DataType::new_list(DataType::Int32, true), true),
        ]));

        let nrows = 512;
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(Int32Array::from_iter_values(0..nrows)),
                Arc::new(Float32Array::from_iter_values((0..nrows).map(|i| i as f32))),
                Arc::new(
                    <arrow_array::FixedSizeListArray as FixedSizeListArrayExt>::try_new_from_values(
                        generate_random_array(128 * nrows as usize),
                        128,
                    )
                    .unwrap(),
                ),
                Arc::new(ListArray::from_iter_primitive::<Int32Type, _, _>(
                    (0..nrows).map(|i| Some(vec![Some(i), Some(i + 1)])),
                )),
            ],
        )?;

        let test_dir = tempfile::tempdir()?;
        let test_uri = test_dir.path().to_str().unwrap();

        let mut dataset = Dataset::write(
            RecordBatchIterator::new(vec![Ok(batch.clone())], schema.clone()),
            test_uri,
            Some(WriteParams {
                data_storage_version: Some(data_storage_version),
                ..Default::default()
            }),
        )
        .await?;

        let params = VectorIndexParams::ivf_pq(10, 8, 2, MetricType::L2, 50);
        dataset
            .create_index(&["vec"], IndexType::Vector, None, &params, false)
            .await?;
        dataset
            .create_index(
                &["i"],
                IndexType::Scalar,
                None,
                &ScalarIndexParams::default(),
                false,
            )
            .await?;
        dataset.validate().await?;

        let indices = dataset.load_indices().await?;
        assert_eq!(indices.len(), 2);

        // Cast a scalar column to another type, nullability
        dataset
            .alter_columns(&[ColumnAlteration::new("f".into())
                .cast_to(DataType::Float16)
                .set_nullable(true)])
            .await?;
        dataset.validate().await?;
        let expected_schema = ArrowSchema::new(vec![
            ArrowField::new("i", DataType::Int32, false),
            ArrowField::new("f", DataType::Float16, true),
            ArrowField::new(
                "vec",
                DataType::FixedSizeList(
                    Arc::new(ArrowField::new("item", DataType::Float32, true)),
                    128,
                ),
                false,
            ),
            ArrowField::new("l", DataType::new_list(DataType::Int32, true), true),
        ]);
        assert_eq!(&ArrowSchema::from(dataset.schema()), &expected_schema);

        // Each fragment gains a file with the new columns
        dataset.fragments().iter().for_each(|f| {
            assert_eq!(f.files.len(), 2);
        });

        // Cast scalar column with index, should not keep index (TODO: keep it)
        dataset
            .alter_columns(&[ColumnAlteration::new("i".into()).cast_to(DataType::Int64)])
            .await?;
        dataset.validate().await?;

        let expected_schema = ArrowSchema::new(vec![
            ArrowField::new("i", DataType::Int64, false),
            ArrowField::new("f", DataType::Float16, true),
            ArrowField::new(
                "vec",
                DataType::FixedSizeList(
                    Arc::new(ArrowField::new("item", DataType::Float32, true)),
                    128,
                ),
                false,
            ),
            ArrowField::new("l", DataType::new_list(DataType::Int32, true), true),
        ]);
        assert_eq!(&ArrowSchema::from(dataset.schema()), &expected_schema);

        // We currently lose the index when casting a column
        let indices = dataset.load_indices().await?;
        assert_eq!(indices.len(), 1);

        // Each fragment gains a file with the new columns
        dataset.fragments().iter().for_each(|f| {
            assert_eq!(f.files.len(), 3);
        });

        // Cast vector column, should not keep index (TODO: keep it)
        dataset
            .alter_columns(&[
                ColumnAlteration::new("vec".into()).cast_to(DataType::FixedSizeList(
                    Arc::new(ArrowField::new("item", DataType::Float16, true)),
                    128,
                )),
            ])
            .await?;
        dataset.validate().await?;

        // Finally, case list column to show we can handle children.
        dataset
            .alter_columns(&[ColumnAlteration::new("l".into())
                .cast_to(DataType::new_list(DataType::Int64, true))])
            .await?;
        dataset.validate().await?;

        let expected_schema = ArrowSchema::new(vec![
            ArrowField::new("i", DataType::Int64, false),
            ArrowField::new("f", DataType::Float16, true),
            ArrowField::new(
                "vec",
                DataType::FixedSizeList(
                    Arc::new(ArrowField::new("item", DataType::Float16, true)),
                    128,
                ),
                false,
            ),
            ArrowField::new("l", DataType::new_list(DataType::Int64, true), true),
        ]);
        assert_eq!(&ArrowSchema::from(dataset.schema()), &expected_schema);

        // We currently lose the index when casting a column
        let indices = dataset.load_indices().await?;
        assert_eq!(indices.len(), 0);

        // Each fragment gains a file with the new columns, but then the original file is dropped
        dataset.fragments().iter().for_each(|f| {
            assert_eq!(f.files.len(), 4);
        });

        let expected_data = RecordBatch::try_new(
            Arc::new(expected_schema),
            vec![
                Arc::new(Int64Array::from_iter_values(0..nrows as i64)),
                Arc::new(Float16Array::from_iter_values(
                    (0..nrows).map(|i| f16::from_f32(i as f32)),
                )),
                lance_arrow::cast::cast_with_options(
                    batch["vec"].as_ref(),
                    &DataType::FixedSizeList(
                        Arc::new(ArrowField::new("item", DataType::Float16, true)),
                        128,
                    ),
                    &Default::default(),
                )?,
                Arc::new(ListArray::from_iter_primitive::<Int64Type, _, _>(
                    (0..nrows as i64).map(|i| Some(vec![Some(i), Some(i + 1)])),
                )),
            ],
        )?;
        let actual_data = dataset.scan().try_into_batch().await?;
        assert_eq!(actual_data, expected_data);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn test_drop_columns(
        #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
        data_storage_version: LanceFileVersion,
    ) -> Result<()> {
        use std::collections::HashMap;

        use arrow_array::{ArrayRef, Float32Array, StructArray};

        let metadata: HashMap<String, String> = [("k1".into(), "v1".into())].into();

        let schema = Arc::new(ArrowSchema::new_with_metadata(
            vec![
                ArrowField::new("i", DataType::Int32, false),
                ArrowField::new(
                    "s",
                    DataType::Struct(ArrowFields::from(vec![
                        ArrowField::new("d", DataType::Int32, true),
                        ArrowField::new("l", DataType::Int32, true),
                    ])),
                    true,
                ),
                ArrowField::new("x", DataType::Float32, false),
            ],
            metadata.clone(),
        ));

        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![
                Arc::new(Int32Array::from(vec![1, 2])),
                Arc::new(StructArray::from(vec![
                    (
                        Arc::new(ArrowField::new("d", DataType::Int32, true)),
                        Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef,
                    ),
                    (
                        Arc::new(ArrowField::new("l", DataType::Int32, true)),
                        Arc::new(Int32Array::from(vec![1, 2])),
                    ),
                ])),
                Arc::new(Float32Array::from(vec![1.0, 2.0])),
            ],
        )?;

        let test_dir = tempfile::tempdir()?;
        let test_uri = test_dir.path().to_str().unwrap();

        let batches = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());
        let mut dataset = Dataset::write(
            batches,
            test_uri,
            Some(WriteParams {
                data_storage_version: Some(data_storage_version),
                ..Default::default()
            }),
        )
        .await?;

        let lance_schema = dataset.schema().clone();
        let original_fragments = dataset.fragments().to_vec();

        dataset.drop_columns(&["x"]).await?;
        dataset.validate().await?;

        let expected_schema = lance_schema.project(&["i", "s"])?;
        assert_eq!(dataset.schema(), &expected_schema);

        assert_eq!(dataset.version().version, 2);
        assert_eq!(dataset.fragments().as_ref(), &original_fragments);

        dataset.drop_columns(&["s.d"]).await?;
        dataset.validate().await?;

        let expected_schema = expected_schema.project(&["i", "s.l"])?;
        assert_eq!(dataset.schema(), &expected_schema);

        let expected_data = RecordBatch::try_new(
            Arc::new(ArrowSchema::from(&expected_schema)),
            vec![
                Arc::new(Int32Array::from(vec![1, 2])),
                Arc::new(StructArray::from(vec![(
                    Arc::new(ArrowField::new("l", DataType::Int32, true)),
                    Arc::new(Int32Array::from(vec![1, 2])) as ArrayRef,
                )])),
            ],
        )?;
        let actual_data = dataset.scan().try_into_batch().await?;
        assert_eq!(actual_data, expected_data);

        assert_eq!(dataset.version().version, 3);
        assert_eq!(dataset.fragments().as_ref(), &original_fragments);

        Ok(())
    }

    #[rstest]
    #[tokio::test]
    async fn test_drop_add_columns(
        #[values(LanceFileVersion::Legacy, LanceFileVersion::Stable)]
        data_storage_version: LanceFileVersion,
    ) -> Result<()> {
        let schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "i",
            DataType::Int32,
            false,
        )]));
        let batch =
            RecordBatch::try_new(schema.clone(), vec![Arc::new(Int32Array::from(vec![1, 2]))])?;

        let test_dir = tempfile::tempdir()?;
        let test_uri = test_dir.path().to_str().unwrap();

        let batches = RecordBatchIterator::new(vec![Ok(batch)], schema.clone());
        let mut dataset = Dataset::write(
            batches,
            test_uri,
            Some(WriteParams {
                data_storage_version: Some(data_storage_version),
                ..Default::default()
            }),
        )
        .await?;
        assert_eq!(dataset.manifest.max_field_id(), 0);

        // Test we can add 1 column, drop it, then add another column. Validate
        // the field ids are as expected.
        dataset
            .add_columns(
                NewColumnTransform::SqlExpressions(vec![("x".into(), "i + 1".into())]),
                Some(vec!["i".into()]),
                None,
            )
            .await?;
        assert_eq!(dataset.manifest.max_field_id(), 1);

        dataset.drop_columns(&["x"]).await?;
        assert_eq!(dataset.manifest.max_field_id(), 0);

        dataset
            .add_columns(
                NewColumnTransform::SqlExpressions(vec![("y".into(), "2 * i".into())]),
                Some(vec!["i".into()]),
                None,
            )
            .await?;
        assert_eq!(dataset.manifest.max_field_id(), 1);

        let data = dataset.scan().try_into_batch().await?;
        let expected_data = RecordBatch::try_new(
            Arc::new(schema.try_with_column(ArrowField::new("y", DataType::Int32, false))?),
            vec![
                Arc::new(Int32Array::from(vec![1, 2])),
                Arc::new(Int32Array::from(vec![2, 4])),
            ],
        )?;
        assert_eq!(data, expected_data);
        dataset.drop_columns(&["y"]).await?;
        assert_eq!(dataset.manifest.max_field_id(), 0);

        // Test we can add 2 columns, drop 1, then add another column. Validate
        // the field ids are as expected.
        dataset
            .add_columns(
                NewColumnTransform::SqlExpressions(vec![
                    ("a".into(), "i + 3".into()),
                    ("b".into(), "i + 7".into()),
                ]),
                Some(vec!["i".into()]),
                None,
            )
            .await?;
        assert_eq!(dataset.manifest.max_field_id(), 2);

        dataset.drop_columns(&["b"]).await?;
        // Even though we dropped a column, we still have the fragment with a and
        // b. So it should still act as if that field id is still in play.
        assert_eq!(dataset.manifest.max_field_id(), 2);

        dataset
            .add_columns(
                NewColumnTransform::SqlExpressions(vec![("c".into(), "i + 11".into())]),
                Some(vec!["i".into()]),
                None,
            )
            .await?;
        assert_eq!(dataset.manifest.max_field_id(), 3);

        let data = dataset.scan().try_into_batch().await?;
        let expected_schema = Arc::new(ArrowSchema::new(vec![
            ArrowField::new("i", DataType::Int32, false),
            ArrowField::new("a", DataType::Int32, false),
            ArrowField::new("c", DataType::Int32, false),
        ]));
        let expected_data = RecordBatch::try_new(
            expected_schema,
            vec![
                Arc::new(Int32Array::from(vec![1, 2])),
                Arc::new(Int32Array::from(vec![4, 5])),
                Arc::new(Int32Array::from(vec![12, 13])),
            ],
        )?;
        assert_eq!(data, expected_data);

        Ok(())
    }
}