You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by fo...@apache.org on 2022/09/16 03:21:28 UTC

[iceberg] branch master updated: Python: Add CLI command to list files (#5690)

This is an automated email from the ASF dual-hosted git repository.

fokko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 76aaa6a4b5 Python: Add CLI command to list files (#5690)
76aaa6a4b5 is described below

commit 76aaa6a4b5c6ff0f512784e55954650a604dbca0
Author: Fokko Driesprong <fo...@apache.org>
AuthorDate: Thu Sep 15 20:21:22 2022 -0700

    Python: Add CLI command to list files (#5690)
    
    This makes it easy to check the FileIO:
    
    ```
    > pyiceberg files nyc.taxis
    Snapshots: nyc.taxis
    └── Snapshot 5937117119577207079, schema 0: file:/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/nyc.db/taxis/metadata/snap-5937117119577207079-1-94656c4f-4c66-4600-a4ca-f30377300527.avro
        └── Manifest: file:/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/nyc.db/taxis/metadata/94656c4f-4c66-4600-a4ca-f30377300527-m0.avro
            └── Datafile: file:/Users/fokkodriesprong/Desktop/docker-spark-iceberg/wh/nyc.db/taxis/data/00003-4-a245d9ee-8462-4a08-8cbc-26b8b33b9377-00001.parquet
    ```
---
 python/pyiceberg/cli/console.py      |  15 ++
 python/pyiceberg/cli/output.py       |  82 ++++++----
 python/pyiceberg/manifest.py         |   6 +-
 python/pyiceberg/table/snapshots.py  |  15 +-
 python/tests/avro/test_reader.py     |   6 +-
 python/tests/conftest.py             |  16 +-
 python/tests/table/test_snapshots.py |  39 +++++
 python/tests/utils/test_manifest.py  | 293 ++++++++++++++++++++++++++++++++++-
 8 files changed, 437 insertions(+), 35 deletions(-)

diff --git a/python/pyiceberg/cli/console.py b/python/pyiceberg/cli/console.py
index 3a040159d8..e8234d7c7e 100644
--- a/python/pyiceberg/cli/console.py
+++ b/python/pyiceberg/cli/console.py
@@ -24,6 +24,7 @@ from click import Context
 from pyiceberg.catalog import Catalog, load_catalog
 from pyiceberg.cli.output import ConsoleOutput, JsonOutput, Output
 from pyiceberg.exceptions import NoSuchNamespaceError, NoSuchPropertyException, NoSuchTableError
+from pyiceberg.io import load_file_io
 
 
 def catch_exception():
@@ -136,6 +137,20 @@ def describe(ctx: Context, entity: Literal["name", "namespace", "table"], identi
         raise NoSuchTableError(f"Table or namespace does not exist: {identifier}")
 
 
+@run.command()
+@click.argument("identifier")
+@click.option("--history", is_flag=True)
+@click.pass_context
+@catch_exception()
+def files(ctx: Context, identifier: str, history: bool):
+    """Lists all the files of the table"""
+    catalog, output = _catalog_and_output(ctx)
+
+    catalog_table = catalog.load_table(identifier)
+    io = load_file_io({**catalog.properties, **catalog_table.metadata.properties})
+    output.files(catalog_table, io, history)
+
+
 @run.command()
 @click.argument("identifier")
 @click.pass_context
diff --git a/python/pyiceberg/cli/output.py b/python/pyiceberg/cli/output.py
index 270567930e..c7907c7e5a 100644
--- a/python/pyiceberg/cli/output.py
+++ b/python/pyiceberg/cli/output.py
@@ -23,6 +23,7 @@ from rich.console import Console
 from rich.table import Table as RichTable
 from rich.tree import Tree
 
+from pyiceberg.io import FileIO
 from pyiceberg.schema import Schema
 from pyiceberg.table import Table
 from pyiceberg.table.partitioning import PartitionSpec
@@ -33,35 +34,39 @@ class Output(ABC):
     """Output interface for exporting"""
 
     @abstractmethod
-    def exception(self, ex: Exception):
+    def exception(self, ex: Exception) -> None:
         ...
 
     @abstractmethod
-    def identifiers(self, identifiers: List[Identifier]):
+    def identifiers(self, identifiers: List[Identifier]) -> None:
         ...
 
     @abstractmethod
-    def describe_table(self, table):
+    def describe_table(self, table: Table) -> None:
         ...
 
     @abstractmethod
-    def describe_properties(self, properties: Properties):
+    def files(self, table: Table, io: FileIO, history: bool) -> None:
         ...
 
     @abstractmethod
-    def text(self, response: str):
+    def describe_properties(self, properties: Properties) -> None:
         ...
 
     @abstractmethod
-    def schema(self, schema: Schema):
+    def text(self, response: str) -> None:
         ...
 
     @abstractmethod
-    def spec(self, spec: PartitionSpec):
+    def schema(self, schema: Schema) -> None:
         ...
 
     @abstractmethod
-    def uuid(self, uuid: Optional[UUID]):
+    def spec(self, spec: PartitionSpec) -> None:
+        ...
+
+    @abstractmethod
+    def uuid(self, uuid: Optional[UUID]) -> None:
         ...
 
 
@@ -70,27 +75,27 @@ class ConsoleOutput(Output):
 
     verbose: bool
 
-    def __init__(self, **properties: Any):
+    def __init__(self, **properties: Any) -> None:
         self.verbose = properties.get("verbose", False)
 
     @property
     def _table(self) -> RichTable:
         return RichTable.grid(padding=(0, 2))
 
-    def exception(self, ex: Exception):
+    def exception(self, ex: Exception) -> None:
         if self.verbose:
             Console(stderr=True).print_exception()
         else:
             Console(stderr=True).print(ex)
 
-    def identifiers(self, identifiers: List[Identifier]):
+    def identifiers(self, identifiers: List[Identifier]) -> None:
         table = self._table
         for identifier in identifiers:
             table.add_row(".".join(identifier))
 
         Console().print(table)
 
-    def describe_table(self, table: Table):
+    def describe_table(self, table: Table) -> None:
         metadata = table.metadata
         table_properties = self._table
 
@@ -119,25 +124,47 @@ class ConsoleOutput(Output):
         output_table.add_row("Properties", table_properties)
         Console().print(output_table)
 
-    def describe_properties(self, properties: Properties):
+    def files(self, table: Table, io: FileIO, history: bool) -> None:
+        if history:
+            snapshots = table.metadata.snapshots
+        else:
+            if snapshot := table.current_snapshot():
+                snapshots = [snapshot]
+            else:
+                snapshots = []
+
+        snapshot_tree = Tree(f"Snapshots: {'.'.join(table.identifier)}")
+
+        for snapshot in snapshots:
+            manifest_list_str = f": {snapshot.manifest_list}" if snapshot.manifest_list else ""
+            list_tree = snapshot_tree.add(f"Snapshot {snapshot.snapshot_id}, schema {snapshot.schema_id}{manifest_list_str}")
+
+            manifest_list = snapshot.fetch_manifest_list(io)
+            for manifest in manifest_list:
+                manifest_tree = list_tree.add(f"Manifest: {manifest.manifest_path}")
+                for manifest_entry in manifest.fetch_manifest_entry(io):
+                    manifest_tree.add(f"Datafile: {manifest_entry.data_file.file_path}")
+        Console().print(snapshot_tree)
+
+    def describe_properties(self, properties: Properties) -> None:
         output_table = self._table
         for k, v in properties.items():
             output_table.add_row(k, v)
         Console().print(output_table)
 
-    def text(self, response: str):
+    def text(self, response: str) -> None:
         Console().print(response)
 
-    def schema(self, schema: Schema):
+    def schema(self, schema: Schema) -> None:
         output_table = self._table
         for field in schema.fields:
             output_table.add_row(field.name, str(field.field_type), field.doc or "")
         Console().print(output_table)
 
-    def spec(self, spec: PartitionSpec):
+    def spec(self, spec: PartitionSpec) -> None:
         Console().print(str(spec))
 
-    def uuid(self, uuid: Optional[UUID]):
+    def uuid(self, uuid: Optional[UUID]) -> None:
         Console().print(str(uuid) if uuid else "missing")
 
 
@@ -146,32 +173,35 @@ class JsonOutput(Output):
 
     verbose: bool
 
-    def __init__(self, **properties: Any):
+    def __init__(self, **properties: Any) -> None:
         self.verbose = properties.get("verbose", False)
 
     def _out(self, d: Any) -> None:
         print(json.dumps(d))
 
-    def exception(self, ex: Exception):
+    def exception(self, ex: Exception) -> None:
         self._out({"type": ex.__class__.__name__, "message": str(ex)})
 
-    def identifiers(self, identifiers: List[Identifier]):
+    def identifiers(self, identifiers: List[Identifier]) -> None:
         self._out([".".join(identifier) for identifier in identifiers])
 
-    def describe_table(self, table: Table):
+    def describe_table(self, table: Table) -> None:
         print(table.json())
 
-    def describe_properties(self, properties: Properties):
+    def describe_properties(self, properties: Properties) -> None:
         self._out(properties)
 
-    def text(self, response: str):
+    def text(self, response: str) -> None:
         print(json.dumps(response))
 
-    def schema(self, schema: Schema):
+    def schema(self, schema: Schema) -> None:
         print(schema.json())
 
-    def spec(self, spec: PartitionSpec):
+    def files(self, table: Table, io: FileIO, history: bool) -> None:
+        pass
+
+    def spec(self, spec: PartitionSpec) -> None:
         print(spec.json())
 
-    def uuid(self, uuid: Optional[UUID]):
+    def uuid(self, uuid: Optional[UUID]) -> None:
         self._out({"uuid": str(uuid) if uuid else "missing"})
diff --git a/python/pyiceberg/manifest.py b/python/pyiceberg/manifest.py
index 6079fcc336..751dfbef2a 100644
--- a/python/pyiceberg/manifest.py
+++ b/python/pyiceberg/manifest.py
@@ -29,7 +29,7 @@ from pydantic import Field
 
 from pyiceberg.avro.file import AvroFile
 from pyiceberg.avro.reader import AvroStruct
-from pyiceberg.io import InputFile
+from pyiceberg.io import FileIO, InputFile
 from pyiceberg.schema import Schema
 from pyiceberg.types import (
     IcebergType,
@@ -128,6 +128,10 @@ class ManifestFile(IcebergBaseModel):
     partitions: Optional[List[FieldSummary]] = Field()
     key_metadata: Optional[bytes] = Field()
 
+    def fetch_manifest_entry(self, io: FileIO) -> List[ManifestEntry]:
+        file = io.new_input(self.manifest_path)
+        return list(read_manifest_entry(file))
+
 
 def read_manifest_entry(input_file: InputFile) -> Iterator[ManifestEntry]:
     with AvroFile(input_file) as reader:
diff --git a/python/pyiceberg/table/snapshots.py b/python/pyiceberg/table/snapshots.py
index 849b2f3c06..35dd3c87e7 100644
--- a/python/pyiceberg/table/snapshots.py
+++ b/python/pyiceberg/table/snapshots.py
@@ -15,10 +15,17 @@
 # specific language governing permissions and limitations
 # under the License.
 from enum import Enum
-from typing import Dict, Optional, Union
+from typing import (
+    Dict,
+    List,
+    Optional,
+    Union,
+)
 
 from pydantic import Field, PrivateAttr, root_validator
 
+from pyiceberg.io import FileIO
+from pyiceberg.manifest import ManifestFile, read_manifest_list
 from pyiceberg.utils.iceberg_base_model import IcebergBaseModel
 
 OPERATION = "operation"
@@ -103,6 +110,12 @@ class Snapshot(IcebergBaseModel):
         result_str = f"{operation}id={self.snapshot_id}{parent_id}{schema_id}"
         return result_str
 
+    def fetch_manifest_list(self, io: FileIO) -> List[ManifestFile]:
+        if self.manifest_list is not None:
+            file = io.new_input(self.manifest_list)
+            return list(read_manifest_list(file))
+        return []
+
 
 class MetadataLogEntry(IcebergBaseModel):
     metadata_file: str = Field(alias="metadata-file")
diff --git a/python/tests/avro/test_reader.py b/python/tests/avro/test_reader.py
index e4b0cb665e..44505a6174 100644
--- a/python/tests/avro/test_reader.py
+++ b/python/tests/avro/test_reader.py
@@ -376,9 +376,10 @@ def test_read_manifest_file_file(generated_manifest_file_file: str):
         records = list(reader)
 
     assert len(records) == 1, f"Expected 1 records, got {len(records)}"
-    assert records[0] == AvroStruct(
+    actual = records[0]
+    expected = AvroStruct(
         _data=[
-            "/home/iceberg/warehouse/nyc/taxis_partitioned/metadata/0125c686-8aa6-4502-bdcc-b6d17ca41a3b-m0.avro",
+            actual.get(0),
             7989,
             0,
             9182715666859759686,
@@ -391,6 +392,7 @@ def test_read_manifest_file_file(generated_manifest_file_file: str):
             0,
         ]
     )
+    assert actual == expected
 
 
 def test_fixed_reader():
diff --git a/python/tests/conftest.py b/python/tests/conftest.py
index 4030fbe9f1..be51c90357 100644
--- a/python/tests/conftest.py
+++ b/python/tests/conftest.py
@@ -26,7 +26,12 @@ retrieved using `request.getfixturevalue(fixture_name)`.
 """
 import os
 from tempfile import TemporaryDirectory
-from typing import Any, Dict, Union
+from typing import (
+    Any,
+    Dict,
+    Generator,
+    Union,
+)
 from urllib.parse import urlparse
 
 import pytest
@@ -932,7 +937,7 @@ def LocalFileIOFixture():
 
 
 @pytest.fixture(scope="session")
-def generated_manifest_entry_file(avro_schema_manifest_entry):
+def generated_manifest_entry_file(avro_schema_manifest_entry: Dict[str, Any]) -> Generator[str, None, None]:
     from fastavro import parse_schema, writer
 
     parsed_schema = parse_schema(avro_schema_manifest_entry)
@@ -945,11 +950,16 @@ def generated_manifest_entry_file(avro_schema_manifest_entry):
 
 
 @pytest.fixture(scope="session")
-def generated_manifest_file_file(avro_schema_manifest_file):
+def generated_manifest_file_file(
+    avro_schema_manifest_file: Dict[str, Any], generated_manifest_entry_file: str
+) -> Generator[str, None, None]:
     from fastavro import parse_schema, writer
 
     parsed_schema = parse_schema(avro_schema_manifest_file)
 
+    # Make sure that a valid manifest_path is set
+    manifest_file_records[0]["manifest_path"] = generated_manifest_entry_file
+
     with TemporaryDirectory() as tmpdir:
         tmp_avro_file = tmpdir + "/manifest.avro"
         with open(tmp_avro_file, "wb") as out:
diff --git a/python/tests/table/test_snapshots.py b/python/tests/table/test_snapshots.py
index 5dc48807ba..60d3bfa226 100644
--- a/python/tests/table/test_snapshots.py
+++ b/python/tests/table/test_snapshots.py
@@ -17,6 +17,8 @@
 # pylint:disable=redefined-outer-name,eval-used
 import pytest
 
+from pyiceberg.io.pyarrow import PyArrowFileIO
+from pyiceberg.manifest import FieldSummary, ManifestContent, ManifestFile
 from pyiceberg.table.snapshots import Operation, Snapshot, Summary
 
 
@@ -119,3 +121,40 @@ def test_snapshot_with_properties_repr(snapshot_with_properties: Snapshot):
         == """Snapshot(snapshot_id=25, parent_snapshot_id=19, sequence_number=200, timestamp_ms=1602638573590, manifest_list='s3:/a/b/c.avro', summary=Summary(Operation.APPEND, **{'foo': 'bar'}), schema_id=3)"""
     )
     assert snapshot_with_properties == eval(repr(snapshot_with_properties))
+
+
+def test_fetch_manifest_list(generated_manifest_file_file: str):
+    snapshot = Snapshot(
+        snapshot_id=25,
+        parent_snapshot_id=19,
+        sequence_number=200,
+        timestamp_ms=1602638573590,
+        manifest_list=generated_manifest_file_file,
+        summary=Summary(Operation.APPEND),
+        schema_id=3,
+    )
+    io = PyArrowFileIO()
+    actual = snapshot.fetch_manifest_list(io)
+    assert actual == [
+        ManifestFile(
+            manifest_path=actual[0].manifest_path,  # Is a temp path that changes every time
+            manifest_length=7989,
+            partition_spec_id=0,
+            content=ManifestContent.DATA,
+            sequence_number=0,
+            min_sequence_number=0,
+            added_snapshot_id=9182715666859759686,
+            added_data_files_count=3,
+            existing_data_files_count=0,
+            deleted_data_files_count=0,
+            added_rows_count=237993,
+            existing_rows_counts=None,
+            deleted_rows_count=0,
+            partitions=[
+                FieldSummary(
+                    contains_null=True, contains_nan=False, lower_bound=b"\x01\x00\x00\x00", upper_bound=b"\x02\x00\x00\x00"
+                )
+            ],
+            key_metadata=None,
+        )
+    ]
diff --git a/python/tests/utils/test_manifest.py b/python/tests/utils/test_manifest.py
index 66b70a2dc0..ae19c5d242 100644
--- a/python/tests/utils/test_manifest.py
+++ b/python/tests/utils/test_manifest.py
@@ -14,15 +14,22 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+
+from pyiceberg.io import load_file_io
 from pyiceberg.manifest import (
     DataFile,
+    DataFileContent,
     FieldSummary,
     FileFormat,
+    ManifestContent,
     ManifestEntry,
+    ManifestEntryStatus,
     ManifestFile,
     read_manifest_entry,
     read_manifest_list,
 )
+from pyiceberg.table import Snapshot
+from pyiceberg.table.snapshots import Operation, Summary
 from tests.io.test_io import LocalInputFile
 
 
@@ -262,9 +269,10 @@ def test_read_manifest_entry(generated_manifest_entry_file: str):
 
 def test_read_manifest_list(generated_manifest_file_file: str):
     input_file = LocalInputFile(generated_manifest_file_file)
-    assert list(read_manifest_list(input_file)) == [
+    actual = list(read_manifest_list(input_file))
+    expected = [
         ManifestFile(
-            manifest_path="/home/iceberg/warehouse/nyc/taxis_partitioned/metadata/0125c686-8aa6-4502-bdcc-b6d17ca41a3b-m0.avro",
+            manifest_path=actual[0].manifest_path,
             manifest_length=7989,
             partition_spec_id=0,
             added_snapshot_id=9182715666859759686,
@@ -281,3 +289,284 @@ def test_read_manifest_list(generated_manifest_file_file: str):
             deleted_rows_count=0,
         )
     ]
+    assert actual == expected
+
+
+def test_read_manifest(generated_manifest_file_file: str, generated_manifest_entry_file: str):
+    io = load_file_io({})
+
+    snapshot = Snapshot(
+        snapshot_id=25,
+        parent_snapshot_id=19,
+        timestamp_ms=1602638573590,
+        manifest_list=generated_manifest_file_file,
+        summary=Summary(Operation.APPEND),
+        schema_id=3,
+    )
+    manifest_list = snapshot.fetch_manifest_list(io)
+
+    assert manifest_list == [
+        ManifestFile(
+            manifest_path=generated_manifest_entry_file,
+            manifest_length=7989,
+            partition_spec_id=0,
+            content=ManifestContent.DATA,
+            sequence_number=0,
+            min_sequence_number=0,
+            added_snapshot_id=9182715666859759686,
+            added_data_files_count=3,
+            existing_data_files_count=0,
+            deleted_data_files_count=0,
+            added_rows_count=237993,
+            existing_rows_counts=None,
+            deleted_rows_count=0,
+            partitions=[
+                FieldSummary(
+                    contains_null=True, contains_nan=False, lower_bound=b"\x01\x00\x00\x00", upper_bound=b"\x02\x00\x00\x00"
+                )
+            ],
+            key_metadata=None,
+        )
+    ]
+
+    actual = manifest_list[0].fetch_manifest_entry(io)
+    expected = [
+        ManifestEntry(
+            status=ManifestEntryStatus.ADDED,
+            snapshot_id=8744736658442914487,
+            sequence_number=None,
+            data_file=DataFile(
+                content=DataFileContent.DATA,
+                file_path="/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=null/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00001.parquet",
+                file_format=FileFormat.PARQUET,
+                partition={"VendorID": None},
+                record_count=19513,
+                file_size_in_bytes=388872,
+                block_size_in_bytes=67108864,
+                column_sizes={
+                    1: 53,
+                    2: 98153,
+                    3: 98693,
+                    4: 53,
+                    5: 53,
+                    6: 53,
+                    7: 17425,
+                    8: 18528,
+                    9: 53,
+                    10: 44788,
+                    11: 35571,
+                    12: 53,
+                    13: 1243,
+                    14: 2355,
+                    15: 12750,
+                    16: 4029,
+                    17: 110,
+                    18: 47194,
+                    19: 2948,
+                },
+                value_counts={
+                    1: 19513,
+                    2: 19513,
+                    3: 19513,
+                    4: 19513,
+                    5: 19513,
+                    6: 19513,
+                    7: 19513,
+                    8: 19513,
+                    9: 19513,
+                    10: 19513,
+                    11: 19513,
+                    12: 19513,
+                    13: 19513,
+                    14: 19513,
+                    15: 19513,
+                    16: 19513,
+                    17: 19513,
+                    18: 19513,
+                    19: 19513,
+                },
+                null_value_counts={
+                    1: 19513,
+                    2: 0,
+                    3: 0,
+                    4: 19513,
+                    5: 19513,
+                    6: 19513,
+                    7: 0,
+                    8: 0,
+                    9: 19513,
+                    10: 0,
+                    11: 0,
+                    12: 19513,
+                    13: 0,
+                    14: 0,
+                    15: 0,
+                    16: 0,
+                    17: 0,
+                    18: 0,
+                    19: 0,
+                },
+                nan_value_counts={16: 0, 17: 0, 18: 0, 19: 0, 10: 0, 11: 0, 12: 0, 13: 0, 14: 0, 15: 0},
+                distinct_counts=None,
+                lower_bounds={
+                    2: b"2020-04-01 00:00",
+                    3: b"2020-04-01 00:12",
+                    7: b"\x03\x00\x00\x00",
+                    8: b"\x01\x00\x00\x00",
+                    10: b"\xf6(\\\x8f\xc2\x05S\xc0",
+                    11: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    13: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    14: b"\x00\x00\x00\x00\x00\x00\xe0\xbf",
+                    15: b")\\\x8f\xc2\xf5(\x08\xc0",
+                    16: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    17: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    18: b"\xf6(\\\x8f\xc2\xc5S\xc0",
+                    19: b"\x00\x00\x00\x00\x00\x00\x04\xc0",
+                },
+                upper_bounds={
+                    2: b"2020-04-30 23:5:",
+                    3: b"2020-05-01 00:41",
+                    7: b"\t\x01\x00\x00",
+                    8: b"\t\x01\x00\x00",
+                    10: b"\xcd\xcc\xcc\xcc\xcc,_@",
+                    11: b"\x1f\x85\xebQ\\\xe2\xfe@",
+                    13: b"\x00\x00\x00\x00\x00\x00\x12@",
+                    14: b"\x00\x00\x00\x00\x00\x00\xe0?",
+                    15: b"q=\n\xd7\xa3\xf01@",
+                    16: b"\x00\x00\x00\x00\x00`B@",
+                    17: b"333333\xd3?",
+                    18: b"\x00\x00\x00\x00\x00\x18b@",
+                    19: b"\x00\x00\x00\x00\x00\x00\x04@",
+                },
+                key_metadata=None,
+                split_offsets=[4],
+                equality_ids=None,
+                sort_order_id=0,
+            ),
+        ),
+        ManifestEntry(
+            status=ManifestEntryStatus.ADDED,
+            snapshot_id=8744736658442914487,
+            sequence_number=None,
+            data_file=DataFile(
+                content=DataFileContent.DATA,
+                file_path="/home/iceberg/warehouse/nyc/taxis_partitioned/data/VendorID=1/00000-633-d8a4223e-dc97-45a1-86e1-adaba6e8abd7-00002.parquet",
+                file_format=FileFormat.PARQUET,
+                partition={"VendorID": 1},
+                record_count=95050,
+                file_size_in_bytes=1265950,
+                block_size_in_bytes=67108864,
+                column_sizes={
+                    1: 318,
+                    2: 329806,
+                    3: 331632,
+                    4: 15343,
+                    5: 2351,
+                    6: 3389,
+                    7: 71269,
+                    8: 76429,
+                    9: 16383,
+                    10: 86992,
+                    11: 89608,
+                    12: 265,
+                    13: 19377,
+                    14: 1692,
+                    15: 76162,
+                    16: 4354,
+                    17: 759,
+                    18: 120650,
+                    19: 11804,
+                },
+                value_counts={
+                    1: 95050,
+                    2: 95050,
+                    3: 95050,
+                    4: 95050,
+                    5: 95050,
+                    6: 95050,
+                    7: 95050,
+                    8: 95050,
+                    9: 95050,
+                    10: 95050,
+                    11: 95050,
+                    12: 95050,
+                    13: 95050,
+                    14: 95050,
+                    15: 95050,
+                    16: 95050,
+                    17: 95050,
+                    18: 95050,
+                    19: 95050,
+                },
+                null_value_counts={
+                    1: 0,
+                    2: 0,
+                    3: 0,
+                    4: 0,
+                    5: 0,
+                    6: 0,
+                    7: 0,
+                    8: 0,
+                    9: 0,
+                    10: 0,
+                    11: 0,
+                    12: 95050,
+                    13: 0,
+                    14: 0,
+                    15: 0,
+                    16: 0,
+                    17: 0,
+                    18: 0,
+                    19: 0,
+                },
+                nan_value_counts={16: 0, 17: 0, 18: 0, 19: 0, 10: 0, 11: 0, 12: 0, 13: 0, 14: 0, 15: 0},
+                distinct_counts=None,
+                lower_bounds={
+                    1: b"\x01\x00\x00\x00",
+                    2: b"2020-04-01 00:00",
+                    3: b"2020-04-01 00:03",
+                    4: b"\x00\x00\x00\x00",
+                    5: b"\x01\x00\x00\x00",
+                    6: b"N",
+                    7: b"\x01\x00\x00\x00",
+                    8: b"\x01\x00\x00\x00",
+                    9: b"\x01\x00\x00\x00",
+                    10: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    11: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    13: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    14: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    15: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    16: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    17: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    18: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                    19: b"\x00\x00\x00\x00\x00\x00\x00\x00",
+                },
+                upper_bounds={
+                    1: b"\x01\x00\x00\x00",
+                    2: b"2020-04-30 23:5:",
+                    3: b"2020-05-01 00:1:",
+                    4: b"\x06\x00\x00\x00",
+                    5: b"c\x00\x00\x00",
+                    6: b"Y",
+                    7: b"\t\x01\x00\x00",
+                    8: b"\t\x01\x00\x00",
+                    9: b"\x04\x00\x00\x00",
+                    10: b"\\\x8f\xc2\xf5(8\x8c@",
+                    11: b"\xcd\xcc\xcc\xcc\xcc,f@",
+                    13: b"\x00\x00\x00\x00\x00\x00\x1c@",
+                    14: b"\x9a\x99\x99\x99\x99\x99\xf1?",
+                    15: b"\x00\x00\x00\x00\x00\x00Y@",
+                    16: b"\x00\x00\x00\x00\x00\xb0X@",
+                    17: b"333333\xd3?",
+                    18: b"\xc3\xf5(\\\x8f:\x8c@",
+                    19: b"\x00\x00\x00\x00\x00\x00\x04@",
+                },
+                key_metadata=None,
+                split_offsets=[4],
+                equality_ids=None,
+                sort_order_id=0,
+            ),
+        ),
+    ]
+
+    assert actual == expected