|
23 | 23 | from typing import Any, Dict
|
24 | 24 | from urllib.parse import urlparse
|
25 | 25 |
|
| 26 | +import numpy as np |
26 | 27 | import pandas as pd
|
27 | 28 | import pyarrow as pa
|
28 | 29 | import pyarrow.parquet as pq
|
|
38 | 39 | from pyiceberg.catalog.rest import RestCatalog
|
39 | 40 | from pyiceberg.catalog.sql import SqlCatalog
|
40 | 41 | from pyiceberg.exceptions import NoSuchTableError
|
41 |
| -from pyiceberg.expressions import In |
| 42 | +from pyiceberg.expressions import GreaterThanOrEqual, In, Not |
42 | 43 | from pyiceberg.io.pyarrow import _dataframe_to_data_files
|
43 | 44 | from pyiceberg.partitioning import PartitionField, PartitionSpec
|
44 | 45 | from pyiceberg.schema import Schema
|
45 | 46 | from pyiceberg.table import TableProperties
|
46 |
| -from pyiceberg.transforms import IdentityTransform |
47 |
| -from pyiceberg.types import IntegerType, LongType, NestedField, StringType |
| 47 | +from pyiceberg.transforms import DayTransform, IdentityTransform |
| 48 | +from pyiceberg.types import ( |
| 49 | + DateType, |
| 50 | + DoubleType, |
| 51 | + IntegerType, |
| 52 | + LongType, |
| 53 | + NestedField, |
| 54 | + StringType, |
| 55 | +) |
48 | 56 | from utils import _create_table
|
49 | 57 |
|
50 | 58 |
|
@@ -1331,3 +1339,71 @@ def test_overwrite_all_data_with_filter(session_catalog: Catalog) -> None:
|
1331 | 1339 | tbl.overwrite(data, In("id", ["1", "2", "3"]))
|
1332 | 1340 |
|
1333 | 1341 | assert len(tbl.scan().to_arrow()) == 3
|
| 1342 | + |
| 1343 | + |
| 1344 | +@pytest.mark.integration |
| 1345 | +def test_delete_threshold() -> None: |
| 1346 | + catalog = load_catalog( |
| 1347 | + "local", |
| 1348 | + **{ |
| 1349 | + "type": "rest", |
| 1350 | + "uri": "http://localhost:8181", |
| 1351 | + "s3.endpoint": "http://localhost:9000", |
| 1352 | + "s3.access-key-id": "admin", |
| 1353 | + "s3.secret-access-key": "password", |
| 1354 | + }, |
| 1355 | + ) |
| 1356 | + |
| 1357 | + schema = Schema( |
| 1358 | + NestedField(field_id=101, name="id", field_type=LongType(), required=True), |
| 1359 | + NestedField(field_id=103, name="created_at", field_type=DateType(), required=False), |
| 1360 | + NestedField(field_id=104, name="relevancy_score", field_type=DoubleType(), required=False), |
| 1361 | + ) |
| 1362 | + |
| 1363 | + partition_spec = PartitionSpec(PartitionField(source_id=103, field_id=2000, transform=DayTransform(), name="created_at_day")) |
| 1364 | + |
| 1365 | + try: |
| 1366 | + catalog.drop_table( |
| 1367 | + identifier="default.scores", |
| 1368 | + ) |
| 1369 | + except NoSuchTableError: |
| 1370 | + pass |
| 1371 | + |
| 1372 | + catalog.create_table( |
| 1373 | + identifier="default.scores", |
| 1374 | + schema=schema, |
| 1375 | + partition_spec=partition_spec, |
| 1376 | + ) |
| 1377 | + |
| 1378 | + # Parameters |
| 1379 | + num_rows = 100 # Number of rows in the dataframe |
| 1380 | + id_min, id_max = 1, 10000 |
| 1381 | + date_start, date_end = date(2024, 1, 1), date(2024, 2, 1) |
| 1382 | + |
| 1383 | + # Generate the 'id' column |
| 1384 | + id_column = np.random.randint(id_min, id_max, num_rows) |
| 1385 | + |
| 1386 | + # Generate the 'created_at' column as dates only |
| 1387 | + date_range = pd.date_range(start=date_start, end=date_end, freq="D") # Daily frequency for dates |
| 1388 | + created_at_column = np.random.choice(date_range, num_rows) # Convert to string (YYYY-MM-DD format) |
| 1389 | + |
| 1390 | + # Generate the 'relevancy_score' column with a peak around 0.1 |
| 1391 | + relevancy_score_column = np.random.beta(a=2, b=20, size=num_rows) # Adjusting parameters to peak around 0.1 |
| 1392 | + |
| 1393 | + # Create the dataframe |
| 1394 | + df = pd.DataFrame({"id": id_column, "created_at": created_at_column, "relevancy_score": relevancy_score_column}) |
| 1395 | + |
| 1396 | + iceberg_table = catalog.load_table("default.scores") |
| 1397 | + |
| 1398 | + # Convert the pandas DataFrame to a PyArrow Table with the Iceberg schema |
| 1399 | + arrow_schema = iceberg_table.schema().as_arrow() |
| 1400 | + docs_table = pa.Table.from_pandas(df, schema=arrow_schema) |
| 1401 | + |
| 1402 | + # Append the data to the Iceberg table |
| 1403 | + iceberg_table.append(docs_table) |
| 1404 | + |
| 1405 | + delete_condition = GreaterThanOrEqual("relevancy_score", 0.1) |
| 1406 | + lower_before = len(iceberg_table.scan(row_filter=Not(delete_condition)).to_arrow()) |
| 1407 | + assert len(iceberg_table.scan(row_filter=Not(delete_condition)).to_arrow()) == lower_before |
| 1408 | + iceberg_table.delete(delete_condition) |
| 1409 | + assert len(iceberg_table.scan().to_arrow()) == lower_before |
0 commit comments