6868from ray .data .row import TableRow
6969from ray .data .aggregate import AggregateFn , Sum , Max , Min , Mean , Std
7070from ray .data .random_access_dataset import RandomAccessDataset
71- from ray .data .impl .table_block import VALUE_COL_NAME
7271from ray .data .impl .remote_fn import cached_remote_fn
7372from ray .data .impl .block_batching import batch_blocks , BatchType
7473from ray .data .impl .plan import ExecutionPlan , OneToOneStage , AllToAllStage
@@ -235,8 +234,8 @@ def map(
235234
236235 def transform (block : Block ) -> Iterable [Block ]:
237236 DatasetContext ._set_current (context )
238- output_buffer = BlockOutputBuffer (None , context .target_max_block_size )
239237 block = BlockAccessor .for_block (block )
238+ output_buffer = BlockOutputBuffer (None , context .target_max_block_size )
240239 for row in block .iter_rows ():
241240 output_buffer .add (fn (row ))
242241 if output_buffer .has_next ():
@@ -261,9 +260,6 @@ def map_batches(
261260 ) -> "Dataset[Any]" :
262261 """Apply the given function to batches of records of this dataset.
263262
264- The format of the data batch provided to ``fn`` can be controlled via the
265- ``batch_format`` argument, and the output of the UDF can be any batch type.
266-
267263 This is a blocking operation.
268264
269265 Examples:
@@ -310,9 +306,10 @@ def map_batches(
310306 blocks as batches. Defaults to a system-chosen batch size.
311307 compute: The compute strategy, either "tasks" (default) to use Ray
312308 tasks, or ActorPoolStrategy(min, max) to use an autoscaling actor pool.
313- batch_format: Specify "native" to use the native block format (promotes
314- tables to Pandas and tensors to NumPy), "pandas" to select
315- ``pandas.DataFrame``, or "pyarrow" to select `pyarrow.Table``.
309+ batch_format: Specify "native" to use the native block format
310+ (promotes Arrow to pandas), "pandas" to select
311+ ``pandas.DataFrame`` as the batch format,
312+ or "pyarrow" to select ``pyarrow.Table``.
316313 ray_remote_args: Additional resource requirements to request from
317314 ray (e.g., num_gpus=1 to request GPUs for the map tasks).
318315 """
@@ -341,7 +338,9 @@ def transform(block: Block) -> Iterable[Block]:
341338 # bug where we include the entire base view on serialization.
342339 view = block .slice (start , end , copy = batch_size is not None )
343340 if batch_format == "native" :
344- view = BlockAccessor .for_block (view ).to_native ()
341+ # Always promote Arrow blocks to pandas for consistency.
342+ if isinstance (view , pa .Table ) or isinstance (view , bytes ):
343+ view = BlockAccessor .for_block (view ).to_pandas ()
345344 elif batch_format == "pandas" :
346345 view = BlockAccessor .for_block (view ).to_pandas ()
347346 elif batch_format == "pyarrow" :
@@ -356,7 +355,6 @@ def transform(block: Block) -> Iterable[Block]:
356355 if not (
357356 isinstance (applied , list )
358357 or isinstance (applied , pa .Table )
359- or isinstance (applied , np .ndarray )
360358 or isinstance (applied , pd .core .frame .DataFrame )
361359 ):
362360 raise ValueError (
@@ -366,7 +364,7 @@ def transform(block: Block) -> Iterable[Block]:
366364 "The return type must be either list, "
367365 "pandas.DataFrame, or pyarrow.Table"
368366 )
369- output_buffer .add_batch (applied )
367+ output_buffer .add_block (applied )
370368 if output_buffer .has_next ():
371369 yield output_buffer .next ()
372370
@@ -703,8 +701,6 @@ def process_batch(batch):
703701 )
704702 if isinstance (batch , pd .DataFrame ):
705703 return batch .sample (frac = fraction )
706- if isinstance (batch , np .ndarray ):
707- return np .array ([row for row in batch if random .random () <= fraction ])
708704 raise ValueError (f"Unsupported batch type: { type (batch )} " )
709705
710706 return self .map_batches (process_batch )
@@ -2075,7 +2071,7 @@ def write_numpy(
20752071 self ,
20762072 path : str ,
20772073 * ,
2078- column : str = VALUE_COL_NAME ,
2074+ column : str = "value" ,
20792075 filesystem : Optional ["pyarrow.fs.FileSystem" ] = None ,
20802076 try_create_dir : bool = True ,
20812077 arrow_open_stream_args : Optional [Dict [str , Any ]] = None ,
@@ -2103,8 +2099,7 @@ def write_numpy(
21032099 path: The path to the destination root directory, where npy
21042100 files will be written to.
21052101 column: The name of the table column that contains the tensor to
2106- be written. The default is ``"__value__"``, the column name that
2107- Datasets uses for storing tensors in single-column tables.
2102+ be written. This defaults to "value".
21082103 filesystem: The filesystem implementation to write to.
21092104 try_create_dir: Try to create all directories in destination path
21102105 if True. Does nothing if all directories already exist.
@@ -2251,10 +2246,10 @@ def iter_batches(
22512246 current block during the scan.
22522247 batch_size: Record batch size, or None to let the system pick.
22532248 batch_format: The format in which to return each batch.
2254- Specify "native" to use the native block format (promoting
2255- tables to Pandas and tensors to NumPy ), "pandas" to select
2256- ``pandas.DataFrame``, or "pyarrow" to select ``pyarrow.Table``. Default
2257- is "native".
2249+ Specify "native" to use the current block format (promoting
2250+ Arrow to pandas automatically ), "pandas" to
2251+ select ``pandas.DataFrame`` or "pyarrow" to select
2252+ ``pyarrow.Table``. Default is "native".
22582253 drop_last: Whether to drop the last batch if it's incomplete.
22592254
22602255 Returns:
@@ -2776,9 +2771,8 @@ def to_numpy_refs(
27762771 Time complexity: O(dataset size / parallelism)
27772772
27782773 Args:
2779- column: The name of the column to convert to numpy, or None to specify the
2780- entire row. If not specified for Arrow or Pandas blocks, each returned
2781- future will represent a dict of column ndarrays.
2774+ column: The name of the column to convert to numpy, or None to
2775+ specify the entire row. Required for Arrow tables.
27822776
27832777 Returns:
27842778 A list of remote NumPy ndarrays created from this dataset.
0 commit comments