|
43 | 43 | ) |
44 | 44 | from pyiceberg.partitioning import PartitionField, PartitionSpec |
45 | 45 | from pyiceberg.schema import Schema |
46 | | -from pyiceberg.table.metadata import TableMetadataUtil, TableMetadataV2 |
| 46 | +from pyiceberg.table.metadata import TableMetadataUtil, TableMetadataV1, TableMetadataV2 |
47 | 47 | from pyiceberg.table.refs import SnapshotRef, SnapshotRefType |
48 | 48 | from pyiceberg.table.snapshots import ( |
49 | 49 | MetadataLogEntry, |
@@ -295,6 +295,61 @@ def test_create_table(table_schema_simple: Schema, hive_database: HiveDatabase, |
295 | 295 | assert metadata.model_dump() == expected.model_dump() |
296 | 296 |
|
297 | 297 |
|
| 298 | +@patch("time.time", MagicMock(return_value=12345)) |
| 299 | +def test_create_v1_table(table_schema_simple: Schema, hive_database: HiveDatabase, hive_table: HiveTable) -> None: |
| 300 | + catalog = HiveCatalog(HIVE_CATALOG_NAME, uri=HIVE_METASTORE_FAKE_URL) |
| 301 | + |
| 302 | + catalog._client = MagicMock() |
| 303 | + catalog._client.__enter__().create_table.return_value = None |
| 304 | + catalog._client.__enter__().get_table.return_value = hive_table |
| 305 | + catalog._client.__enter__().get_database.return_value = hive_database |
| 306 | + catalog.create_table( |
| 307 | + ("default", "table"), schema=table_schema_simple, properties={"owner": "javaberg", "format-version": "1"} |
| 308 | + ) |
| 309 | + |
| 310 | + # Test creating V1 table |
| 311 | + called_v1_table: HiveTable = catalog._client.__enter__().create_table.call_args[0][0] |
| 312 | + metadata_location = called_v1_table.parameters["metadata_location"] |
| 313 | + with open(metadata_location, encoding=UTF8) as f: |
| 314 | + payload = f.read() |
| 315 | + |
| 316 | + expected_schema = Schema( |
| 317 | + NestedField(field_id=1, name="foo", field_type=StringType(), required=False), |
| 318 | + NestedField(field_id=2, name="bar", field_type=IntegerType(), required=True), |
| 319 | + NestedField(field_id=3, name="baz", field_type=BooleanType(), required=False), |
| 320 | + schema_id=0, |
| 321 | + identifier_field_ids=[2], |
| 322 | + ) |
| 323 | + actual_v1_metadata = TableMetadataUtil.parse_raw(payload) |
| 324 | + expected_spec = PartitionSpec() |
| 325 | + expected_spec_dump = expected_spec.model_dump() |
| 326 | + expected_spec_dump["fields"] = [] |
| 327 | + expected_v1_metadata = TableMetadataV1( |
| 328 | + location=actual_v1_metadata.location, |
| 329 | + table_uuid=actual_v1_metadata.table_uuid, |
| 330 | + last_updated_ms=actual_v1_metadata.last_updated_ms, |
| 331 | + last_column_id=3, |
| 332 | + schema=expected_schema, |
| 333 | + schemas=[expected_schema], |
| 334 | + current_schema_id=0, |
| 335 | + last_partition_id=1000, |
| 336 | + properties={"owner": "javaberg", "write.parquet.compression-codec": "zstd"}, |
| 337 | + partition_spec=[expected_spec_dump], |
| 338 | + partition_specs=[expected_spec], |
| 339 | + default_spec_id=0, |
| 340 | + current_snapshot_id=None, |
| 341 | + snapshots=[], |
| 342 | + snapshot_log=[], |
| 343 | + metadata_log=[], |
| 344 | + sort_orders=[SortOrder(order_id=0)], |
| 345 | + default_sort_order_id=0, |
| 346 | + refs={}, |
| 347 | + format_version=1, |
| 348 | + ) |
| 349 | + |
| 350 | + assert actual_v1_metadata.model_dump() == expected_v1_metadata.model_dump() |
| 351 | + |
| 352 | + |
298 | 353 | def test_load_table(hive_table: HiveTable) -> None: |
299 | 354 | catalog = HiveCatalog(HIVE_CATALOG_NAME, uri=HIVE_METASTORE_FAKE_URL) |
300 | 355 |
|
|
0 commit comments