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

[Data] Add read_clickhouse API to read ClickHouse Dataset #48817

Open
wants to merge 9 commits into
base: master
Choose a base branch
from

Conversation

jecsand838
Copy link

Why are these changes needed?

Greetings from Elastiflow!

This PR introduces a new ClickHouseDatasource connector for Ray, which provides a convenient way to read data from ClickHouse into Ray Datasets. The ClickHouseDatasource is particularly useful for users who are working with large datasets stored in ClickHouse and want to leverage Ray's distributed computing capabilities for AI and ML use-cases. We found this functionality useful while evaluating ML technologies and wanted to contribute this back.

Key Features and Benefits:

  1. Seamless Integration: The ClickHouseDatasource allows for seamless integration of ClickHouse data into Ray workflows, enabling users to easily access their data and apply Ray's powerful parallel computation.
  2. Custom Query Support: Users can specify custom columns, filters, and orderings, allowing for flexible query generation directly from the Ray interface, which helps in reading only the necessary data, thereby improving performance.
  3. User-Friendly API: The connector abstracts the complexity of setting up and querying ClickHouse, providing a simple API that allows users to focus on data analysis rather than data extraction.

Tested locally with a ClickHouse table containing ~12m records.

Screenshot 2024-11-20 at 3 52 42 AM

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 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 :(

@jcotant1 jcotant1 added the data Ray Data-related issues label Nov 20, 2024
Comment on lines +22 to +26
columns: Optional[List[str]] = None,
filters: Optional[Dict[str, Tuple[str, Any]]] = None,
order_by: Optional[Tuple[List[str], bool]] = None,
client_settings: Optional[Dict[str, Any]] = None,
client_kwargs: Optional[Dict[str, Any]] = None,
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's make all optional args as kwargs

Comment on lines +20 to +21
entity: str,
dsn: str,
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 please give an example of the DSN?


def __init__(
self,
entity: str,
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: I'd suggest we employ more common term like table (and in py-doc expand that this could also be a view of one)

Comment on lines +37 to +42
filters: Optional fields and values mapping to use to filter the data via
WHERE clause. The value should be a tuple where the first element is
one of ('is', 'not', 'less', 'greater') and the second
element is the value to filter by. The default operator
is 'is'. Only strings, ints, floats, booleans,
and None are allowed as values.
Copy link
Contributor

Choose a reason for hiding this comment

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

IIUC this is requiring predicate in DNF format, let's call it out explicitly and add an example to help with understanding of it.

Also let's add a link to the page of parameters to ClickHouse explaining these in more details

f"Unsupported operator '{op}' for filter on '{column}'. "
f"Defaulting to 'is'"
)
op = "is"
Copy link
Contributor

Choose a reason for hiding this comment

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

Same as below

Comment on lines +98 to +113
if value is None:
operator = validate_non_numeric_ops(key, operator)
if operator == "is":
filter_conditions.append(f"{key} IS NULL")
elif operator == "not":
filter_conditions.append(f"{key} IS NOT NULL")
elif isinstance(value, str):
operator = validate_non_numeric_ops(key, operator)
filter_conditions.append(f"{key} {ops[operator]} '{value}'")
elif isinstance(value, bool):
operator = validate_non_numeric_ops(key, operator)
filter_conditions.append(
f"{key} {ops[operator]} {str(value).lower()}"
)
elif isinstance(value, (int, float)):
filter_conditions.append(f"{key} {ops[operator]} {value}")
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's split up value conversion from filter composition to avoid duplication

op = "is"
return op

ops = {"is": "=", "not": "!=", "less": "<", "greater": ">"}
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's use Python operators so that we're not reinventing the wheel here

Comment on lines +171 to +173
# Fetch the fragments from the ClickHouse client
with self._client.query_arrow_stream(self._query) as stream:
record_batches = list(stream) # Collect all record batches
Copy link
Contributor

Choose a reason for hiding this comment

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

So actual reading of the data needs to be performed inside the read task (that's currently consolidated in _read_fn)

Copy link
Contributor

Choose a reason for hiding this comment

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

I'd also recommend you to take a look at SQLDatasource to see how it could be structured in a way compatible with Ray read APIs

Copy link
Author

Choose a reason for hiding this comment

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

Ah, I see what you mean. I'll make that change. Thank you for pointing that out!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
data Ray Data-related issues
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants