diff --git a/bindings/python/pyproject.toml b/bindings/python/pyproject.toml index 71880ef18a..36b5834228 100644 --- a/bindings/python/pyproject.toml +++ b/bindings/python/pyproject.toml @@ -49,7 +49,7 @@ include = [ ignore = ["F403", "F405"] [tool.hatch.envs.dev] -dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==45.*", "pyiceberg[sql-sqlite,pyarrow]>=0.9.1"] +dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==45.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1"] [tool.hatch.envs.dev.scripts] build = "maturin build --out dist --sdist" diff --git a/bindings/python/src/data_file.rs b/bindings/python/src/data_file.rs new file mode 100644 index 0000000000..3339b38407 --- /dev/null +++ b/bindings/python/src/data_file.rs @@ -0,0 +1,165 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashMap; + +use iceberg::spec::{DataFile, DataFileFormat, PrimitiveLiteral}; +use pyo3::IntoPyObjectExt; +use pyo3::exceptions::PyValueError; +use pyo3::prelude::*; +use pyo3::types::PyBytes; + +#[pyclass()] +pub struct PyPrimitiveLiteral { + inner: PrimitiveLiteral, +} + +#[pymethods] +impl PyPrimitiveLiteral { + pub fn value(&self, py: Python<'_>) -> PyResult> { + match &self.inner { + PrimitiveLiteral::Boolean(v) => v.into_py_any(py), + PrimitiveLiteral::Int(v) => v.into_py_any(py), + PrimitiveLiteral::Long(v) => v.into_py_any(py), + PrimitiveLiteral::Float(v) => v.0.into_py_any(py), // unwrap OrderedFloat + PrimitiveLiteral::Double(v) => v.0.into_py_any(py), + PrimitiveLiteral::String(v) => v.into_py_any(py), + PrimitiveLiteral::Binary(v) => PyBytes::new(py, v).into_py_any(py), + PrimitiveLiteral::Int128(v) => v.into_py_any(py), // Python handles big ints + PrimitiveLiteral::UInt128(v) => v.into_py_any(py), + PrimitiveLiteral::AboveMax => Err(PyValueError::new_err("AboveMax is not supported")), + PrimitiveLiteral::BelowMin => Err(PyValueError::new_err("BelowMin is not supported")), + } + } +} + +#[pyclass] +pub struct PyDataFile { + inner: DataFile, +} + +#[pymethods] +impl PyDataFile { + #[getter] + fn content(&self) -> i32 { + self.inner.content_type() as i32 + } + + #[getter] + fn file_path(&self) -> &str { + self.inner.file_path() + } + + #[getter] + fn file_format(&self) -> &str { + match self.inner.file_format() { + DataFileFormat::Avro => "avro", + DataFileFormat::Orc => "orc", + DataFileFormat::Parquet => "parquet", + DataFileFormat::Puffin => "puffin", + } + } + + #[getter] + fn partition(&self) -> Vec> { + self.inner + .partition() + .iter() + .map(|lit| { + lit.and_then(|l| { + Some(PyPrimitiveLiteral { + inner: l.as_primitive_literal()?, + }) + }) + }) + .collect() + } + + #[getter] + fn record_count(&self) -> u64 { + self.inner.record_count() + } + + #[getter] + fn file_size_in_bytes(&self) -> u64 { + self.inner.file_size_in_bytes() + } + + #[getter] + fn column_sizes(&self) -> &HashMap { + self.inner.column_sizes() + } + + #[getter] + fn value_counts(&self) -> &HashMap { + self.inner.value_counts() + } + + #[getter] + fn null_value_counts(&self) -> &HashMap { + self.inner.null_value_counts() + } + + #[getter] + fn nan_value_counts(&self) -> &HashMap { + self.inner.nan_value_counts() + } + + #[getter] + fn upper_bounds(&self) -> HashMap> { + self.inner + .upper_bounds() + .iter() + .map(|(k, v)| (*k, v.to_bytes().unwrap().to_vec())) + .collect() + } + + #[getter] + fn lower_bounds(&self) -> HashMap> { + self.inner + .lower_bounds() + .iter() + .map(|(k, v)| (*k, v.to_bytes().unwrap().to_vec())) + .collect() + } + + #[getter] + fn key_metadata(&self) -> Option<&[u8]> { + self.inner.key_metadata() + } + + #[getter] + fn split_offsets(&self) -> &[i64] { + self.inner.split_offsets() + } + + #[getter] + fn equality_ids(&self) -> &[i32] { + self.inner.equality_ids() + } + + #[getter] + fn sort_order_id(&self) -> Option { + self.inner.sort_order_id() + } +} + +impl PyDataFile { + pub fn new(inner: DataFile) -> Self { + Self { inner } + } +} diff --git a/bindings/python/src/lib.rs b/bindings/python/src/lib.rs index 283dc1b143..131d864bdd 100644 --- a/bindings/python/src/lib.rs +++ b/bindings/python/src/lib.rs @@ -17,8 +17,10 @@ use pyo3::prelude::*; +mod data_file; mod datafusion_table_provider; mod error; +mod manifest; mod runtime; mod transform; @@ -26,5 +28,6 @@ mod transform; fn pyiceberg_core_rust(py: Python<'_>, m: &Bound<'_, PyModule>) -> PyResult<()> { datafusion_table_provider::register_module(py, m)?; transform::register_module(py, m)?; + manifest::register_module(py, m)?; Ok(()) } diff --git a/bindings/python/src/manifest.rs b/bindings/python/src/manifest.rs new file mode 100644 index 0000000000..ee756949c7 --- /dev/null +++ b/bindings/python/src/manifest.rs @@ -0,0 +1,240 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use iceberg::spec::{ + FieldSummary, FormatVersion, Manifest, ManifestEntry, ManifestFile, ManifestList, +}; +use pyo3::prelude::*; + +use crate::data_file::PyDataFile; + +#[pyclass] +pub struct PyManifest { + inner: Manifest, +} + +#[pymethods] +impl PyManifest { + fn entries(&self) -> Vec { + // TODO: Most of the time, we're only interested in 'alive' entries, + // that are the ones that are either ADDED or EXISTING + // so we can add a boolean to skip the DELETED entries right away before + // moving it into the Python world + self.inner + .entries() + .iter() + .map(|entry| PyManifestEntry { + inner: entry.clone(), + }) + .collect() + } +} + +#[pyclass] +pub struct PyFieldSummary { + inner: FieldSummary, +} + +#[pymethods] +impl crate::manifest::PyFieldSummary { + #[getter] + fn contains_null(&self) -> bool { + self.inner.contains_null + } + + #[getter] + fn contains_nan(&self) -> Option { + self.inner.contains_nan + } + + #[getter] + fn lower_bound(&self) -> Option> { + self.inner.lower_bound.clone().map(|b| b.to_vec()) + } + + #[getter] + fn upper_bound(&self) -> Option> { + self.inner.upper_bound.clone().map(|b| b.to_vec()) + } +} + +#[pyclass] +pub struct PyManifestFile { + inner: ManifestFile, +} + +#[pymethods] +impl crate::manifest::PyManifestFile { + #[getter] + fn manifest_path(&self) -> &str { + self.inner.manifest_path.as_str() + } + #[getter] + fn manifest_length(&self) -> i64 { + self.inner.manifest_length + } + #[getter] + fn partition_spec_id(&self) -> i32 { + self.inner.partition_spec_id + } + + #[getter] + fn content(&self) -> i32 { + self.inner.content as i32 + } + + #[getter] + fn sequence_number(&self) -> i64 { + self.inner.sequence_number + } + + #[getter] + fn min_sequence_number(&self) -> i64 { + self.inner.min_sequence_number + } + + #[getter] + fn added_snapshot_id(&self) -> i64 { + self.inner.added_snapshot_id + } + + #[getter] + fn added_files_count(&self) -> Option { + self.inner.added_files_count + } + + #[getter] + fn existing_files_count(&self) -> Option { + self.inner.existing_files_count + } + + #[getter] + fn deleted_files_count(&self) -> Option { + self.inner.deleted_files_count + } + + #[getter] + fn added_rows_count(&self) -> Option { + self.inner.added_rows_count + } + + #[getter] + fn existing_rows_count(&self) -> Option { + self.inner.existing_rows_count + } + + #[getter] + fn deleted_rows_count(&self) -> Option { + self.inner.deleted_rows_count + } + + #[getter] + fn partitions(&self) -> Vec { + self.inner + .partitions + .clone() + .unwrap() + .iter() + .map(|s| PyFieldSummary { inner: s.clone() }) + .collect() + } + + #[getter] + fn key_metadata(&self) -> Option> { + self.inner.key_metadata.clone() + } +} + +#[pyclass] +pub struct PyManifestEntry { + inner: Arc, +} + +#[pymethods] +impl PyManifestEntry { + #[getter] + fn status(&self) -> i32 { + self.inner.status as i32 + } + + #[getter] + fn snapshot_id(&self) -> Option { + self.inner.snapshot_id + } + + #[getter] + fn sequence_number(&self) -> Option { + self.inner.sequence_number + } + + #[getter] + fn file_sequence_number(&self) -> Option { + self.inner.file_sequence_number + } + + #[getter] + fn data_file(&self) -> PyDataFile { + PyDataFile::new(self.inner.data_file.clone()) + } +} + +#[pyfunction] +pub fn read_manifest_entries(bs: &[u8]) -> PyManifest { + // TODO: Some error handling + PyManifest { + inner: Manifest::parse_avro(bs).unwrap(), + } +} + +#[pyclass] +pub struct PyManifestList { + inner: ManifestList, +} + +#[pymethods] +impl crate::manifest::PyManifestList { + fn entries(&self) -> Vec { + self.inner + .entries() + .iter() + .map(|file| PyManifestFile { + inner: file.clone(), + }) + .collect() + } +} + +#[pyfunction] +pub fn read_manifest_list(bs: &[u8]) -> PyManifestList { + PyManifestList { + inner: ManifestList::parse_with_version(bs, FormatVersion::V2).unwrap(), + } +} + +pub fn register_module(py: Python<'_>, m: &Bound<'_, PyModule>) -> PyResult<()> { + let this = PyModule::new(py, "manifest")?; + + this.add_function(wrap_pyfunction!(read_manifest_entries, &this)?)?; + this.add_function(wrap_pyfunction!(read_manifest_list, &this)?)?; + + m.add_submodule(&this)?; + py.import("sys")? + .getattr("modules")? + .set_item("pyiceberg_core.manifest", this) +} diff --git a/bindings/python/tests/conftest.py b/bindings/python/tests/conftest.py new file mode 100644 index 0000000000..75ee3f36a3 --- /dev/null +++ b/bindings/python/tests/conftest.py @@ -0,0 +1,434 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + + +from tempfile import TemporaryDirectory +from typing import ( + Any, + Dict, + Generator, +) + +import pytest +from pydantic_core import to_json + +from pyiceberg.partitioning import PartitionField, PartitionSpec +from pyiceberg.schema import Schema +from pyiceberg.transforms import IdentityTransform, DayTransform +from pyiceberg.types import ( + IntegerType, + NestedField, + TimestampType, +) + + +@pytest.fixture(scope="session") +def avro_schema_manifest_entry() -> Dict[str, Any]: + return { + "type": "record", + "name": "manifest_entry", + "fields": [ + {"name": "status", "type": "int", "field-id": 0}, + { + "name": "snapshot_id", + "type": ["null", "long"], + "default": None, + "field-id": 1, + }, + { + "name": "data_file", + "type": { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "file_path", + "type": "string", + "doc": "Location URI with FS scheme", + "field-id": 100, + }, + { + "name": "file_format", + "type": "string", + "doc": "File format name: avro, orc, or parquet", + "field-id": 101, + }, + { + "name": "partition", + "type": { + "type": "record", + "name": "r102", + "fields": [ + { + "field-id": 1000, + "default": None, + "name": "VendorID", + "type": ["null", "int"], + }, + { + "field-id": 1001, + "default": None, + "name": "tpep_pickup_datetime", + "type": [ + "null", + {"type": "int", "logicalType": "date"}, + ], + }, + ], + }, + "field-id": 102, + }, + { + "name": "record_count", + "type": "long", + "doc": "Number of records in the file", + "field-id": 103, + }, + { + "name": "file_size_in_bytes", + "type": "long", + "doc": "Total file size in bytes", + "field-id": 104, + }, + { + "name": "block_size_in_bytes", + "type": "long", + "field-id": 105, + }, + { + "name": "column_sizes", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k117_v118", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 117, + }, + { + "name": "value", + "type": "long", + "field-id": 118, + }, + ], + }, + "logicalType": "map", + }, + ], + "doc": "Map of column id to total size on disk", + "default": None, + "field-id": 108, + }, + { + "name": "value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k119_v120", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 119, + }, + { + "name": "value", + "type": "long", + "field-id": 120, + }, + ], + }, + "logicalType": "map", + }, + ], + "doc": "Map of column id to total count, including null and NaN", + "default": None, + "field-id": 109, + }, + { + "name": "null_value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k121_v122", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 121, + }, + { + "name": "value", + "type": "long", + "field-id": 122, + }, + ], + }, + "logicalType": "map", + }, + ], + "doc": "Map of column id to null value count", + "default": None, + "field-id": 110, + }, + { + "name": "nan_value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k138_v139", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 138, + }, + { + "name": "value", + "type": "long", + "field-id": 139, + }, + ], + }, + "logicalType": "map", + }, + ], + "doc": "Map of column id to number of NaN values in the column", + "default": None, + "field-id": 137, + }, + { + "name": "lower_bounds", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k126_v127", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 126, + }, + { + "name": "value", + "type": "bytes", + "field-id": 127, + }, + ], + }, + "logicalType": "map", + }, + ], + "doc": "Map of column id to lower bound", + "default": None, + "field-id": 125, + }, + { + "name": "upper_bounds", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k129_v130", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 129, + }, + { + "name": "value", + "type": "bytes", + "field-id": 130, + }, + ], + }, + "logicalType": "map", + }, + ], + "doc": "Map of column id to upper bound", + "default": None, + "field-id": 128, + }, + { + "name": "key_metadata", + "type": ["null", "bytes"], + "doc": "Encryption key metadata blob", + "default": None, + "field-id": 131, + }, + { + "name": "split_offsets", + "type": [ + "null", + {"type": "array", "items": "long", "element-id": 133}, + ], + "doc": "Splittable offsets", + "default": None, + "field-id": 132, + }, + { + "name": "sort_order_id", + "type": ["null", "int"], + "doc": "Sort order ID", + "default": None, + "field-id": 140, + }, + ], + }, + "field-id": 2, + }, + ], + } + + +manifest_entry_records = [ + { + "status": 1, + "snapshot_id": 8744736658442914487, + "data_file": { + "file_path": "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet", + "file_format": "PARQUET", + "partition": {"VendorID": 1, "tpep_pickup_datetime": 1925}, + "record_count": 19513, + "file_size_in_bytes": 388872, + "block_size_in_bytes": 67108864, + "column_sizes": [ + {"key": 1, "value": 53}, + {"key": 2, "value": 98153}, + ], + "value_counts": [ + {"key": 1, "value": 19513}, + {"key": 2, "value": 19513}, + ], + "null_value_counts": [ + {"key": 1, "value": 19513}, + {"key": 2, "value": 0}, + ], + "nan_value_counts": [], + "lower_bounds": [ + {"key": 1, "value": b"\x00\x00\x00\x00"}, + {"key": 2, "value": b"\x00\x00\x00\x00\x00\x00\x00\x00"}, + ], + "upper_bounds": [ + {"key": 1, "value": b"\x00\x00\xe0\xbf"}, + {"key": 2, "value": b"\x00\x00\x00\x00\x00\x00\xe0\xbf"}, + ], + "key_metadata": None, + "split_offsets": [4], + "sort_order_id": 0, + }, + }, + { + "status": 1, + "snapshot_id": 8744736658442914487, + "data_file": { + "file_path": "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet", + "file_format": "PARQUET", + "partition": {"VendorID": 1, "tpep_pickup_datetime": None}, + "record_count": 95050, + "file_size_in_bytes": 1265950, + "block_size_in_bytes": 67108864, + "column_sizes": [ + {"key": 1, "value": 318}, + {"key": 2, "value": 329806}, + ], + "value_counts": [ + {"key": 1, "value": 95050}, + {"key": 2, "value": 95050}, + ], + "null_value_counts": [ + {"key": 1, "value": 0}, + {"key": 2, "value": 0}, + ], + "nan_value_counts": [], + "lower_bounds": [ + {"key": 1, "value": b"\x00\x00\x00\x00"}, + {"key": 2, "value": b"\x00\x00\x00\x00\x00\x00\x00\x00"}, + ], + "upper_bounds": [ + {"key": 1, "value": b"\x00\x00\xe0\xbf"}, + {"key": 2, "value": b"\x00\x00\x00\x00\x00\x00\xe0\xbf"}, + ], + "key_metadata": None, + "split_offsets": [4], + "sort_order_id": 0, + }, + }, +] + + +@pytest.fixture(scope="session") +def test_schema() -> Schema: + return Schema( + NestedField(1, "VendorID", IntegerType(), False), + NestedField(2, "tpep_pickup_datetime", TimestampType(), False), + ) + + +@pytest.fixture(scope="session") +def test_partition_spec() -> Schema: + return PartitionSpec( + PartitionField(1, 1000, IdentityTransform(), "VendorID"), + PartitionField(2, 1001, DayTransform(), "tpep_pickup_day"), + ) + + +@pytest.fixture(scope="session") +def generated_manifest_entry_file( + avro_schema_manifest_entry: Dict[str, Any], + test_schema: Schema, + test_partition_spec: PartitionSpec, +) -> Generator[str, None, None]: + from fastavro import parse_schema, writer + + parsed_schema = parse_schema(avro_schema_manifest_entry) + + with TemporaryDirectory() as tmpdir: + tmp_avro_file = tmpdir + "/manifest.avro" + with open(tmp_avro_file, "wb") as out: + writer( + out, + parsed_schema, + manifest_entry_records, + metadata={ + "schema": test_schema.model_dump_json(), + "partition-spec": to_json(test_partition_spec.fields).decode( + "utf-8" + ), + }, + ) + yield tmp_avro_file diff --git a/bindings/python/tests/test_manifest.py b/bindings/python/tests/test_manifest.py new file mode 100644 index 0000000000..0e838c3954 --- /dev/null +++ b/bindings/python/tests/test_manifest.py @@ -0,0 +1,142 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from typing import Any, List +import pytest + + +from pyiceberg.io import FileIO +from pyiceberg.io.pyarrow import PyArrowFileIO +from pyiceberg.manifest import ( + DataFile, + DataFileContent, + FileFormat, + ManifestEntry, + ManifestEntryStatus, + _manifests, +) + + +@pytest.fixture(autouse=True) +def clear_global_manifests_cache() -> None: + # Clear the global cache before each test + _manifests.cache_clear() # type: ignore + + +def fetch_manifest_entry( + manifest_entry_file: str, io: FileIO, discard_deleted: bool = True +) -> List[ManifestEntry]: + from pyiceberg_core import manifest + + bs = io.new_input(manifest_entry_file).open().read() + manifest = manifest.read_manifest_entries(bs) + + # TODO: Don't convert the types + # but this is the easiest for now until we + # have the write part in there as well + def _convert_entry(entry: Any) -> ManifestEntry: + data_file = DataFile( + DataFileContent(entry.data_file.content), + entry.data_file.file_path, + FileFormat(entry.data_file.file_format), + [p.value() if p is not None else None for p in entry.data_file.partition], + entry.data_file.record_count, + entry.data_file.file_size_in_bytes, + entry.data_file.column_sizes, + entry.data_file.value_counts, + entry.data_file.null_value_counts, + entry.data_file.nan_value_counts, + entry.data_file.lower_bounds, + entry.data_file.upper_bounds, + entry.data_file.key_metadata, + entry.data_file.split_offsets, + entry.data_file.equality_ids, + entry.data_file.sort_order_id, + ) + + return ManifestEntry( + ManifestEntryStatus(entry.status), + entry.snapshot_id, + entry.sequence_number, + entry.file_sequence_number, + data_file, + ) + + return [ + _convert_entry(entry) + # _inherit_from_manifest(, self) + for entry in manifest.entries() + if not discard_deleted or entry.status != ManifestEntryStatus.DELETED + ] + + +def test_read_manifest_entry(generated_manifest_entry_file: str) -> None: + # manifest = ManifestFile.from_args( + # manifest_path=generated_manifest_entry_file, + # manifest_length=0, + # partition_spec_id=0, + # added_snapshot_id=0, + # sequence_number=0, + # partitions=[], + # ) + # manifest_entries = manifest.fetch_manifest_entry(PyArrowFileIO()) + manifest_entries = fetch_manifest_entry( + f"file://{generated_manifest_entry_file}", PyArrowFileIO() + ) + manifest_entry = manifest_entries[0] + + assert manifest_entry.status == ManifestEntryStatus.ADDED + assert manifest_entry.snapshot_id == 8744736658442914487 + assert manifest_entry.sequence_number == 0 + assert isinstance(manifest_entry.data_file, DataFile) + + data_file = manifest_entry.data_file + + assert data_file.content == DataFileContent.DATA + assert ( + data_file.file_path + == "/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet" + ) + assert data_file.file_format == FileFormat.PARQUET + assert repr(data_file.partition) == "[1, None]" + assert data_file.record_count == 19513 + assert data_file.file_size_in_bytes == 388872 + assert data_file.column_sizes == { + 1: 53, + 2: 98153, + } + assert data_file.value_counts == { + 1: 19513, + 2: 19513, + } + assert data_file.null_value_counts == { + 1: 19513, + 2: 0, + } + assert data_file.nan_value_counts == {} + assert data_file.lower_bounds == { + 1: b"\x00\x00\x00\x00", + 2: b"\x00\x00\x00\x00\x00\x00\x00\x00", + } + assert data_file.upper_bounds == { + 1: b"\x00\x00\xe0\xbf", + 2: b"\x00\x00\x00\x00\x00\x00\xe0\xbf", + } + assert data_file.key_metadata is None + assert data_file.split_offsets == [4] + assert data_file.equality_ids == [] + assert data_file.sort_order_id == 0