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] Always launch one task for read_sql #48923

Merged
merged 14 commits into from
Dec 3, 2024
91 changes: 5 additions & 86 deletions python/ray/data/_internal/datasource/sql_datasource.py
Original file line number Diff line number Diff line change
@@ -1,8 +1,7 @@
import math
from contextlib import contextmanager
from typing import Any, Callable, Iterable, Iterator, List, Optional

from ray.data.block import Block, BlockAccessor, BlockMetadata
from ray.data.block import Block, BlockMetadata
from ray.data.datasource.datasource import Datasource, ReadTask

Connection = Any # A Python DB API2-compliant `Connection` object.
Expand Down Expand Up @@ -72,99 +71,19 @@ def _connect(connection_factory: Callable[[], Connection]) -> Iterator[Cursor]:


class SQLDatasource(Datasource):

NUM_SAMPLE_ROWS = 100
MIN_ROWS_PER_READ_TASK = 50

def __init__(self, sql: str, connection_factory: Callable[[], Connection]):
self.sql = sql
self.connection_factory = connection_factory

def estimate_inmemory_data_size(self) -> Optional[int]:
pass
return None

def get_read_tasks(self, parallelism: int) -> List[ReadTask]:
def fallback_read_fn() -> Iterable[Block]:
with _connect(self.connection_factory) as cursor:
cursor.execute(self.sql)
block = _cursor_to_block(cursor)
return [block]

# If `parallelism` is 1, directly fetch all rows. This avoids unnecessary
# queries to fetch a sample block and compute the total number of rows.
if parallelism == 1:
metadata = BlockMetadata(None, None, None, None, None)
return [ReadTask(fallback_read_fn, metadata)]

# Databases like DB2, Oracle, and MS SQL Server don't support `LIMIT`.
try:
with _connect(self.connection_factory) as cursor:
cursor.execute(f"SELECT * FROM ({self.sql}) as T LIMIT 1 OFFSET 0")
is_limit_supported = True
except Exception:
is_limit_supported = False

if not is_limit_supported:
metadata = BlockMetadata(None, None, None, None, None)
return [ReadTask(fallback_read_fn, metadata)]

num_rows_total = self._get_num_rows()

if num_rows_total == 0:
return []

parallelism = min(
parallelism, math.ceil(num_rows_total / self.MIN_ROWS_PER_READ_TASK)
)
num_rows_per_block = num_rows_total // parallelism
num_blocks_with_extra_row = num_rows_total % parallelism

sample_block_accessor = BlockAccessor.for_block(self._get_sample_block())
estimated_size_bytes_per_row = math.ceil(
sample_block_accessor.size_bytes() / sample_block_accessor.num_rows()
)
sample_block_schema = sample_block_accessor.schema()

tasks = []
offset = 0
for i in range(parallelism):
num_rows = num_rows_per_block
if i < num_blocks_with_extra_row:
num_rows += 1

read_fn = self._create_read_fn(num_rows, offset)
metadata = BlockMetadata(
num_rows,
estimated_size_bytes_per_row * num_rows,
sample_block_schema,
None,
None,
)
tasks.append(ReadTask(read_fn, metadata))

offset += num_rows

return tasks

def _get_num_rows(self) -> int:
with _connect(self.connection_factory) as cursor:
cursor.execute(f"SELECT COUNT(*) FROM ({self.sql}) as T")
return cursor.fetchone()[0]

def _get_sample_block(self) -> Block:
with _connect(self.connection_factory) as cursor:
cursor.execute(
f"SELECT * FROM ({self.sql}) as T LIMIT {self.NUM_SAMPLE_ROWS}"
)
return _cursor_to_block(cursor)

def _create_read_fn(self, num_rows: int, offset: int):
def read_fn() -> Iterable[Block]:
with _connect(self.connection_factory) as cursor:
cursor.execute(
f"SELECT * FROM ({self.sql}) as T LIMIT {num_rows} OFFSET {offset}"
)
cursor.execute(self.sql)
block = _cursor_to_block(cursor)
return [block]

return read_fn
metadata = BlockMetadata(None, None, None, None, None)
return [ReadTask(read_fn, metadata)]
6 changes: 6 additions & 0 deletions python/ray/data/read_api.py
Original file line number Diff line number Diff line change
Expand Up @@ -2158,6 +2158,12 @@ def create_connection():
Returns:
A :class:`Dataset` containing the queried data.
"""
if parallelism != -1 and parallelism != 1:
raise ValueError(
"To ensure correctness, 'read_sql' always launches one task. The "
"'parallelism' argument you specified can't be used."
)
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe just raise an error. warning is implicit.

Copy link
Contributor

Choose a reason for hiding this comment

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

To understand, if DB table is huge (1B rows or more), will this be single threaded ingest?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yeah. Many DBAPI implementations don't support multithreading

Copy link
Contributor

Choose a reason for hiding this comment

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

If I understand this right, we may end up with very slow ingest with just 1 task for DBs and also OOM kills. While for files, we are able to do support parallel ingests in a scaled out fashion.

Copy link
Member Author

Choose a reason for hiding this comment

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

That's right.

What do we do as an alternative that's both scalable and correct? Many OFFSET implementations require scanning the entire database. So, OFFSET and LIMIT often perform the same or worse than a single task that reads the entire database.


datasource = SQLDatasource(sql=sql, connection_factory=connection_factory)
return read_datasource(
datasource,
Expand Down
11 changes: 8 additions & 3 deletions python/ray/data/tests/test_sql.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,14 @@ def temp_database_fixture() -> Generator[str, None, None]:
yield file.name


@pytest.mark.parametrize("parallelism", [-1, 1])
def test_read_sql(temp_database: str, parallelism: int):
def test_read_sql_with_parallelism_warns(temp_database):
with pytest.raises(ValueError):
ray.data.read_sql(
"SELECT * FROM movie", lambda: sqlite3.connect(temp_database), parallelism=2
)


def test_read_sql(temp_database: str):
connection = sqlite3.connect(temp_database)
connection.execute("CREATE TABLE movie(title, year, score)")
expected_values = [
Expand All @@ -37,7 +43,6 @@ def test_read_sql(temp_database: str, parallelism: int):
dataset = ray.data.read_sql(
"SELECT * FROM movie",
lambda: sqlite3.connect(temp_database),
override_num_blocks=parallelism,
)
actual_values = [tuple(record.values()) for record in dataset.take_all()]

Expand Down
Loading