-
Notifications
You must be signed in to change notification settings - Fork 3.1k
Add lance format support #7913
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
Merged
+296
−7
Merged
Add lance format support #7913
Changes from 12 commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
49305e3
add lance as supported format
eddyxu 16ad14d
add more test
eddyxu ce896c8
remove debug prints
eddyxu 1c2ac82
re
eddyxu bfc5169
simplify
eddyxu 0847e17
do not need configure file
eddyxu dda3fb8
push
eddyxu a3ec2a3
claude code
eddyxu 20a4e1d
pass tests
eddyxu 0eb085c
handle streaming uri
eddyxu 1f075a5
style
eddyxu 69da83a
support {local, streaming} x {dataset, single files}
lhoestq a50b0c2
added columns pushdown to LanceFileReader
lhoestq File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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 hidden or 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 hidden or 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
Empty file.
This file contains hidden or 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,163 @@ | ||
| import re | ||
| from dataclasses import dataclass | ||
| from pathlib import Path | ||
| from typing import TYPE_CHECKING, Dict, List, Optional | ||
|
|
||
| import pyarrow as pa | ||
| from huggingface_hub import HfApi | ||
|
|
||
| import datasets | ||
| from datasets.builder import Key | ||
| from datasets.table import table_cast | ||
| from datasets.utils.file_utils import is_local_path | ||
|
|
||
|
|
||
| if TYPE_CHECKING: | ||
| import lance | ||
| import lance.file | ||
|
|
||
| logger = datasets.utils.logging.get_logger(__name__) | ||
|
|
||
|
|
||
| @dataclass | ||
| class LanceConfig(datasets.BuilderConfig): | ||
| """ | ||
| BuilderConfig for Lance format. | ||
|
|
||
| Args: | ||
| features: (`Features`, *optional*): | ||
| Cast the data to `features`. | ||
| columns: (`List[str]`, *optional*): | ||
| List of columns to load, the other ones are ignored. | ||
| batch_size: (`int`, *optional*): | ||
| Size of the RecordBatches to iterate on. Default to 256. | ||
| token: (`str`, *optional*): | ||
| Optional HF token to use to download datasets. | ||
| """ | ||
|
|
||
| features: Optional[datasets.Features] = None | ||
| columns: Optional[List[str]] = None | ||
| batch_size: Optional[int] = 256 | ||
| token: Optional[str] = None | ||
|
|
||
|
|
||
| def resolve_dataset_uris(files: List[str]) -> Dict[str, List[str]]: | ||
| dataset_uris = set() | ||
| for file_path in files: | ||
| path = Path(file_path) | ||
| if path.parent.name in {"_transactions", "_indices", "_versions"}: | ||
| dataset_root = path.parent.parent | ||
| dataset_uris.add(str(dataset_root)) | ||
| return list(dataset_uris) | ||
|
|
||
|
|
||
| def _fix_hf_uri(uri: str) -> str: | ||
| # replace the revision tag from hf uri | ||
| if "@" in uri: | ||
| matched = re.match(r"(hf://.+?)(@[0-9a-f]+)(/.*)", uri) | ||
| if matched: | ||
| uri = matched.group(1) + matched.group(3) | ||
| return uri | ||
|
|
||
|
|
||
| def _fix_local_version_file(uri: str) -> str: | ||
| # replace symlinks with real files for _version | ||
| if "/_versions/" in uri and is_local_path(uri): | ||
| path = Path(uri) | ||
| if path.is_symlink(): | ||
| data = path.read_bytes() | ||
| path.unlink() | ||
| path.write_bytes(data) | ||
| return uri | ||
|
|
||
|
|
||
| class Lance(datasets.ArrowBasedBuilder): | ||
| BUILDER_CONFIG_CLASS = LanceConfig | ||
| METADATA_EXTENSIONS = [".idx", ".txn", ".manifest"] | ||
|
|
||
| def _info(self): | ||
| return datasets.DatasetInfo(features=self.config.features) | ||
|
|
||
| def _split_generators(self, dl_manager): | ||
| import lance | ||
| import lance.file | ||
|
|
||
| if not self.config.data_files: | ||
| raise ValueError(f"At least one data file must be specified, but got data_files={self.config.data_files}") | ||
| if self.repo_id: | ||
| api = HfApi(**dl_manager.download_config.storage_options["hf"]) | ||
| dataset_sha = api.dataset_info(self.repo_id).sha | ||
| if dataset_sha != self.hash: | ||
| raise NotImplementedError( | ||
| f"lance doesn't support loading other revisions than 'main' yet, but got {self.hash}" | ||
| ) | ||
| data_files = dl_manager.download(self.config.data_files) | ||
|
|
||
| # TODO: remove once Lance supports HF links with revisions | ||
| data_files = {split: [_fix_hf_uri(file) for file in files] for split, files in data_files.items()} | ||
| # TODO: remove once Lance supports symlinks for _version files | ||
| data_files = {split: [_fix_local_version_file(file) for file in files] for split, files in data_files.items()} | ||
|
|
||
| splits = [] | ||
| for split_name, files in data_files.items(): | ||
| storage_options = dl_manager.download_config.storage_options.get(files[0].split("://", 0)[0] + "://") | ||
|
|
||
| lance_dataset_uris = resolve_dataset_uris(files) | ||
| if lance_dataset_uris: | ||
| fragments = [ | ||
| frag | ||
| for uri in lance_dataset_uris | ||
| for frag in lance.dataset(uri, storage_options=storage_options).get_fragments() | ||
| ] | ||
| if self.info.features is None: | ||
| pa_schema = fragments[0]._ds.schema | ||
| splits.append( | ||
| datasets.SplitGenerator( | ||
| name=split_name, | ||
| gen_kwargs={"fragments": fragments, "lance_files": None}, | ||
| ) | ||
| ) | ||
| else: | ||
| lance_files = [lance.file.LanceFileReader(file, storage_options=storage_options) for file in files] | ||
| if self.info.features is None: | ||
| pa_schema = lance_files[0].metadata().schema | ||
| splits.append( | ||
| datasets.SplitGenerator( | ||
| name=split_name, | ||
| gen_kwargs={"fragments": None, "lance_files": lance_files}, | ||
| ) | ||
| ) | ||
| if self.info.features is None: | ||
| if self.config.columns: | ||
| fields = [ | ||
| pa_schema.field(name) for name in self.config.columns if pa_schema.get_field_index(name) != -1 | ||
| ] | ||
| pa_schema = pa.schema(fields) | ||
| self.info.features = datasets.Features.from_arrow_schema(pa_schema) | ||
|
|
||
| return splits | ||
|
|
||
| def _cast_table(self, pa_table: pa.Table) -> pa.Table: | ||
| if self.info.features is not None: | ||
| # more expensive cast to support nested features with keys in a different order | ||
| # allows str <-> int/float or str to Audio for example | ||
| pa_table = table_cast(pa_table, self.info.features.arrow_schema) | ||
| return pa_table | ||
|
|
||
| def _generate_tables( | ||
| self, | ||
| fragments: Optional[List["lance.LanceFragment"]], | ||
| lance_files: Optional[List["lance.file.LanceFileReader"]], | ||
| ): | ||
| if fragments: | ||
| for frag_idx, fragment in enumerate(fragments): | ||
| for batch_idx, batch in enumerate( | ||
| fragment.to_batches(columns=self.config.columns, batch_size=self.config.batch_size) | ||
| ): | ||
| table = pa.Table.from_batches([batch]) | ||
| yield Key(frag_idx, batch_idx), self._cast_table(table) | ||
| else: | ||
| for file_idx, lance_file in enumerate(lance_files): | ||
| for batch_idx, batch in enumerate(lance_file.read_all(batch_size=self.config.batch_size).to_batches()): | ||
| table = pa.Table.from_batches([batch]) | ||
| yield Key(file_idx, batch_idx), self._cast_table(table) | ||
This file contains hidden or 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,106 @@ | ||
| import lance | ||
| import numpy as np | ||
| import pyarrow as pa | ||
| import pytest | ||
|
|
||
| from datasets import load_dataset | ||
|
|
||
|
|
||
| @pytest.fixture | ||
| def lance_dataset(tmp_path) -> str: | ||
| data = pa.table( | ||
| { | ||
| "id": pa.array([1, 2, 3, 4]), | ||
| "value": pa.array([10.0, 20.0, 30.0, 40.0]), | ||
| "text": pa.array(["a", "b", "c", "d"]), | ||
| "vector": pa.FixedSizeListArray.from_arrays(pa.array([0.1] * 16, pa.float32()), list_size=4), | ||
| } | ||
| ) | ||
| dataset_path = tmp_path / "test_dataset.lance" | ||
| lance.write_dataset(data, dataset_path) | ||
| return str(dataset_path) | ||
|
|
||
|
|
||
| @pytest.fixture | ||
| def lance_hf_dataset(tmp_path) -> str: | ||
| data = pa.table( | ||
| { | ||
| "id": pa.array([1, 2, 3, 4]), | ||
| "value": pa.array([10.0, 20.0, 30.0, 40.0]), | ||
| "text": pa.array(["a", "b", "c", "d"]), | ||
| "vector": pa.FixedSizeListArray.from_arrays(pa.array([0.1] * 16, pa.float32()), list_size=4), | ||
| } | ||
| ) | ||
| dataset_dir = tmp_path / "data" / "train.lance" | ||
| dataset_dir.parent.mkdir(parents=True, exist_ok=True) | ||
| lance.write_dataset(data, dataset_dir) | ||
| lance.write_dataset(data[:2], tmp_path / "data" / "test.lance") | ||
|
|
||
| with open(tmp_path / "README.md", "w") as f: | ||
| f.write("""--- | ||
| size_categories: | ||
| - 1M<n<10M | ||
| source_datasets: | ||
| - lance_test | ||
| --- | ||
| # Test Lance Dataset\n\n | ||
| # My Markdown is fancier\n | ||
| """) | ||
|
|
||
| return str(tmp_path) | ||
|
|
||
|
|
||
| def test_load_lance_dataset(lance_dataset): | ||
| dataset_dict = load_dataset(lance_dataset) | ||
| assert "train" in dataset_dict.keys() | ||
|
|
||
| dataset = dataset_dict["train"] | ||
| assert "id" in dataset.column_names | ||
| assert "value" in dataset.column_names | ||
| assert "text" in dataset.column_names | ||
| assert "vector" in dataset.column_names | ||
| ids = dataset["id"] | ||
| assert ids == [1, 2, 3, 4] | ||
|
|
||
|
|
||
| @pytest.mark.parametrize("streaming", [False, True]) | ||
| def test_load_hf_dataset(lance_hf_dataset, streaming): | ||
| dataset_dict = load_dataset(lance_hf_dataset, columns=["id", "text"], streaming=streaming) | ||
| assert "train" in dataset_dict.keys() | ||
| assert "test" in dataset_dict.keys() | ||
| dataset = dataset_dict["train"] | ||
|
|
||
| assert "id" in dataset.column_names | ||
| assert "text" in dataset.column_names | ||
| assert "value" not in dataset.column_names | ||
| assert "vector" not in dataset.column_names | ||
| ids = list(dataset["id"]) | ||
| assert ids == [1, 2, 3, 4] | ||
| text = list(dataset["text"]) | ||
| assert text == ["a", "b", "c", "d"] | ||
| assert "value" not in dataset.column_names | ||
|
|
||
|
|
||
| def test_load_vectors(lance_hf_dataset): | ||
| dataset_dict = load_dataset(lance_hf_dataset, columns=["vector"]) | ||
| assert "train" in dataset_dict.keys() | ||
| dataset = dataset_dict["train"] | ||
|
|
||
| assert "vector" in dataset.column_names | ||
| vectors = dataset.data["vector"].combine_chunks().values.to_numpy(zero_copy_only=False) | ||
| assert np.allclose(vectors, np.full(16, 0.1)) | ||
|
|
||
|
|
||
| @pytest.mark.parametrize("streaming", [False, True]) | ||
| def test_load_lance_streaming_modes(lance_hf_dataset, streaming): | ||
| """Test loading Lance dataset in both streaming and non-streaming modes.""" | ||
| from datasets import IterableDataset | ||
|
|
||
| ds = load_dataset(lance_hf_dataset, split="train", streaming=streaming) | ||
| if streaming: | ||
| assert isinstance(ds, IterableDataset) | ||
| items = list(ds) | ||
| else: | ||
| items = list(ds) | ||
| assert len(items) == 4 | ||
| assert all("id" in item for item in items) |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
should we support columns pushdown here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just added it at
LanceFileReader()initialization, since the argument is not available inread_all()Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, how does it work with
multiple data files within same fragment?In Lance, one fragment can be 1 or more data files, where each data files cover a few columns. This is how we can add new features / column cheaply without rewriting the datasets, by adding new data files to existing fragment.
Maybe we can address it as follow up tasks.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
in that case it's a dataset no ? since it requires a manifest or something to tell what the fragments are made of
LanceFileReader()is only used for single files, i.e. that don't belong to a lance dataset directory or require manifest filesThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see. Lets 🚢 !!