|
1 | 1 | from __future__ import annotations
|
2 | 2 |
|
3 | 3 | from io import BytesIO
|
| 4 | +from pathlib import Path |
4 | 5 |
|
5 |
| -import bioframe |
6 | 6 | import dask
|
7 | 7 | import dask.dataframe as dd
|
8 | 8 | import oxbow as ox
|
|
11 | 11 |
|
12 | 12 | __version__ = "0.1.0"
|
13 | 13 |
|
14 |
| -__all__ = ("__version__", "read_bam") |
| 14 | +__all__ = ("__version__", "read_bam", "read_vcf", "read_bcf") |
15 | 15 |
|
16 | 16 |
|
17 |
| -def _read_bam_query_from_path( |
18 |
| - path: str, chrom: str, start: int, end: int |
| 17 | +def _read_bam_vpos_from_path( |
| 18 | + path: str, vpos_lo: tuple[int, int], vpos_hi: tuple[int, int] |
19 | 19 | ) -> pd.DataFrame:
|
20 |
| - stream = BytesIO(ox.read_bam(path, f"{chrom}:{start}-{end}")) |
| 20 | + stream = BytesIO(ox.read_bam_vpos(path, vpos_lo, vpos_hi)) |
21 | 21 | ipc = pyarrow.ipc.open_file(stream)
|
22 | 22 | return ipc.read_pandas()
|
23 | 23 |
|
24 | 24 |
|
25 |
| -def read_bam(path: str, chunksize: int = 10_000_000) -> dd.DataFrame: |
| 25 | +def _read_vcf_vpos_from_path( |
| 26 | + path: str, vpos_lo: tuple[int, int], vpos_hi: tuple[int, int] |
| 27 | +) -> pd.DataFrame: |
| 28 | + stream = BytesIO(ox.read_vcf_vpos(path, vpos_lo, vpos_hi)) |
| 29 | + ipc = pyarrow.ipc.open_file(stream) |
| 30 | + return ipc.read_pandas() |
| 31 | + |
| 32 | + |
| 33 | +def _read_bcf_vpos_from_path( |
| 34 | + path: str, vpos_lo: tuple[int, int], vpos_hi: tuple[int, int] |
| 35 | +) -> pd.DataFrame: |
| 36 | + stream = BytesIO(ox.read_bcf_vpos(path, vpos_lo, vpos_hi)) |
| 37 | + ipc = pyarrow.ipc.open_file(stream) |
| 38 | + return ipc.read_pandas() |
| 39 | + |
| 40 | + |
| 41 | +def read_bam( |
| 42 | + path: str | Path, chunksize: int = 10_000_000, index: str | Path | None = None |
| 43 | +) -> dd.DataFrame: |
26 | 44 | """
|
27 | 45 | Map an indexed BAM file to a Dask DataFrame.
|
28 | 46 |
|
29 | 47 | Parameters
|
30 | 48 | ----------
|
31 |
| - path : str |
| 49 | + path : str or Path |
32 | 50 | Path to the BAM file.
|
33 | 51 | chunksize : int, optional [default=10_000_000]
|
34 |
| - Chunk size, currently in base pair coordinates. |
| 52 | + Approximate partition size, in compressed bytes. |
| 53 | + index : str or Path, optional |
| 54 | + Path to the index file. If not provided, the index file is assumed to |
| 55 | + be at the same location as the BAM file, with the same name but with |
| 56 | + the additional .bai or .csi extension. |
| 57 | +
|
| 58 | + Returns |
| 59 | + ------- |
| 60 | + dask.dataframe.DataFrame |
| 61 | + """ |
| 62 | + path = Path(path) |
| 63 | + if index is None: |
| 64 | + bai_index = path.with_suffix(".bai") |
| 65 | + csi_index = path.with_suffix(".csi") |
| 66 | + if bai_index.exists(): |
| 67 | + index = bai_index |
| 68 | + elif csi_index.exists(): |
| 69 | + index = csi_index |
| 70 | + else: |
| 71 | + msg = "Index .bai or .csi file not found." |
| 72 | + raise FileNotFoundError(msg) |
| 73 | + |
| 74 | + vpos = ox.partition_from_index_file(index, chunksize) |
| 75 | + chunks = [ |
| 76 | + dask.delayed(_read_bam_vpos_from_path)(path, tuple(vpos[i]), tuple(vpos[i + 1])) |
| 77 | + for i in range(len(vpos) - 1) |
| 78 | + ] |
| 79 | + |
| 80 | + return dd.from_delayed(chunks) |
| 81 | + |
| 82 | + |
| 83 | +def read_vcf( |
| 84 | + path: str | Path, chunksize: int = 10_000_000, index: str | Path | None = None |
| 85 | +) -> dd.DataFrame: |
| 86 | + """ |
| 87 | + Map an indexed, bgzf-compressed VCF.gz file to a Dask DataFrame. |
| 88 | +
|
| 89 | + Parameters |
| 90 | + ---------- |
| 91 | + path : str or Path |
| 92 | + Path to the VCF.gz file. |
| 93 | + chunksize : int, optional [default=10_000_000] |
| 94 | + Approximate partition size, in compressed bytes. |
| 95 | + index : str or Path, optional |
| 96 | + Path to the index file. If not provided, the index file is assumed to |
| 97 | + be at the same location as the VCF.gz file, with the same name but with |
| 98 | + the additional .tbi or .csi extension. |
35 | 99 |
|
36 | 100 | Returns
|
37 | 101 | -------
|
38 | 102 | dask.dataframe.DataFrame
|
39 |
| - A Dask DataFrame with the BAM file contents. |
40 | 103 | """
|
41 |
| - chromsizes = bioframe.fetch_chromsizes("hg38") |
42 |
| - chunk_spans = bioframe.binnify(chromsizes, chunksize) |
| 104 | + path = Path(path) |
| 105 | + if index is None: |
| 106 | + tbi_index = path.with_suffix(".tbi") |
| 107 | + csi_index = path.with_suffix(".csi") |
| 108 | + if tbi_index.exists(): |
| 109 | + index = tbi_index |
| 110 | + elif csi_index.exists(): |
| 111 | + index = csi_index |
| 112 | + else: |
| 113 | + msg = "Index .tbi or .csi file not found." |
| 114 | + raise FileNotFoundError(msg) |
| 115 | + |
| 116 | + vpos = ox.partition_from_index_file(index, chunksize) |
43 | 117 | chunks = [
|
44 |
| - dask.delayed(_read_bam_query_from_path)(path, chrom, start + 1, end) |
45 |
| - for chrom, start, end in chunk_spans.to_numpy() |
| 118 | + dask.delayed(_read_vcf_vpos_from_path)(path, tuple(vpos[i]), tuple(vpos[i + 1])) |
| 119 | + for i in range(len(vpos) - 1) |
46 | 120 | ]
|
| 121 | + |
| 122 | + return dd.from_delayed(chunks) |
| 123 | + |
| 124 | + |
| 125 | +def read_bcf( |
| 126 | + path: str | Path, chunksize: int = 10_000_000, index: str | Path | None = None |
| 127 | +) -> dd.DataFrame: |
| 128 | + """ |
| 129 | + Map an indexed BCF file to a Dask DataFrame. |
| 130 | +
|
| 131 | + Parameters |
| 132 | + ---------- |
| 133 | + path : str or Path |
| 134 | + Path to the BCF file. |
| 135 | + chunksize : int, optional [default=10_000_000] |
| 136 | + Approximate partition size, in compressed bytes. |
| 137 | + index : str or Path, optional |
| 138 | + Path to the index file. If not provided, the index file is assumed to |
| 139 | + be at the same location as the BCF file, with the same name but with |
| 140 | + the additional .csi extension. |
| 141 | +
|
| 142 | + Returns |
| 143 | + ------- |
| 144 | + dask.dataframe.DataFrame |
| 145 | + """ |
| 146 | + path = Path(path) |
| 147 | + if index is None: |
| 148 | + csi_index = path.with_suffix(".csi") |
| 149 | + if csi_index.exists(): |
| 150 | + index = csi_index |
| 151 | + else: |
| 152 | + msg = "Index .csi file not found." |
| 153 | + raise FileNotFoundError(msg) |
| 154 | + |
| 155 | + vpos = ox.partition_from_index_file(index, chunksize) |
| 156 | + chunks = [ |
| 157 | + dask.delayed(_read_bcf_vpos_from_path)(path, tuple(vpos[i]), tuple(vpos[i + 1])) |
| 158 | + for i in range(len(vpos) - 1) |
| 159 | + ] |
| 160 | + |
47 | 161 | return dd.from_delayed(chunks)
|
0 commit comments