-
Notifications
You must be signed in to change notification settings - Fork 5.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Data] Add
read_lance
API to read Lance Dataset (#45106)
This PR adds the capability to load a LanceDB dataset into a Ray Dataset. Signed-off-by: Brent Bain <[email protected]> Signed-off-by: Cheng Su <[email protected]> Co-authored-by: Brent Bain <[email protected]>
- Loading branch information
1 parent
a6569d4
commit 24bf8a1
Showing
8 changed files
with
256 additions
and
0 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
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
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,83 @@ | ||
import logging | ||
from typing import TYPE_CHECKING, Dict, Iterator, List, Optional | ||
|
||
import numpy as np | ||
|
||
from ray.data._internal.util import _check_import | ||
from ray.data.block import BlockMetadata | ||
from ray.data.datasource.datasource import Datasource, ReadTask | ||
from ray.util.annotations import DeveloperAPI | ||
|
||
if TYPE_CHECKING: | ||
import pyarrow | ||
|
||
|
||
logger = logging.getLogger(__name__) | ||
|
||
|
||
@DeveloperAPI | ||
class LanceDatasource(Datasource): | ||
"""Lance datasource, for reading Lance dataset.""" | ||
|
||
def __init__( | ||
self, | ||
uri: str, | ||
columns: Optional[List[str]] = None, | ||
filter: Optional[str] = None, | ||
storage_options: Optional[Dict[str, str]] = None, | ||
): | ||
_check_import(self, module="lance", package="pylance") | ||
|
||
import lance | ||
|
||
self.uri = uri | ||
self.columns = columns | ||
self.filter = filter | ||
self.storage_options = storage_options | ||
|
||
self.lance_ds = lance.dataset(uri=uri, storage_options=storage_options) | ||
self.fragments = self.lance_ds.get_fragments() | ||
|
||
def get_read_tasks(self, parallelism: int) -> List[ReadTask]: | ||
read_tasks = [] | ||
for fragments in np.array_split(self.fragments, parallelism): | ||
if len(fragments) <= 0: | ||
continue | ||
|
||
num_rows = sum([f.count_rows() for f in fragments]) | ||
input_files = [ | ||
data_file.path() for f in fragments for data_file in f.data_files() | ||
] | ||
|
||
# TODO(chengsu): Take column projection into consideration for schema. | ||
metadata = BlockMetadata( | ||
num_rows=num_rows, | ||
schema=fragments[0].schema, | ||
input_files=input_files, | ||
size_bytes=None, | ||
exec_stats=None, | ||
) | ||
columns = self.columns | ||
row_filter = self.filter | ||
|
||
read_task = ReadTask( | ||
lambda f=fragments: _read_fragments(f, columns, row_filter), | ||
metadata, | ||
) | ||
read_tasks.append(read_task) | ||
|
||
return read_tasks | ||
|
||
def estimate_inmemory_data_size(self) -> Optional[int]: | ||
# TODO(chengsu): Add memory size estimation to improve auto-tune of parallelism. | ||
return None | ||
|
||
|
||
def _read_fragments(fragments, columns, row_filter) -> Iterator["pyarrow.Table"]: | ||
"""Read Lance fragments in batches.""" | ||
import pyarrow | ||
|
||
for fragment in fragments: | ||
batches = fragment.to_batches(columns=columns, filter=row_filter) | ||
for batch in batches: | ||
yield pyarrow.Table.from_batches([batch]) |
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,89 @@ | ||
import os | ||
|
||
import lance | ||
import pyarrow as pa | ||
import pytest | ||
from pkg_resources import parse_version | ||
from pytest_lazyfixture import lazy_fixture | ||
|
||
import ray | ||
from ray._private.utils import _get_pyarrow_version | ||
from ray.data.datasource.path_util import _unwrap_protocol | ||
|
||
|
||
@pytest.mark.parametrize( | ||
"fs,data_path", | ||
[ | ||
(None, lazy_fixture("local_path")), | ||
(lazy_fixture("local_fs"), lazy_fixture("local_path")), | ||
(lazy_fixture("s3_fs"), lazy_fixture("s3_path")), | ||
( | ||
lazy_fixture("s3_fs_with_space"), | ||
lazy_fixture("s3_path_with_space"), | ||
), # Path contains space. | ||
( | ||
lazy_fixture("s3_fs_with_anonymous_crendential"), | ||
lazy_fixture("s3_path_with_anonymous_crendential"), | ||
), | ||
], | ||
) | ||
def test_lance_read_basic(fs, data_path): | ||
# NOTE: Lance only works with PyArrow 12 or above. | ||
pyarrow_version = _get_pyarrow_version() | ||
if pyarrow_version is not None: | ||
pyarrow_version = parse_version(pyarrow_version) | ||
if pyarrow_version is not None and pyarrow_version < parse_version("12.0.0"): | ||
return | ||
|
||
df1 = pa.table({"one": [2, 1, 3, 4, 6, 5], "two": ["b", "a", "c", "e", "g", "f"]}) | ||
setup_data_path = _unwrap_protocol(data_path) | ||
path = os.path.join(setup_data_path, "test.lance") | ||
lance.write_dataset(df1, path) | ||
|
||
ds_lance = lance.dataset(path) | ||
df2 = pa.table( | ||
{ | ||
"one": [1, 2, 3, 4, 5, 6], | ||
"three": [4, 5, 8, 9, 12, 13], | ||
"four": ["u", "v", "w", "x", "y", "z"], | ||
} | ||
) | ||
ds_lance.merge(df2, "one") | ||
|
||
ds = ray.data.read_lance(path) | ||
|
||
# Test metadata-only ops. | ||
assert ds.count() == 6 | ||
assert ds.schema() is not None | ||
|
||
assert ( | ||
" ".join(str(ds).split()) | ||
== "Dataset( num_rows=6, schema={one: int64, two: string, three: int64, four: string} )" # noqa: E501 | ||
), ds | ||
assert ( | ||
" ".join(repr(ds).split()) | ||
== "Dataset( num_rows=6, schema={one: int64, two: string, three: int64, four: string} )" # noqa: E501 | ||
), ds | ||
|
||
# Test read. | ||
values = [[s["one"], s["two"]] for s in ds.take_all()] | ||
assert sorted(values) == [ | ||
[1, "a"], | ||
[2, "b"], | ||
[3, "c"], | ||
[4, "e"], | ||
[5, "f"], | ||
[6, "g"], | ||
] | ||
|
||
# Test column projection. | ||
ds = ray.data.read_lance(path, columns=["one"]) | ||
values = [s["one"] for s in ds.take()] | ||
assert sorted(values) == [1, 2, 3, 4, 5, 6] | ||
assert ds.schema().names == ["one", "two", "three", "four"] | ||
|
||
|
||
if __name__ == "__main__": | ||
import sys | ||
|
||
sys.exit(pytest.main(["-v", __file__])) |
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 |
---|---|---|
|
@@ -13,3 +13,4 @@ google-cloud-bigquery-storage==2.24.0 | |
google-api-core==1.34.0 | ||
webdataset | ||
raydp==1.7.0b20231020.dev0 | ||
pylance |