|
108 | 108 | InputStream, |
109 | 109 | OutputFile, |
110 | 110 | OutputStream, |
| 111 | + _parse_location, |
111 | 112 | ) |
112 | 113 | from pyiceberg.manifest import ( |
113 | 114 | DataFile, |
@@ -1195,7 +1196,7 @@ def _task_to_record_batches( |
1195 | 1196 | name_mapping: Optional[NameMapping] = None, |
1196 | 1197 | use_large_types: bool = True, |
1197 | 1198 | ) -> Iterator[pa.RecordBatch]: |
1198 | | - _, _, path = PyArrowFileIO.parse_location(task.file.file_path) |
| 1199 | + _, _, path = _parse_location(task.file.file_path) |
1199 | 1200 | arrow_format = ds.ParquetFileFormat(pre_buffer=True, buffer_size=(ONE_MEGABYTE * 8)) |
1200 | 1201 | with fs.open_input_file(path) as fin: |
1201 | 1202 | fragment = arrow_format.make_fragment(fin) |
@@ -1304,6 +1305,195 @@ def _read_all_delete_files(fs: FileSystem, tasks: Iterable[FileScanTask]) -> Dic |
1304 | 1305 | return deletes_per_file |
1305 | 1306 |
|
1306 | 1307 |
|
| 1308 | +def _fs_from_file_path(file_path: str, io: FileIO) -> FileSystem: |
| 1309 | + scheme, netloc, _ = _parse_location(file_path) |
| 1310 | + if isinstance(io, PyArrowFileIO): |
| 1311 | + return io.fs_by_scheme(scheme, netloc) |
| 1312 | + else: |
| 1313 | + try: |
| 1314 | + from pyiceberg.io.fsspec import FsspecFileIO |
| 1315 | + |
| 1316 | + if isinstance(io, FsspecFileIO): |
| 1317 | + from pyarrow.fs import PyFileSystem |
| 1318 | + |
| 1319 | + return PyFileSystem(FSSpecHandler(io.get_fs(scheme))) |
| 1320 | + else: |
| 1321 | + raise ValueError(f"Expected PyArrowFileIO or FsspecFileIO, got: {io}") |
| 1322 | + except ModuleNotFoundError as e: |
| 1323 | + # When FsSpec is not installed |
| 1324 | + raise ValueError(f"Expected PyArrowFileIO or FsspecFileIO, got: {io}") from e |
| 1325 | + |
| 1326 | + |
| 1327 | +class ArrowScan: |
| 1328 | + _table_metadata: TableMetadata |
| 1329 | + _io: FileIO |
| 1330 | + _fs: FileSystem |
| 1331 | + _projected_schema: Schema |
| 1332 | + _bound_row_filter: BooleanExpression |
| 1333 | + _case_sensitive: bool |
| 1334 | + _limit: Optional[int] |
| 1335 | + """Scan the Iceberg Table and create an Arrow construct. |
| 1336 | +
|
| 1337 | + Attributes: |
| 1338 | + _table_metadata: Current table metadata of the Iceberg table |
| 1339 | + _io: PyIceberg FileIO implementation from which to fetch the io properties |
| 1340 | + _fs: PyArrow FileSystem to use to read the files |
| 1341 | + _projected_schema: Iceberg Schema to project onto the data files |
| 1342 | + _bound_row_filter: Schema bound row expression to filter the data with |
| 1343 | + _case_sensitive: Case sensitivity when looking up column names |
| 1344 | + _limit: Limit the number of records. |
| 1345 | + """ |
| 1346 | + |
| 1347 | + def __init__( |
| 1348 | + self, |
| 1349 | + table_metadata: TableMetadata, |
| 1350 | + io: FileIO, |
| 1351 | + projected_schema: Schema, |
| 1352 | + row_filter: BooleanExpression, |
| 1353 | + case_sensitive: bool = True, |
| 1354 | + limit: Optional[int] = None, |
| 1355 | + ) -> None: |
| 1356 | + self._table_metadata = table_metadata |
| 1357 | + self._io = io |
| 1358 | + self._fs = _fs_from_file_path(table_metadata.location, io) # TODO: use different FileSystem per file |
| 1359 | + self._projected_schema = projected_schema |
| 1360 | + self._bound_row_filter = bind(table_metadata.schema(), row_filter, case_sensitive=case_sensitive) |
| 1361 | + self._case_sensitive = case_sensitive |
| 1362 | + self._limit = limit |
| 1363 | + |
| 1364 | + @property |
| 1365 | + def _use_large_types(self) -> bool: |
| 1366 | + """Whether to represent data as large arrow types. |
| 1367 | +
|
| 1368 | + Defaults to True. |
| 1369 | + """ |
| 1370 | + return property_as_bool(self._io.properties, PYARROW_USE_LARGE_TYPES_ON_READ, True) |
| 1371 | + |
| 1372 | + @property |
| 1373 | + def _projected_field_ids(self) -> Set[int]: |
| 1374 | + """Set of field IDs that should be projected from the data files.""" |
| 1375 | + return { |
| 1376 | + id |
| 1377 | + for id in self._projected_schema.field_ids |
| 1378 | + if not isinstance(self._projected_schema.find_type(id), (MapType, ListType)) |
| 1379 | + }.union(extract_field_ids(self._bound_row_filter)) |
| 1380 | + |
| 1381 | + def to_table(self, tasks: Iterable[FileScanTask]) -> pa.Table: |
| 1382 | + """Scan the Iceberg table and return a pa.Table. |
| 1383 | +
|
| 1384 | + Returns a pa.Table with data from the Iceberg table by resolving the |
| 1385 | + right columns that match the current table schema. Only data that |
| 1386 | + matches the provided row_filter expression is returned. |
| 1387 | +
|
| 1388 | + Args: |
| 1389 | + tasks: FileScanTasks representing the data files and delete files to read from. |
| 1390 | +
|
| 1391 | + Returns: |
| 1392 | + A PyArrow table. Total number of rows will be capped if specified. |
| 1393 | +
|
| 1394 | + Raises: |
| 1395 | + ResolveError: When a required field cannot be found in the file |
| 1396 | + ValueError: When a field type in the file cannot be projected to the schema type |
| 1397 | + """ |
| 1398 | + deletes_per_file = _read_all_delete_files(self._fs, tasks) |
| 1399 | + executor = ExecutorFactory.get_or_create() |
| 1400 | + |
| 1401 | + def _table_from_scan_task(task: FileScanTask) -> pa.Table: |
| 1402 | + batches = list(self._record_batches_from_scan_tasks_and_deletes([task], deletes_per_file)) |
| 1403 | + if len(batches) > 0: |
| 1404 | + return pa.Table.from_batches(batches) |
| 1405 | + else: |
| 1406 | + return None |
| 1407 | + |
| 1408 | + futures = [ |
| 1409 | + executor.submit( |
| 1410 | + _table_from_scan_task, |
| 1411 | + task, |
| 1412 | + ) |
| 1413 | + for task in tasks |
| 1414 | + ] |
| 1415 | + total_row_count = 0 |
| 1416 | + # for consistent ordering, we need to maintain future order |
| 1417 | + futures_index = {f: i for i, f in enumerate(futures)} |
| 1418 | + completed_futures: SortedList[Future[pa.Table]] = SortedList(iterable=[], key=lambda f: futures_index[f]) |
| 1419 | + for future in concurrent.futures.as_completed(futures): |
| 1420 | + completed_futures.add(future) |
| 1421 | + if table_result := future.result(): |
| 1422 | + total_row_count += len(table_result) |
| 1423 | + # stop early if limit is satisfied |
| 1424 | + if self._limit is not None and total_row_count >= self._limit: |
| 1425 | + break |
| 1426 | + |
| 1427 | + # by now, we've either completed all tasks or satisfied the limit |
| 1428 | + if self._limit is not None: |
| 1429 | + _ = [f.cancel() for f in futures if not f.done()] |
| 1430 | + |
| 1431 | + tables = [f.result() for f in completed_futures if f.result()] |
| 1432 | + |
| 1433 | + if len(tables) < 1: |
| 1434 | + return pa.Table.from_batches([], schema=schema_to_pyarrow(self._projected_schema, include_field_ids=False)) |
| 1435 | + |
| 1436 | + result = pa.concat_tables(tables, promote_options="permissive") |
| 1437 | + |
| 1438 | + if self._limit is not None: |
| 1439 | + return result.slice(0, self._limit) |
| 1440 | + |
| 1441 | + return result |
| 1442 | + |
| 1443 | + def to_record_batches(self, tasks: Iterable[FileScanTask]) -> Iterator[pa.RecordBatch]: |
| 1444 | + """Scan the Iceberg table and return an Iterator[pa.RecordBatch]. |
| 1445 | +
|
| 1446 | + Returns an Iterator of pa.RecordBatch with data from the Iceberg table |
| 1447 | + by resolving the right columns that match the current table schema. |
| 1448 | + Only data that matches the provided row_filter expression is returned. |
| 1449 | +
|
| 1450 | + Args: |
| 1451 | + tasks: FileScanTasks representing the data files and delete files to read from. |
| 1452 | +
|
| 1453 | + Returns: |
| 1454 | + An Iterator of PyArrow RecordBatches. |
| 1455 | + Total number of rows will be capped if specified. |
| 1456 | +
|
| 1457 | + Raises: |
| 1458 | + ResolveError: When a required field cannot be found in the file |
| 1459 | + ValueError: When a field type in the file cannot be projected to the schema type |
| 1460 | + """ |
| 1461 | + deletes_per_file = _read_all_delete_files(self._fs, tasks) |
| 1462 | + return self._record_batches_from_scan_tasks_and_deletes(tasks, deletes_per_file) |
| 1463 | + |
| 1464 | + def _record_batches_from_scan_tasks_and_deletes( |
| 1465 | + self, tasks: Iterable[FileScanTask], deletes_per_file: Dict[str, List[ChunkedArray]] |
| 1466 | + ) -> Iterator[pa.RecordBatch]: |
| 1467 | + total_row_count = 0 |
| 1468 | + for task in tasks: |
| 1469 | + if self._limit is not None and total_row_count >= self._limit: |
| 1470 | + break |
| 1471 | + batches = _task_to_record_batches( |
| 1472 | + self._fs, |
| 1473 | + task, |
| 1474 | + self._bound_row_filter, |
| 1475 | + self._projected_schema, |
| 1476 | + self._projected_field_ids, |
| 1477 | + deletes_per_file.get(task.file.file_path), |
| 1478 | + self._case_sensitive, |
| 1479 | + self._table_metadata.name_mapping(), |
| 1480 | + self._use_large_types, |
| 1481 | + ) |
| 1482 | + for batch in batches: |
| 1483 | + if self._limit is not None: |
| 1484 | + if total_row_count >= self._limit: |
| 1485 | + break |
| 1486 | + elif total_row_count + len(batch) >= self._limit: |
| 1487 | + batch = batch.slice(0, self._limit - total_row_count) |
| 1488 | + yield batch |
| 1489 | + total_row_count += len(batch) |
| 1490 | + |
| 1491 | + |
| 1492 | +@deprecated( |
| 1493 | + deprecated_in="0.8.0", |
| 1494 | + removed_in="0.9.0", |
| 1495 | + help_message="project_table is deprecated. Use ArrowScan.to_table instead.", |
| 1496 | +) |
1307 | 1497 | def project_table( |
1308 | 1498 | tasks: Iterable[FileScanTask], |
1309 | 1499 | table_metadata: TableMetadata, |
@@ -1398,6 +1588,11 @@ def project_table( |
1398 | 1588 | return result |
1399 | 1589 |
|
1400 | 1590 |
|
| 1591 | +@deprecated( |
| 1592 | + deprecated_in="0.8.0", |
| 1593 | + removed_in="0.9.0", |
| 1594 | + help_message="project_table is deprecated. Use ArrowScan.to_record_batches instead.", |
| 1595 | +) |
1401 | 1596 | def project_batches( |
1402 | 1597 | tasks: Iterable[FileScanTask], |
1403 | 1598 | table_metadata: TableMetadata, |
|
0 commit comments