9
9
from __future__ import annotations
10
10
11
11
import logging
12
+ from dataclasses import dataclass
12
13
from typing import (
13
14
TYPE_CHECKING ,
14
15
Any ,
34
35
)
35
36
from ray .data .datasource .datasource import ReadTask
36
37
from ray .data .datasource .file_meta_provider import (
37
- DefaultParquetMetadataProvider ,
38
- ParquetMetadataProvider ,
39
38
_handle_read_os_error ,
40
39
)
40
+ from ray .data .datasource .parquet_meta_provider import (
41
+ ParquetMetadataProvider ,
42
+ )
41
43
from ray .data .datasource .partitioning import PathPartitionFilter
42
44
from ray .data .datasource .path_util import (
43
45
_has_file_extension ,
55
57
56
58
_logger : logging .Logger = logging .getLogger (__name__ )
57
59
58
- FRAGMENTS_PER_META_FETCH = 6
59
- PARALLELIZE_META_FETCH_THRESHOLD = 24
60
60
61
61
# The number of rows to read per batch. This is sized to generate 10MiB batches
62
62
# for rows about 1KiB in size.
93
93
PARQUET_ENCODING_RATIO_ESTIMATE_NUM_ROWS = 1024
94
94
95
95
96
+ @dataclass (frozen = True )
97
+ class _SampleInfo :
98
+ actual_bytes_per_row : int | None
99
+ estimated_bytes_per_row : int | None
100
+
101
+
96
102
# TODO(ekl) this is a workaround for a pyarrow serialization bug, where serializing a
97
103
# raw pyarrow file fragment causes S3 network calls.
98
104
class _SerializedFragment :
@@ -117,38 +123,6 @@ def _deserialize_fragments(
117
123
return [p .deserialize () for p in serialized_fragments ]
118
124
119
125
120
- class _ParquetFileFragmentMetaData :
121
- """Class to store metadata of a Parquet file fragment.
122
-
123
- This includes all attributes from `pyarrow.parquet.FileMetaData` except for `schema`,
124
- which is stored in `self.schema_pickled` as a pickled object from
125
- `cloudpickle.loads()`, used in deduplicating schemas across multiple fragments.
126
- """
127
-
128
- def __init__ (self , fragment_metadata : "pyarrow.parquet.FileMetaData" ):
129
- self .created_by = fragment_metadata .created_by
130
- self .format_version = fragment_metadata .format_version
131
- self .num_columns = fragment_metadata .num_columns
132
- self .num_row_groups = fragment_metadata .num_row_groups
133
- self .num_rows = fragment_metadata .num_rows
134
- self .serialized_size = fragment_metadata .serialized_size
135
- # This is a pickled schema object, to be set later with
136
- # `self.set_schema_pickled()`. To get the underlying schema, use
137
- # `cloudpickle.loads(self.schema_pickled)`.
138
- self .schema_pickled : bytes | None = None
139
-
140
- # Calculate the total byte size of the file fragment using the original
141
- # object, as it is not possible to access row groups from this class.
142
- self .total_byte_size = 0
143
- for row_group_idx in range (fragment_metadata .num_row_groups ):
144
- row_group_metadata = fragment_metadata .row_group (row_group_idx )
145
- self .total_byte_size += row_group_metadata .total_byte_size
146
-
147
- def set_schema_pickled (self , schema_pickled : bytes ) -> None :
148
- """Note: to get the underlying schema, use `cloudpickle.loads(self.schema_pickled)`."""
149
- self .schema_pickled = schema_pickled
150
-
151
-
152
126
# This retry helps when the upstream datasource is not able to handle
153
127
# overloaded read request or failed with some retriable failures.
154
128
# For example when reading data from HA hdfs service, hdfs might
@@ -213,7 +187,7 @@ def __init__( # noqa: PLR0912,PLR0915
213
187
arrow_parquet_args : dict [str , Any ] | None = None ,
214
188
_block_udf : Callable [[Block ], Block ] | None = None ,
215
189
filesystem : "pyarrow.fs.FileSystem" | None = None ,
216
- meta_provider : ParquetMetadataProvider = DefaultParquetMetadataProvider (),
190
+ meta_provider : ParquetMetadataProvider = ParquetMetadataProvider (),
217
191
partition_filter : PathPartitionFilter | None = None ,
218
192
shuffle : Literal ["files" ] | None = None ,
219
193
include_paths : bool = False ,
@@ -299,8 +273,7 @@ def __init__( # noqa: PLR0912,PLR0915
299
273
prefetch_remote_args = {}
300
274
if self ._local_scheduling :
301
275
prefetch_remote_args ["scheduling_strategy" ] = self ._local_scheduling
302
- raw_metadata = meta_provider .prefetch_file_metadata (pq_ds .fragments , ** prefetch_remote_args ) or []
303
- self ._metadata = self ._dedupe_metadata (raw_metadata )
276
+ self ._metadata = meta_provider .prefetch_file_metadata (pq_ds .fragments , ** prefetch_remote_args ) or []
304
277
except OSError as e :
305
278
_handle_read_os_error (e , paths )
306
279
except pa .ArrowInvalid as ex :
@@ -319,43 +292,15 @@ def __init__( # noqa: PLR0912,PLR0915
319
292
self ._columns = columns
320
293
self ._schema = schema
321
294
self ._arrow_parquet_args = arrow_parquet_args
322
- self ._encoding_ratio = self ._estimate_files_encoding_ratio ()
323
295
self ._file_metadata_shuffler = None
324
296
self ._include_paths = include_paths
325
297
self ._path_root = path_root
326
298
if shuffle == "files" :
327
299
self ._file_metadata_shuffler = np .random .default_rng ()
328
300
329
- def _dedupe_metadata (
330
- self ,
331
- raw_metadatas : list ["pyarrow.parquet.FileMetaData" ],
332
- ) -> list [_ParquetFileFragmentMetaData ]:
333
- """Deduplicate schemas to reduce memory usage.
334
-
335
- For datasets with a large number of columns, the FileMetaData
336
- (in particular the schema) can be very large. We can reduce the
337
- memory usage by only keeping unique schema objects across all
338
- file fragments. This method deduplicates the schemas and returns
339
- a list of `_ParquetFileFragmentMetaData` objects.
340
- """
341
- schema_to_id : dict [int , Any ] = {} # schema_id -> serialized_schema
342
- id_to_schema : dict [Any , bytes ] = {} # serialized_schema -> schema_id
343
- stripped_metadatas = []
344
- for fragment_metadata in raw_metadatas :
345
- stripped_md = _ParquetFileFragmentMetaData (fragment_metadata )
346
-
347
- schema_ser = cloudpickle .dumps (fragment_metadata .schema .to_arrow_schema ()) # type: ignore[no-untyped-call]
348
- if schema_ser not in schema_to_id :
349
- schema_id : int | None = len (schema_to_id )
350
- schema_to_id [schema_ser ] = schema_id
351
- id_to_schema [schema_id ] = schema_ser
352
- stripped_md .set_schema_pickled (schema_ser )
353
- else :
354
- schema_id = schema_to_id .get (schema_ser )
355
- existing_schema_ser = id_to_schema [schema_id ]
356
- stripped_md .set_schema_pickled (existing_schema_ser )
357
- stripped_metadatas .append (stripped_md )
358
- return stripped_metadatas
301
+ sample_infos = self ._sample_fragments ()
302
+ self ._encoding_ratio = _estimate_files_encoding_ratio (sample_infos )
303
+ self ._default_read_batch_size_rows = _estimate_default_read_batch_size_rows (sample_infos )
359
304
360
305
def estimate_inmemory_data_size (self ) -> int | None :
361
306
"""Return an estimate of the Parquet files encoding ratio.
@@ -414,25 +359,18 @@ def get_read_tasks(self, parallelism: int) -> list[ReadTask]:
414
359
if meta .size_bytes is not None :
415
360
meta .size_bytes = int (meta .size_bytes * self ._encoding_ratio )
416
361
417
- if meta .num_rows and meta .size_bytes :
418
- # Make sure the batches read are small enough to enable yielding of
419
- # output blocks incrementally during the read.
420
- row_size = meta .size_bytes / meta .num_rows
421
- # Make sure the row batch size is small enough that block splitting
422
- # is still effective.
423
- max_parquet_reader_row_batch_size_bytes = DataContext .get_current ().target_max_block_size // 10
424
- default_read_batch_size_rows = max (
425
- 1 ,
426
- min (
427
- PARQUET_READER_ROW_BATCH_SIZE ,
428
- max_parquet_reader_row_batch_size_bytes // row_size ,
429
- ),
430
- )
431
- else :
432
- default_read_batch_size_rows = PARQUET_READER_ROW_BATCH_SIZE
433
- block_udf , arrow_parquet_args , columns , schema , path_root , include_paths = (
362
+ (
363
+ block_udf ,
364
+ arrow_parquet_args ,
365
+ default_read_batch_size_rows ,
366
+ columns ,
367
+ schema ,
368
+ path_root ,
369
+ include_paths ,
370
+ ) = (
434
371
self ._block_udf ,
435
372
self ._arrow_parquet_args ,
373
+ self ._default_read_batch_size_rows ,
436
374
self ._columns ,
437
375
self ._schema ,
438
376
self ._path_root ,
@@ -456,14 +394,7 @@ def get_read_tasks(self, parallelism: int) -> list[ReadTask]:
456
394
457
395
return read_tasks
458
396
459
- def _estimate_files_encoding_ratio (self ) -> float :
460
- """Return an estimate of the Parquet files encoding ratio.
461
-
462
- To avoid OOMs, it is safer to return an over-estimate than an underestimate.
463
- """
464
- if not DataContext .get_current ().decoding_size_estimation :
465
- return PARQUET_ENCODING_RATIO_ESTIMATE_DEFAULT
466
-
397
+ def _sample_fragments (self ) -> list [_SampleInfo ]:
467
398
# Sample a few rows from Parquet files to estimate the encoding ratio.
468
399
# Launch tasks to sample multiple files remotely in parallel.
469
400
# Evenly distributed to sample N rows in i-th row group in i-th file.
@@ -495,11 +426,10 @@ def _estimate_files_encoding_ratio(self) -> float:
495
426
)
496
427
)
497
428
sample_bar = ProgressBar ("Parquet Files Sample" , len (futures ))
498
- sample_ratios = sample_bar .fetch_until_complete (futures )
429
+ sample_infos = sample_bar .fetch_until_complete (futures )
499
430
sample_bar .close () # type: ignore[no-untyped-call]
500
- ratio = np .mean (sample_ratios )
501
- _logger .debug (f"Estimated Parquet encoding ratio from sampling is { ratio } ." )
502
- return max (ratio , PARQUET_ENCODING_RATIO_ESTIMATE_LOWER_BOUND ) # type: ignore[no-any-return]
431
+
432
+ return sample_infos
503
433
504
434
def get_name (self ) -> str :
505
435
"""Return a human-readable name for this datasource.
@@ -577,33 +507,12 @@ def _read_fragments(
577
507
yield table
578
508
579
509
580
- def _fetch_metadata_serialization_wrapper (
581
- fragments : list [_SerializedFragment ],
582
- ) -> list ["pyarrow.parquet.FileMetaData" ]:
583
- fragments : list ["pyarrow._dataset.ParquetFileFragment" ] = _deserialize_fragments_with_retry (fragments ) # type: ignore[no-redef]
584
-
585
- return _fetch_metadata (fragments )
586
-
587
-
588
- def _fetch_metadata (
589
- fragments : list ["pyarrow.dataset.ParquetFileFragment" ],
590
- ) -> list ["pyarrow.parquet.FileMetaData" ]:
591
- fragment_metadata = []
592
- for f in fragments :
593
- try :
594
- fragment_metadata .append (f .metadata )
595
- except AttributeError :
596
- break
597
- return fragment_metadata
598
-
599
-
600
510
def _sample_fragment (
601
511
columns : list [str ] | None ,
602
512
schema : type | "pyarrow.lib.Schema" | None ,
603
513
file_fragment : _SerializedFragment ,
604
- ) -> float :
514
+ ) -> _SampleInfo :
605
515
# Sample the first rows batch from file fragment `serialized_fragment`.
606
- # Return the encoding ratio calculated from the sampled rows.
607
516
fragment = _deserialize_fragments_with_retry ([file_fragment ])[0 ]
608
517
609
518
# Only sample the first row group.
@@ -616,23 +525,57 @@ def _sample_fragment(
616
525
schema = schema ,
617
526
batch_size = batch_size ,
618
527
)
619
- # Use first batch in-memory size as ratio estimation.
528
+ # Use first batch in-memory size for estimation.
620
529
try :
621
530
batch = next (batches )
622
531
except StopIteration :
623
- ratio = PARQUET_ENCODING_RATIO_ESTIMATE_LOWER_BOUND
532
+ sample_data = _SampleInfo ( actual_bytes_per_row = None , estimated_bytes_per_row = None )
624
533
else :
625
534
if batch .num_rows > 0 :
626
- in_memory_size = batch .nbytes / batch .num_rows
627
535
metadata = fragment .metadata
628
536
total_size = 0
629
537
for idx in range (metadata .num_row_groups ):
630
538
total_size += metadata .row_group (idx ).total_byte_size
631
- file_size = total_size / metadata .num_rows
632
- ratio = in_memory_size / file_size
539
+ sample_data = _SampleInfo (
540
+ actual_bytes_per_row = batch .nbytes / batch .num_rows ,
541
+ estimated_bytes_per_row = total_size / metadata .num_rows ,
542
+ )
633
543
else :
634
- ratio = PARQUET_ENCODING_RATIO_ESTIMATE_LOWER_BOUND
635
- _logger .debug (
636
- f"Estimated Parquet encoding ratio is { ratio } for fragment { fragment } " f"with batch size { batch_size } ."
637
- )
638
- return ratio
544
+ sample_data = _SampleInfo (actual_bytes_per_row = None , estimated_bytes_per_row = None )
545
+ return sample_data
546
+
547
+
548
+ def _estimate_files_encoding_ratio (sample_infos : list [_SampleInfo ]) -> float :
549
+ """Return an estimate of the Parquet files encoding ratio.
550
+
551
+ To avoid OOMs, it is safer to return an over-estimate than an underestimate.
552
+ """
553
+ if not DataContext .get_current ().decoding_size_estimation :
554
+ return PARQUET_ENCODING_RATIO_ESTIMATE_DEFAULT
555
+
556
+ def compute_encoding_ratio (sample_info : _SampleInfo ) -> float :
557
+ if sample_info .actual_bytes_per_row is None or sample_info .estimated_bytes_per_row is None :
558
+ return PARQUET_ENCODING_RATIO_ESTIMATE_LOWER_BOUND
559
+ else :
560
+ return sample_info .actual_bytes_per_row / sample_info .estimated_bytes_per_row
561
+
562
+ ratio = np .mean (list (map (compute_encoding_ratio , sample_infos )))
563
+ _logger .debug (f"Estimated Parquet encoding ratio from sampling is { ratio } ." )
564
+ return max (ratio , PARQUET_ENCODING_RATIO_ESTIMATE_LOWER_BOUND ) # type: ignore[return-value]
565
+
566
+
567
+ def _estimate_default_read_batch_size_rows (sample_infos : list [_SampleInfo ]) -> int :
568
+ def compute_batch_size_rows (sample_info : _SampleInfo ) -> int :
569
+ if sample_info .actual_bytes_per_row is None :
570
+ return PARQUET_READER_ROW_BATCH_SIZE
571
+ else :
572
+ max_parquet_reader_row_batch_size_bytes = DataContext .get_current ().target_max_block_size // 10
573
+ return max (
574
+ 1 ,
575
+ min (
576
+ PARQUET_READER_ROW_BATCH_SIZE ,
577
+ max_parquet_reader_row_batch_size_bytes // sample_info .actual_bytes_per_row ,
578
+ ),
579
+ )
580
+
581
+ return np .mean (list (map (compute_batch_size_rows , sample_infos ))) # type: ignore[return-value]
0 commit comments