forked from ray-project/ray
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Datasets] Change
map_batches
to fetch input blocks on-demand (ray-…
…project#29289) Signed-off-by: Cheng Su [email protected] This is the fix the issue we found during AIR benchmark. When the map_batches have multiple input blocks (it can happen when dynamic block splitting is enabled by default, or multiple input blocks are coalesced together), previously we always fetch and buffer all input blocks before producing first batch. This is bad especially for dynamic block splitting, because it essentially buffers all split blocks again in memory. So in this PR, change map_batches to fetch and buffer input blocks on-demand, i.e. only fetch blocks when needed to construct the next required batch. Signed-off-by: Weichen Xu <[email protected]>
- Loading branch information
1 parent
cbce66d
commit 05c2d34
Showing
2 changed files
with
88 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
import numpy as np | ||
import pandas as pd | ||
import pytest | ||
|
||
import ray | ||
from ray.data.block import BlockMetadata | ||
from ray.data.datasource import Datasource | ||
from ray.data.datasource.datasource import ReadTask, Reader | ||
|
||
from ray.tests.conftest import * # noqa | ||
|
||
|
||
def test_read_large_data(ray_start_cluster): | ||
# Test 20G input with single task | ||
num_batch = 20 | ||
ctx = ray.data.context.DatasetContext.get_current() | ||
block_splitting_enabled = ctx.block_splitting_enabled | ||
ctx.block_splitting_enabled = True | ||
|
||
try: | ||
cluster = ray_start_cluster | ||
cluster.add_node(num_cpus=1) | ||
|
||
ray.init(cluster.address) | ||
|
||
# Data source generates multiple 1G random bytes data | ||
class LargeBytesDatasource(Datasource): | ||
def create_reader(self, **read_args): | ||
return LargeBytesReader() | ||
|
||
class LargeBytesReader(Reader): | ||
def estimate_inmemory_data_size(self): | ||
return None | ||
|
||
def get_read_tasks(self, parallelism: int): | ||
def _1g_batches_generator(): | ||
for _ in range(num_batch): | ||
yield pd.DataFrame( | ||
{"one": [np.random.bytes(1024 * 1024 * 1024)]} | ||
) | ||
|
||
return parallelism * [ | ||
ReadTask( | ||
lambda: _1g_batches_generator(), | ||
BlockMetadata( | ||
num_rows=None, | ||
size_bytes=None, | ||
schema=None, | ||
input_files=None, | ||
exec_stats=None, | ||
), | ||
) | ||
] | ||
|
||
def foo(batch): | ||
return pd.DataFrame({"one": [1]}) | ||
|
||
ds = ray.data.read_datasource( | ||
LargeBytesDatasource(), | ||
parallelism=1, | ||
) | ||
|
||
ds = ds.map_batches(foo, batch_size=None) | ||
assert ds.count() == num_batch | ||
finally: | ||
ctx.block_splitting_enabled = block_splitting_enabled | ||
|
||
|
||
if __name__ == "__main__": | ||
import sys | ||
|
||
sys.exit(pytest.main(["-v", __file__])) |