Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: Add existing parquet files #960

Merged
merged 27 commits into from
Mar 3, 2025
Merged
Show file tree
Hide file tree
Changes from 12 commits
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
6853bab
feat: Add existing parquet files
jonathanc-n Feb 9, 2025
c574c5f
Merge branch 'main' into add-existing-files
jonathanc-n Feb 9, 2025
3f258ad
clippy fix
jonathanc-n Feb 9, 2025
39ce23d
Merge branch 'add-existing-files' of https://github.com/jonathanc-n/i…
jonathanc-n Feb 9, 2025
dd4abb7
Merge branch 'main' into add-existing-files
jonathanc-n Feb 10, 2025
65abfc3
Merge branch 'main' into add-existing-files
jonathanc-n Feb 12, 2025
4cb5b1e
change data file builder
jonathanc-n Feb 12, 2025
909d098
Merge branch 'add-existing-files' of https://github.com/jonathanc-n/i…
jonathanc-n Feb 12, 2025
e1dd355
fmt fix
jonathanc-n Feb 12, 2025
8756a71
clippy fix
jonathanc-n Feb 12, 2025
afbc642
Merge branch 'main' into add-existing-files
jonathanc-n Feb 12, 2025
a9c6b94
switch to unpartitioned
jonathanc-n Feb 13, 2025
e01ead8
code organization fixes
jonathanc-n Feb 18, 2025
96aabfe
Merge branch 'main' into add-existing-files
jonathanc-n Feb 18, 2025
b38496e
Update crates/iceberg/src/transaction.rs
jonathanc-n Feb 18, 2025
267124d
Merge branch 'main' into add-existing-files
jonathanc-n Feb 18, 2025
33cac26
Merge branch 'main' into add-existing-files
jonathanc-n Feb 21, 2025
86ea8eb
fixes
jonathanc-n Feb 21, 2025
c0cfb56
clippy
jonathanc-n Feb 21, 2025
175d4ce
Merge branch 'main' into add-existing-files
jonathanc-n Feb 24, 2025
0c7caaa
Merge branch 'main' into add-existing-files
jonathanc-n Feb 25, 2025
6f0bc0b
fixes
jonathanc-n Feb 28, 2025
d6cf198
Update crates/iceberg/src/writer/file_writer/parquet_writer.rs
jonathanc-n Mar 3, 2025
2665ebc
Update crates/iceberg/src/writer/file_writer/parquet_writer.rs
jonathanc-n Mar 3, 2025
d90871a
Update crates/iceberg/src/transaction.rs
jonathanc-n Mar 3, 2025
0b5d78c
Merge branch 'main' into add-existing-files
jonathanc-n Mar 3, 2025
43a6c85
clippy fix
jonathanc-n Mar 3, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions crates/iceberg/src/arrow/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1129,14 +1129,14 @@ impl<'a> BoundPredicateVisitor for PredicateConverter<'a> {
/// - `metadata_size_hint`: Provide a hint as to the size of the parquet file's footer.
/// - `preload_column_index`: Load the Column Index as part of [`Self::get_metadata`].
/// - `preload_offset_index`: Load the Offset Index as part of [`Self::get_metadata`].
struct ArrowFileReader<R: FileRead> {
pub struct ArrowFileReader<R: FileRead> {
meta: FileMetadata,
r: R,
}

impl<R: FileRead> ArrowFileReader<R> {
/// Create a new ArrowFileReader
fn new(meta: FileMetadata, r: R) -> Self {
pub fn new(meta: FileMetadata, r: R) -> Self {
Self { meta, r }
}
}
Expand Down
263 changes: 260 additions & 3 deletions crates/iceberg/src/scan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1139,14 +1139,14 @@ pub mod tests {
use crate::scan::FileScanTask;
use crate::spec::{
DataContentType, DataFileBuilder, DataFileFormat, Datum, Literal, ManifestEntry,
ManifestListWriter, ManifestStatus, ManifestWriterBuilder, NestedField, PrimitiveType,
Schema, Struct, TableMetadata, Type,
ManifestListWriter, ManifestStatus, ManifestWriterBuilder, NestedField, PartitionSpec,
PrimitiveType, Schema, Struct, StructType, TableMetadata, Type,
};
use crate::table::Table;
use crate::TableIdent;

pub struct TableTestFixture {
table_location: String,
pub table_location: String,
pub table: Table,
}

Expand Down Expand Up @@ -1194,6 +1194,55 @@ pub mod tests {
}
}

pub fn new_unpartitioned() -> Self {
let tmp_dir = TempDir::new().unwrap();
let table_location = tmp_dir.path().join("table1");
let manifest_list1_location = table_location.join("metadata/manifests_list_1.avro");
let manifest_list2_location = table_location.join("metadata/manifests_list_2.avro");
let table_metadata1_location = table_location.join("metadata/v1.json");

let file_io = FileIO::from_path(table_location.to_str().unwrap())
.unwrap()
.build()
.unwrap();

let mut table_metadata = {
let template_json_str = fs::read_to_string(format!(
"{}/testdata/example_table_metadata_v2.json",
env!("CARGO_MANIFEST_DIR")
))
.unwrap();
let mut context = Context::new();
context.insert("table_location", &table_location);
context.insert("manifest_list_1_location", &manifest_list1_location);
context.insert("manifest_list_2_location", &manifest_list2_location);
context.insert("table_metadata_1_location", &table_metadata1_location);

let metadata_json = Tera::one_off(&template_json_str, &context, false).unwrap();
serde_json::from_str::<TableMetadata>(&metadata_json).unwrap()
};

table_metadata.default_spec = Arc::new(PartitionSpec::unpartition_spec());
table_metadata.partition_specs.clear();
table_metadata.default_partition_type = StructType::new(vec![]);
table_metadata
.partition_specs
.insert(0, table_metadata.default_spec.clone());

let table = Table::builder()
.metadata(table_metadata)
.identifier(TableIdent::from_strs(["db", "table1"]).unwrap())
.file_io(file_io.clone())
.metadata_location(table_metadata1_location.to_str().unwrap())
.build()
.unwrap();

Self {
table_location: table_location.to_str().unwrap().to_string(),
table,
}
}

fn next_manifest_file(&self) -> OutputFile {
self.table
.file_io()
Expand Down Expand Up @@ -1413,6 +1462,214 @@ pub mod tests {
writer.close().unwrap();
}
}

pub async fn setup_unpartitioned_manifest_files(&mut self) {
let current_snapshot = self.table.metadata().current_snapshot().unwrap();
let parent_snapshot = current_snapshot
.parent_snapshot(self.table.metadata())
.unwrap();
let current_schema = current_snapshot.schema(self.table.metadata()).unwrap();
let current_partition_spec = Arc::new(PartitionSpec::unpartition_spec());

// Write data files using an empty partition for unpartitioned tables.
let mut writer = ManifestWriterBuilder::new(
self.next_manifest_file(),
Some(current_snapshot.snapshot_id()),
vec![],
current_schema.clone(),
current_partition_spec.as_ref().clone(),
)
.build_v2_data();

// Create an empty partition value.
let empty_partition = Struct::empty();

writer
.add_entry(
ManifestEntry::builder()
.status(ManifestStatus::Added)
.data_file(
DataFileBuilder::default()
.content(DataContentType::Data)
.file_path(format!("{}/1.parquet", &self.table_location))
.file_format(DataFileFormat::Parquet)
.file_size_in_bytes(100)
.record_count(1)
.partition(empty_partition.clone())
.key_metadata(None)
.build()
.unwrap(),
)
.build(),
)
.unwrap();

writer
.add_delete_entry(
ManifestEntry::builder()
.status(ManifestStatus::Deleted)
.snapshot_id(parent_snapshot.snapshot_id())
.sequence_number(parent_snapshot.sequence_number())
.file_sequence_number(parent_snapshot.sequence_number())
.data_file(
DataFileBuilder::default()
.content(DataContentType::Data)
.file_path(format!("{}/2.parquet", &self.table_location))
.file_format(DataFileFormat::Parquet)
.file_size_in_bytes(100)
.record_count(1)
.partition(empty_partition.clone())
.build()
.unwrap(),
)
.build(),
)
.unwrap();

writer
.add_existing_entry(
ManifestEntry::builder()
.status(ManifestStatus::Existing)
.snapshot_id(parent_snapshot.snapshot_id())
.sequence_number(parent_snapshot.sequence_number())
.file_sequence_number(parent_snapshot.sequence_number())
.data_file(
DataFileBuilder::default()
.content(DataContentType::Data)
.file_path(format!("{}/3.parquet", &self.table_location))
.file_format(DataFileFormat::Parquet)
.file_size_in_bytes(100)
.record_count(1)
.partition(empty_partition.clone())
.build()
.unwrap(),
)
.build(),
)
.unwrap();

let data_file_manifest = writer.write_manifest_file().await.unwrap();

// Write to manifest list
let mut manifest_list_write = ManifestListWriter::v2(
self.table
.file_io()
.new_output(current_snapshot.manifest_list())
.unwrap(),
current_snapshot.snapshot_id(),
current_snapshot.parent_snapshot_id(),
current_snapshot.sequence_number(),
);
manifest_list_write
.add_manifests(vec![data_file_manifest].into_iter())
.unwrap();
manifest_list_write.close().await.unwrap();

// prepare data for parquet files
let schema = {
let fields = vec![
arrow_schema::Field::new("x", arrow_schema::DataType::Int64, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"1".to_string(),
)])),
arrow_schema::Field::new("y", arrow_schema::DataType::Int64, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"2".to_string(),
)])),
arrow_schema::Field::new("z", arrow_schema::DataType::Int64, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"3".to_string(),
)])),
arrow_schema::Field::new("a", arrow_schema::DataType::Utf8, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"4".to_string(),
)])),
arrow_schema::Field::new("dbl", arrow_schema::DataType::Float64, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"5".to_string(),
)])),
arrow_schema::Field::new("i32", arrow_schema::DataType::Int32, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"6".to_string(),
)])),
arrow_schema::Field::new("i64", arrow_schema::DataType::Int64, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"7".to_string(),
)])),
arrow_schema::Field::new("bool", arrow_schema::DataType::Boolean, false)
.with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"8".to_string(),
)])),
];
Arc::new(arrow_schema::Schema::new(fields))
};

// Build the arrays for the RecordBatch
let col1 = Arc::new(Int64Array::from_iter_values(vec![1; 1024])) as ArrayRef;

let mut values = vec![2; 512];
values.append(vec![3; 200].as_mut());
values.append(vec![4; 300].as_mut());
values.append(vec![5; 12].as_mut());
let col2 = Arc::new(Int64Array::from_iter_values(values)) as ArrayRef;

let mut values = vec![3; 512];
values.append(vec![4; 512].as_mut());
let col3 = Arc::new(Int64Array::from_iter_values(values)) as ArrayRef;

let mut values = vec!["Apache"; 512];
values.append(vec!["Iceberg"; 512].as_mut());
let col4 = Arc::new(StringArray::from_iter_values(values)) as ArrayRef;

let mut values = vec![100.0f64; 512];
values.append(vec![150.0f64; 12].as_mut());
values.append(vec![200.0f64; 500].as_mut());
let col5 = Arc::new(Float64Array::from_iter_values(values)) as ArrayRef;

let mut values = vec![100i32; 512];
values.append(vec![150i32; 12].as_mut());
values.append(vec![200i32; 500].as_mut());
let col6 = Arc::new(Int32Array::from_iter_values(values)) as ArrayRef;

let mut values = vec![100i64; 512];
values.append(vec![150i64; 12].as_mut());
values.append(vec![200i64; 500].as_mut());
let col7 = Arc::new(Int64Array::from_iter_values(values)) as ArrayRef;

let mut values = vec![false; 512];
values.append(vec![true; 512].as_mut());
let values: BooleanArray = values.into();
let col8 = Arc::new(values) as ArrayRef;

let to_write = RecordBatch::try_new(schema.clone(), vec![
col1, col2, col3, col4, col5, col6, col7, col8,
])
.unwrap();

// Write the Parquet files
let props = WriterProperties::builder()
.set_compression(Compression::SNAPPY)
.build();

for n in 1..=3 {
let file = File::create(format!("{}/{}.parquet", &self.table_location, n)).unwrap();
let mut writer =
ArrowWriter::try_new(file, to_write.schema(), Some(props.clone())).unwrap();

writer.write(&to_write).expect("Writing batch");

// writer must be closed to write footer
writer.close().unwrap();
}
}
}

#[test]
Expand Down
Loading
Loading