Skip to content

Conversation

owenowenisme
Copy link
Member

@owenowenisme owenowenisme commented Oct 9, 2025

Why are these changes needed?

Add ApproximateQuantile aggregator to Ray Data using DataSketches KLL.

Reason:
• Enables efficient support for the summary API.
• More scalable than exact Quantile on large datasets.

Note:
• DataSketches is not added as a Ray dependency; if missing, users are prompted to install it.


Here's a simple test to show the efficiency difference between ApproximateQuantile and Quantile

import ray
import ray.data
import time

ray.init(num_cpus=16)
from ray.data.aggregate import ApproximateQuantile, Quantile

ds = ray.data.range(10**8)
start_time = time.time()

print(ds.aggregate(ApproximateQuantile(on="id", quantiles=[0.5])))
print(f"Time taken ApproximateQuantile: {time.time() - start_time} seconds")

ds = ray.data.range(10**8)
start_time = time.time()

print(ds.aggregate(Quantile(on="id", q=0.5)))
print(f"Time taken Quantile: {time.time() - start_time} seconds")

In this run with 1e8 rows, the approximate median returned 49,979,428.0 in ~12.46s, while the exact Quantile returned 49,999,999.5 in ~163.33s. The difference reflects the sketch’s accuracy trade-off for significant speed and scalability gains.

When k=800 (the default), we are guaranteed to have the error rate < 0.45% , in this test our error rate is (49,999,999.5-49,979,428.0)/49,999,999.5= 0.00041143 = 0.041143% which is < 0.45% , but we get the approximate median 13.11x faster.

{'approx_quantile(id)': [49979428.0]}
Time taken ApproximateQuantile: 12.457247257232666 seconds
{'quantile(id)': 49999999.5}
Time taken Quantile: 163.32705521583557 seconds

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 pre-commit jobs to lint the changes in this PR. (pre-commit setup)
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • 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 :(

@owenowenisme owenowenisme force-pushed the data/add-approximate-quantile-to-aggregrator branch from e0584b6 to 45381b1 Compare October 9, 2025 13:20
@owenowenisme owenowenisme added the go add ONLY when ready to merge, run all tests label Oct 9, 2025
Signed-off-by: You-Cheng Lin (Owen) <mses010108@gmail.com>
@owenowenisme owenowenisme force-pushed the data/add-approximate-quantile-to-aggregrator branch from 45381b1 to 024f199 Compare October 9, 2025 23:55
Signed-off-by: You-Cheng Lin (Owen) <mses010108@gmail.com>
Signed-off-by: You-Cheng Lin (Owen) <mses010108@gmail.com>
Signed-off-by: You-Cheng Lin (Owen) <mses010108@gmail.com>
Signed-off-by: You-Cheng Lin (Owen) <mses010108@gmail.com>
@owenowenisme owenowenisme marked this pull request as ready for review October 10, 2025 08:27
@owenowenisme owenowenisme requested a review from a team as a code owner October 10, 2025 08:27
cursor[bot]

This comment was marked as outdated.

Signed-off-by: You-Cheng Lin <106612301+owenowenisme@users.noreply.github.com>
Signed-off-by: You-Cheng Lin <106612301+owenowenisme@users.noreply.github.com>
cursor[bot]

This comment was marked as outdated.

@ray-gardener ray-gardener bot added the data Ray Data-related issues label Oct 10, 2025
"""
self._require_datasketches()
self._quantiles = quantiles
self._k = k
Copy link
Contributor

Choose a reason for hiding this comment

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

instead of k, let's use capacity_per_level

Copy link
Member Author

@owenowenisme owenowenisme Oct 11, 2025

Choose a reason for hiding this comment

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

capacity_per_level does not feel accurate to me, I think maybe we don't need to hide the detail of k, since user will need to see the doc from datasketches anyway.

I added link to k params description to guide users to the doc for more info.

sketch = self.zero(self._k)
for value in column:
# we ignore nulls here
if value.as_py() is not None:
Copy link
Contributor

Choose a reason for hiding this comment

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

do we need an as_py() conversion here? What type is this value?

Copy link
Member Author

@owenowenisme owenowenisme Oct 11, 2025

Choose a reason for hiding this comment

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

This is because we will get this error when the value is none.

 def test_approximate_quantile_ignores_nulls(self, ray_start_regular_shared_2_cpus):
        data = [
            {"id": 1, "value": 5.0},
            {"id": 2, "value": None},
            {"id": 3, "value": 15.0},
            {"id": 4, "value": None},
            {"id": 5, "value": 25.0},
        ]
        ds = ray.data.from_items(data)

        result = ds.aggregate(ApproximateQuantile(on="value", quantiles=[0.5]))
        assert result["approx_quantile(value)"] == [15.0]
TypeError: float() argument must be a string or a number, not 'pyarrow.lib.NullScalar'

Signed-off-by: You-Cheng Lin (Owen) <mses010108@gmail.com>
Signed-off-by: You-Cheng Lin (Owen) <mses010108@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

data Ray Data-related issues go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants