Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Datasets] Only enable blocks bundling when batch_size is set #29971

Merged
merged 6 commits into from
Nov 10, 2022

Conversation

c21
Copy link
Contributor

@c21 c21 commented Nov 3, 2022

Signed-off-by: Cheng Su scnju13@gmail.com

Why are these changes needed?

Before this PR, we always enable blocks bundling in map_batches, to bundle small blocks together for the given batch_size. This is good for batch prediction on GPU, but not good for CPU preprocessing with default batch size (4096, which is too large). So here we decide to disable blocks bundling by default, and only enable blocks bundling when user specifies batch_size. See this doc for full discussion.

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Signed-off-by: Cheng Su <scnju13@gmail.com>
Signed-off-by: Cheng Su <scnju13@gmail.com>
Signed-off-by: Cheng Su <scnju13@gmail.com>
@@ -82,7 +82,7 @@
OK_PREFIX = "✔️ "

# Default batch size for batch transformations.
DEFAULT_BATCH_SIZE = 4096
DEFAULT_BATCH_SIZE = "4096"
Copy link
Contributor

@clarkzinzow clarkzinzow Nov 8, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since the context-level default being a string that's eventually int-casted is a bit hacky, could we keep this as the 4096 int and have a "default" sentinel as the default arg in the .map_batches() signature, where we then do the check within the function?

def map_batches(
    ...,
    batch_size = "default",
    ...,
):
    ...

    if batch_size == "default":
        batch_size = DEFAULT_BATCH_SIZE
    elif batch_size is not None:
        if batch_size < 1:
            raise ValueError("Batch size cannot be negative or 0")
        # Enable blocks bundling when batch_size is specified by caller.
        target_block_size = batch_size

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clarkzinzow - yeah makes sense, I will make the change. Thanks.

@stephanie-wang
Copy link
Contributor

Could we also update the message about block bundling to reflect the new behavior?

Before:

Having to send 10 or more blocks to a single read->map_batches task to create a batch of size 4096, which may result in less transformation parallelism than expected. This may indicate that your blocks are too small and/or your batch size is too large, and you may want to decrease your read parallelism and/or decrease your batch size.

Suggested after:

`batch_size` is set to X, which requires sending X or more blocks to a single X task. This may reduce parallelism if you have fewer than X cores available. To ensure sufficient parallelism, decrease your map `batch_size` or use the default. You can also increase your block size by decreasing `parallelism` of the read stage.

I think decreasing read parallelism is probably secondary to modifying the batch size. Also, should we recommend a way to check whether the bundling happened in the message? If there is a specific line in Dataset.stats() to point the user to, that would be great.

Signed-off-by: Cheng Su <scnju13@gmail.com>
Comment on lines 518 to 523
f"`batch_size` is set to {target_size}, which reduces parallelism from "
f"{len(blocks)} to {len(block_bundles)}. If the performance is worse than "
"expected, this may indicate that batch size is too large or input block "
"size is too small. To reduce batch size, consider to decrease "
"`batch_size` or use the default in map_batches. To increase input block "
"size, consider to decrease `parallelism` in read."
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @stephanie-wang for suggestion. Please double check error message here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice, really like that you added the before and after parallelism! Suggested some minor grammar changes.

Comment on lines 520 to 523
"expected, this may indicate that batch size is too large or input block "
"size is too small. To reduce batch size, consider to decrease "
"`batch_size` or use the default in map_batches. To increase input block "
"size, consider to decrease `parallelism` in read."
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
"expected, this may indicate that batch size is too large or input block "
"size is too small. To reduce batch size, consider to decrease "
"`batch_size` or use the default in map_batches. To increase input block "
"size, consider to decrease `parallelism` in read."
"expected, this may indicate that the batch size is too large or the input block "
"size is too small. To reduce batch size, consider decreasing "
"`batch_size` or use the default in `map_batches`. To increase input block "
"size, consider decreasing `parallelism` in read."

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@stephanie-wang - updated.

@@ -323,7 +323,7 @@ def map_batches(
self,
fn: BatchUDF,
*,
batch_size: Optional[int] = DEFAULT_BATCH_SIZE,
batch_size: Optional[Union[int, Literal["default"]]] = "default",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you update the documentation of batch_size below as well?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jianoaix - could you suggest the specific documentation to be updated? It already mentions Defaults to 4096. Feel quite clear for end users.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So this introduced a new default value "default", right? We need to document what does that do to the semantics.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thought Defaults to 4096 is clear, but updated anyway. Could you help double check?

Copy link
Contributor

@clarkzinzow clarkzinzow left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same suggestions as Stephanie, after those are added, this LGTM!

Note to myself: I'll need to remember to tweak the batch_size section of the "Transforming Datasets" feature guide that I added in this PR: #29117

Signed-off-by: Cheng Su <scnju13@gmail.com>
@c21 c21 added the tests-ok The tagger certifies test failures are unrelated and assumes personal liability. label Nov 10, 2022
@clarkzinzow clarkzinzow merged commit acf996a into ray-project:master Nov 10, 2022
@c21 c21 deleted the batch-size branch November 15, 2022 00:36
amogkam added a commit that referenced this pull request Dec 9, 2022
…ch `Dataset` (#30960)

Signed-off-by: amogkam amogkamsetty@yahoo.com

In #29971, we disabled block coalescing by default and changed the default batch_size value for map_batches.

However, this same logic did not get carried over to DatasetPipeline, meaning DatasetPipeline.map_batches has block coalescing on by default.
WeichenXu123 pushed a commit to WeichenXu123/ray that referenced this pull request Dec 19, 2022
…oject#29971)

Before this PR, we always enable blocks bundling in map_batches, to bundle small blocks together for the given batch_size. This is good for batch prediction on GPU, but not good for CPU preprocessing with default batch size (4096, which is too large). So here we decide to disable blocks bundling by default, and only enable blocks bundling when user specifies batch_size.

Signed-off-by: Weichen Xu <weichen.xu@databricks.com>
WeichenXu123 pushed a commit to WeichenXu123/ray that referenced this pull request Dec 19, 2022
…ch `Dataset` (ray-project#30960)

Signed-off-by: amogkam amogkamsetty@yahoo.com

In ray-project#29971, we disabled block coalescing by default and changed the default batch_size value for map_batches.

However, this same logic did not get carried over to DatasetPipeline, meaning DatasetPipeline.map_batches has block coalescing on by default.

Signed-off-by: Weichen Xu <weichen.xu@databricks.com>
tamohannes pushed a commit to ju2ez/ray that referenced this pull request Jan 25, 2023
…ch `Dataset` (ray-project#30960)

Signed-off-by: amogkam amogkamsetty@yahoo.com

In ray-project#29971, we disabled block coalescing by default and changed the default batch_size value for map_batches.

However, this same logic did not get carried over to DatasetPipeline, meaning DatasetPipeline.map_batches has block coalescing on by default.

Signed-off-by: tmynn <hovhannes.tamoyan@gmail.com>
raulchen added a commit that referenced this pull request Aug 31, 2024
… with default batch_size (#47433)

When batch_size is not set, input blocks are will be not bundled up. 
Add a comment explaining this. 
See #29971 and
#47363 (comment)

Signed-off-by: Hao Chen <chenh1024@gmail.com>
ujjawal-khare pushed a commit to ujjawal-khare-27/ray that referenced this pull request Oct 12, 2024
… with default batch_size (ray-project#47433)

When batch_size is not set, input blocks are will be not bundled up.
Add a comment explaining this.
See ray-project#29971 and
ray-project#47363 (comment)

Signed-off-by: Hao Chen <chenh1024@gmail.com>
Signed-off-by: ujjawal-khare <ujjawal.khare@dream11.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
tests-ok The tagger certifies test failures are unrelated and assumes personal liability.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants