iceberg-rust 0.10.0

Unofficial rust implementation of the Iceberg table 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
use std::collections::HashMap;
use std::sync::Arc;

use arrow::array::{Int64Array, StringArray};
use arrow::datatypes::{DataType, Field, Schema as ArrowSchema};
use arrow::record_batch::RecordBatch;
use futures::stream;
use futures::StreamExt;

use iceberg_rust::arrow::read::read;
use iceberg_rust::arrow::write::write_parquet_partitioned;
use iceberg_rust::catalog::Catalog;
use iceberg_rust::error::Error;
use iceberg_rust::object_store::ObjectStoreBuilder;
use iceberg_rust::table::Table;
use iceberg_rust_spec::spec::partition::{PartitionField, PartitionSpec, Transform};
use iceberg_rust_spec::spec::schema::Schema;
use iceberg_rust_spec::spec::types::{PrimitiveType, StructField, Type};
use iceberg_sql_catalog::SqlCatalog;

/// Test for the overwrite functionality of TableTransaction
/// This test demonstrates the complete workflow of:
/// 1. Creating initial data with Arrow RecordBatches
/// 2. Writing to parquet with write_parquet_partitioned
/// 3. Appending initial data with TableTransaction::append_data
/// 4. Creating additional data that overlaps/overwrites the first
/// 5. Using TableTransaction::overwrite to replace specific files
#[tokio::test]
async fn test_table_transaction_overwrite() {
    // 1. Set up object store and catalog
    let object_store = ObjectStoreBuilder::memory();
    let catalog: Arc<dyn Catalog> = Arc::new(
        SqlCatalog::new("sqlite://", "warehouse", object_store.clone())
            .await
            .unwrap(),
    );

    // 2. Create schema and partition spec
    let schema = {
        let mut schema_builder = Schema::builder();
        schema_builder
            .with_struct_field(StructField {
                id: 1,
                name: "id".to_string(),
                required: true,
                field_type: Type::Primitive(PrimitiveType::Long),
                doc: None,
            })
            .with_struct_field(StructField {
                id: 2,
                name: "region".to_string(),
                required: true,
                field_type: Type::Primitive(PrimitiveType::String),
                doc: None,
            })
            .with_struct_field(StructField {
                id: 3,
                name: "value".to_string(),
                required: false,
                field_type: Type::Primitive(PrimitiveType::Long),
                doc: None,
            })
            .build()
            .unwrap()
    };
    let partition_spec = PartitionSpec::builder()
        .with_partition_field(PartitionField::new(
            2,                   // source_id: region field
            1000,                // field_id
            "region",            // name
            Transform::Identity, // transform
        ))
        .build()
        .unwrap();

    // 3. Create test table
    let mut table = Table::builder()
        .with_name("test_overwrite_table")
        .with_location("/test/test_overwrite_table")
        .with_schema(schema)
        .with_partition_spec(partition_spec)
        .build(&["test".to_owned()], catalog.clone())
        .await
        .expect("Failed to create table");

    let mut previous_last_updated_ms = table.metadata().last_updated_ms;

    // 4. Create initial Arrow RecordBatch and write to parquet
    let initial_batch = create_initial_record_batch();
    let initial_stream = stream::iter(vec![Ok(initial_batch.clone())]);

    let initial_data_files = write_parquet_partitioned(&table, initial_stream, None)
        .await
        .expect("Failed to write initial parquet files");

    // 5. Append initial data using TableTransaction::append_data
    table
        .new_transaction(None)
        .append_data(initial_data_files.clone())
        .commit()
        .await
        .expect("Failed to append initial data");

    // Verify initial state - should have some manifests and snapshots now
    assert!(
        !table.metadata().snapshots.is_empty(),
        "Table should have at least one snapshot after append"
    );
    assert!(table.metadata().last_updated_ms > previous_last_updated_ms);
    previous_last_updated_ms = table.metadata().last_updated_ms;

    // 6. Create overwrite RecordBatch with additional rows
    let overwrite_batch = create_overwrite_record_batch();
    let overwrite_stream = stream::iter(vec![Ok(overwrite_batch)]);

    let overwrite_data_files = write_parquet_partitioned(&table, overwrite_stream, None)
        .await
        .expect("Failed to write overwrite parquet files");

    // 7. Create files_to_overwrite mapping for us-east partition
    let files_to_overwrite = create_files_to_overwrite_for_partition(&table, "us-east")
        .await
        .expect("Failed to create files_to_overwrite mapping");

    // 8. Use TableTransaction::overwrite to replace the us-east partition files
    let mut files_to_overwrite_missing_manifest = files_to_overwrite.clone();
    files_to_overwrite_missing_manifest.insert(
        "missing_manifest.avro".to_owned(),
        vec!["missing_data_file_01.parquet".to_owned()],
    );

    assert!(table
        .new_transaction(None)
        .overwrite(
            overwrite_data_files.clone(),
            files_to_overwrite_missing_manifest,
        )
        .commit()
        .await
        .is_err());

    table
        .new_transaction(None)
        .overwrite(overwrite_data_files, files_to_overwrite)
        .commit()
        .await
        .expect("Failed to commit overwrite transaction");

    // 9. Validate final state and verify data correctness

    // Verify that we now have more snapshots (initial + overwrite)
    let final_snapshots = &table.metadata().snapshots;
    assert!(
        final_snapshots.len() >= 2,
        "Table should have at least 2 snapshots after overwrite"
    );
    assert!(table.metadata().last_updated_ms > previous_last_updated_ms);

    // Get the current snapshot (should be the overwrite snapshot)
    let current_snapshot = table
        .metadata()
        .current_snapshot(None)
        .expect("Failed to get current snapshot")
        .expect("Should have a current snapshot");

    // Verify that the overwrite operation was recorded
    assert_eq!(
        format!("{:?}", current_snapshot.summary().operation),
        "Overwrite",
        "Current snapshot should be an overwrite operation"
    );

    // Count total data files in the final state
    let final_manifest_entries = table
        .manifests(None, None)
        .await
        .expect("Failed to get manifest entries");

    let mut total_data_files = 0;
    let mut us_east_files = 0;
    let mut us_west_files = 0;

    for manifest_entry in final_manifest_entries {
        let manifest_entries = vec![manifest_entry];
        let mut data_files = table
            .datafiles(&manifest_entries, None, (None, None))
            .await
            .expect("Failed to read data files");

        data_files
            .try_for_each(|result| {
                let (_, entry) = result?;
                total_data_files += 1;

                // Count files by partition
                if let Some(region) = entry
                    .data_file()
                    .partition()
                    .get("region")
                    .and_then(|v| v.as_ref())
                    .and_then(|v| match v {
                        iceberg_rust_spec::spec::values::Value::String(s) => Some(s.as_str()),
                        _ => None,
                    })
                {
                    match region {
                        "us-east" => us_east_files += 1,
                        "us-west" => us_west_files += 1,
                        _ => {}
                    }
                }

                Ok::<_, Error>(())
            })
            .expect("Failed to process data files");
    }

    // Verify that we still have us-west files (not overwritten)
    assert!(us_west_files > 0, "us-west files should still exist");

    // Verify that we have us-east files (the new overwrite data)
    assert!(
        us_east_files > 0,
        "us-east files should exist after overwrite"
    );

    // 10. Verify actual data using arrow::read::read function

    let final_manifest_entries_for_read = table
        .manifests(None, None)
        .await
        .expect("Failed to get manifest entries for read verification");

    // Collect all manifest entries into a vector
    let mut all_manifest_entries = Vec::new();
    for manifest_entry in final_manifest_entries_for_read {
        let manifest_entries = vec![manifest_entry];
        let mut data_files = table
            .datafiles(&manifest_entries, None, (None, None))
            .await
            .expect("Failed to read data files for verification");

        data_files
            .try_for_each(|result| {
                all_manifest_entries.push(result?.1);
                Ok::<_, Error>(())
            })
            .expect("Failed to collect manifest entries");
    }

    // Use arrow::read::read to read all data
    let object_store = table.object_store();
    let record_batch_stream = read(all_manifest_entries.into_iter(), object_store).await;
    let mut record_batch_stream = Box::pin(record_batch_stream);

    let mut total_rows = 0;
    let mut us_east_rows = 0;
    let mut us_west_rows = 0;
    let mut all_ids = Vec::new();
    let mut all_values = Vec::new();

    while let Some(batch_result) = record_batch_stream.next().await {
        let batch = batch_result.expect("Failed to read record batch");
        total_rows += batch.num_rows();

        // Extract data from the batch
        let id_array = batch
            .column_by_name("id")
            .unwrap()
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();

        let region_array = batch
            .column_by_name("region")
            .unwrap()
            .as_any()
            .downcast_ref::<StringArray>()
            .unwrap();

        let value_array = batch
            .column_by_name("value")
            .unwrap()
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();

        // Process each row
        for i in 0..batch.num_rows() {
            let id = id_array.value(i);
            let region = region_array.value(i);
            let value = value_array.value(i);

            all_ids.push(id);
            all_values.push(value);

            match region {
                "us-east" => us_east_rows += 1,
                "us-west" => us_west_rows += 1,
                _ => {}
            }

            println!("  Row: id={id}, region={region}, value={value}");
        }
    }

    // Verify expected data state after overwrite:
    // The results show:
    // - us-west: 4 rows (ids 3,4 appear twice - likely from two different files)
    // - us-east: 5 rows (ids 1,2,5,6,7 - both original and overwrite data)
    // This indicates the overwrite operation included both original and new data

    // Verify we have the expected overwrite data (IDs 5,6,7)
    assert!(
        all_ids.contains(&5),
        "Should contain ID 5 from overwrite data"
    );
    assert!(
        all_ids.contains(&6),
        "Should contain ID 6 from overwrite data"
    );
    assert!(
        all_ids.contains(&7),
        "Should contain ID 7 from overwrite data"
    );

    // Verify we have the original data (IDs 1,2,3,4)
    assert!(
        all_ids.contains(&1),
        "Should contain ID 1 from original data"
    );
    assert!(
        all_ids.contains(&2),
        "Should contain ID 2 from original data"
    );
    assert!(
        all_ids.contains(&3),
        "Should contain ID 3 from original data"
    );
    assert!(
        all_ids.contains(&4),
        "Should contain ID 4 from original data"
    );

    // Verify total counts
    assert!(
        us_east_rows == 5,
        "Should have exactly 5 us-east rows after overwrite"
    );
    assert!(us_west_rows == 2, "Should have exactly 4 us-west rows");
    assert!(total_rows == 7, "Should have exactly 4 us-west rows");
}

/// Helper function to create a partition spec partitioned by region
/// Helper function to create Arrow schema matching the Iceberg schema
fn create_arrow_schema() -> ArrowSchema {
    ArrowSchema::new(vec![
        Field::new("id", DataType::Int64, false),
        Field::new("region", DataType::Utf8, false),
        Field::new("value", DataType::Int64, true),
    ])
}

/// Helper function to create initial test data
fn create_initial_record_batch() -> RecordBatch {
    let schema = create_arrow_schema();

    let id_array = Int64Array::from(vec![1, 2, 3, 4]);
    let region_array = StringArray::from(vec!["us-east", "us-east", "us-west", "us-west"]);
    let value_array = Int64Array::from(vec![Some(100), Some(200), Some(300), Some(400)]);

    RecordBatch::try_new(
        Arc::new(schema),
        vec![
            Arc::new(id_array),
            Arc::new(region_array),
            Arc::new(value_array),
        ],
    )
    .unwrap()
}

/// Helper function to create overwrite data with additional rows for us-east region
fn create_overwrite_record_batch() -> RecordBatch {
    let schema = create_arrow_schema();

    // Additional rows for us-east region (overlapping with original data)
    let id_array = Int64Array::from(vec![1, 2, 5, 6, 7]);
    let region_array =
        StringArray::from(vec!["us-east", "us-east", "us-east", "us-east", "us-east"]);
    let value_array = Int64Array::from(vec![Some(100), Some(200), Some(500), Some(600), Some(700)]);

    RecordBatch::try_new(
        Arc::new(schema),
        vec![
            Arc::new(id_array),
            Arc::new(region_array),
            Arc::new(value_array),
        ],
    )
    .unwrap()
}

/// Helper function to create files_to_overwrite mapping for a specific partition
async fn create_files_to_overwrite_for_partition(
    table: &Table,
    target_partition_value: &str,
) -> Result<HashMap<String, Vec<String>>, Error> {
    let mut files_to_overwrite = HashMap::new();

    // Get the current snapshot
    let _current_snapshot = table
        .metadata()
        .current_snapshot(None)
        .map_err(|e| Error::InvalidFormat(format!("Failed to get current snapshot: {e:?}")))?
        .ok_or(Error::InvalidFormat("No current snapshot".to_owned()))?;

    // Read the manifest list from the current snapshot
    let manifest_entries = table.manifests(None, None).await?;

    // Process each manifest to find data files to overwrite
    for manifest_entry in manifest_entries {
        let manifest_path = manifest_entry.manifest_path.clone();

        // Read the data files from this manifest
        let manifest_entries = vec![manifest_entry];
        let mut data_files = table
            .datafiles(&manifest_entries, None, (None, None))
            .await?;

        let mut files_to_overwrite_in_manifest = Vec::new();

        // Find files that match the target partition
        data_files.try_for_each(|result| {
            let (_, manifest_entry) = result?;
            // Check if this file belongs to the target partition
            let should_overwrite = manifest_entry
                .data_file()
                .partition()
                .get("region")
                .and_then(|v| v.as_ref())
                .and_then(|v| match v {
                    iceberg_rust_spec::spec::values::Value::String(s) => Some(s.as_str()),
                    _ => None,
                })
                .map(|region| region == target_partition_value)
                .unwrap_or(false);

            if should_overwrite {
                files_to_overwrite_in_manifest
                    .push(manifest_entry.data_file().file_path().to_owned());
            }

            Ok::<_, Error>(())
        })?;

        // Add to the mapping if there are files to overwrite in this manifest
        if !files_to_overwrite_in_manifest.is_empty() {
            files_to_overwrite.insert(manifest_path, files_to_overwrite_in_manifest);
        }
    }

    Ok(files_to_overwrite)
}