Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
76 commits
Select commit Hold shift + click to select a range
de9ab79
[WIP] manifest sharding
dcherian Feb 27, 2025
14d9048
WIP
dcherian Mar 4, 2025
6aa7e29
More condition parsing work
dcherian Mar 6, 2025
70412ec
thread config through
dcherian Mar 7, 2025
267c9cf
clippied
dcherian Mar 17, 2025
acf8fa5
WIP proptest
dcherian Mar 17, 2025
2542412
Revert "WIP proptest"
dcherian Mar 17, 2025
d816f8b
Simple test passes!
dcherian Mar 17, 2025
a64252a
Cleanup
dcherian Mar 18, 2025
8ad20e8
Better test
dcherian Mar 18, 2025
ac42185
More tests
dcherian Mar 18, 2025
34da81b
Lossen type
dcherian Mar 18, 2025
d935131
Update gc test
dcherian Mar 18, 2025
d27a2a2
DimensionName(regex)
dcherian Mar 18, 2025
83fef67
more test
dcherian Mar 18, 2025
b47788c
Revert "Lossen type"
dcherian Mar 18, 2025
a56ccce
Add condition parsing tests
dcherian Mar 18, 2025
87af732
WIP
dcherian Mar 19, 2025
63fdc6d
Add notes
dcherian Mar 20, 2025
d491c3e
Iterator -> Stream
dcherian Mar 20, 2025
fc965d7
Optimize reads!
dcherian Mar 21, 2025
b2167f7
Python config
dcherian Mar 24, 2025
dd95c32
Add reprs
dcherian Mar 26, 2025
fb89826
Add from_dict
dcherian Mar 26, 2025
edb1d13
clippy
dcherian Mar 26, 2025
4732a61
[revert] comment out bad test
dcherian Mar 26, 2025
5b70006
Add doctest to Just
dcherian Mar 31, 2025
0ce517e
Add doctest
dcherian Mar 31, 2025
dd9b77d
Merge branch 'main' into split-manifests
dcherian Mar 31, 2025
fc19e7f
Fix appends
dcherian Mar 31, 2025
22deffe
Add updating config on read test
dcherian Mar 31, 2025
2121f9a
Fix types
dcherian Mar 31, 2025
56158c5
add comment to clarify monkey patch
dcherian Mar 31, 2025
1a0215c
Update icechunk-python/python/icechunk/_icechunk_python.pyi
dcherian Apr 1, 2025
deb5a7e
Fix docstring types
dcherian Mar 31, 2025
d749f41
Update type, docstring
dcherian Apr 1, 2025
e648329
ShardDimCondition::Any -> Rest
dcherian Apr 1, 2025
aa82355
Minor cleanup
dcherian Apr 1, 2025
5d2318b
More complex tests
dcherian Apr 1, 2025
10e3b7e
Aggregate extents while grouping shards.
dcherian Apr 1, 2025
137f283
Update reprs
dcherian Apr 1, 2025
1119f81
Benchmarks cleanup
dcherian Apr 1, 2025
f3db41b
Add write benchmark
dcherian Apr 1, 2025
ac6a0ef
Add read benchmark
dcherian Apr 2, 2025
2f42e71
Merge branch 'main' into split-manifests
dcherian Apr 3, 2025
ea50452
Add rust test for large numbers of refs
dcherian Apr 3, 2025
2ea8e2a
Merge branch 'main' into split-manifests
dcherian Apr 3, 2025
33966cb
Add to test_can_read_old.py
dcherian Apr 3, 2025
80f8f5d
shard → split
dcherian Apr 4, 2025
94bbf9e
Merge branch 'main' into split-manifests
dcherian Apr 4, 2025
6fc7eb6
one more rename
dcherian Apr 4, 2025
a76558d
Merge branch 'main' into split-manifests
dcherian Apr 15, 2025
c35b589
Address minor comments.
dcherian Apr 15, 2025
f6156b9
Comment out handling sessionerror.
dcherian Apr 15, 2025
48bebf2
Rest -> Any
dcherian Apr 15, 2025
0d7e01f
Assert len(manifestextents) > 0
dcherian Apr 15, 2025
8c4cc59
New ManifestSplitDim struct
dcherian Apr 15, 2025
872a522
lint
dcherian Apr 15, 2025
77329bf
Merge branch 'main' into split-manifests
dcherian May 7, 2025
8d24d01
Remove unneeded Index
dcherian May 7, 2025
5d6cefa
Add property test
dcherian May 7, 2025
5e34e6c
Add docs
dcherian May 7, 2025
315d1df
Add ManifestSplitCondition.AnyArray
dcherian May 7, 2025
9954cda
fix docs
dcherian May 8, 2025
0b3581d
Add Or condition test
dcherian May 8, 2025
c1c7688
fix docs
dcherian May 8, 2025
9e625cf
fix docs
dcherian May 8, 2025
39125a8
fix docs
dcherian May 8, 2025
b5812d5
Try speeding up docs build
dcherian May 8, 2025
02c95b6
fix docs
dcherian May 8, 2025
8051f92
try fixing docs rendering
dcherian May 12, 2025
bf8936e
Apply suggestions from code review
dcherian May 13, 2025
371c045
more docs
dcherian May 13, 2025
5a955f5
Fix test
dcherian May 13, 2025
163d31c
tweak docs build
dcherian May 13, 2025
0f44af8
Merge branch 'main' into split-manifests
dcherian May 13, 2025
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .pre-commit-config.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,6 @@ repos:
entry: just pre-commit
language: system
pass_filenames: false
files: "icechunk/"
files: "(^|.*/)(icechunk/|icechunk-python/src/).*"

exclude: 'tests/data/.*'
4 changes: 4 additions & 0 deletions Justfile
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,9 @@ alias pre := pre-commit
test *args='':
cargo test --all --all-targets {{args}}

doctest *args='':
cargo test --doc {{args}}

# run all tests with logs enabled
test-logs level *args='':
RUST_LOG=icechunk={{level}} cargo test --all --all-targets {{args}} -- --nocapture
Expand Down Expand Up @@ -46,6 +49,7 @@ pre-commit $RUSTFLAGS="-D warnings -W unreachable-pub -W bare-trait-objects":
just build
just format "--check"
just lint "-p icechunk -p icechunk-python"
just doctest
just test
just run-all-examples
just check-deps
Expand Down
5 changes: 2 additions & 3 deletions docs/.readthedocs.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,10 @@ build:
python: "mambaforge-latest"

jobs:
pre_create_environment:
- conda update --yes --quiet --name=base --channel=defaults conda
install:
- which mamba
- cd icechunk-python && maturin build && pip install ../target/wheels/*.whl && cd ../docs
- cd icechunk-python && maturin build && pip install "$(ls ../target/wheels/*.whl | head -n 1)[docs]" && cd ../docs
- mamba list
# - cd icechunk-python && maturin develop && cd ../docs


Expand Down
33 changes: 1 addition & 32 deletions docs/doc-env.yml
Original file line number Diff line number Diff line change
Expand Up @@ -4,39 +4,8 @@ channels:
- conda-forge
- nodefaults
dependencies:
- python>=3.10
- "sphinx"
- python>=3.12
- pip
- maturin
- xarray
- pooch
- scipy
- dask
- cftime
- distributed
- cairo
- rust
- maturin
- pip:
- "myst_nb"
- mkdocs-awesome-pages-plugin
- "pydata_sphinx_theme"
- mkdocs-mermaid2-plugin
- markdown-exec
- mkdocs-breadcrumbs-plugin
- mkdocs-minify-plugin
- mkdocs-open-in-new-tab
- mkdocs
- mkdocs-material[imaging]
- mkdocstrings[python]
- mkdocs-jupyter
- mkdocs-git-revision-date-localized-plugin
- mkdocs-git-committers-plugin-2
- mkdocs-macros-plugin
- mkdocs-include-markdown-plugin
- mkdocs-redirects
- mkdocs-git-authors-plugin
- "sphinx_copybutton"
- "sphinx_design"
- "sphinx_togglebutton"
- "sphinx-autodoc-typehints"
90 changes: 90 additions & 0 deletions docs/docs/icechunk-python/performance.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
# Performance

!!! info

This is advanced material, and you will need it only if you have arrays with more than a million chunks.
Icechunk aims to provide an excellent experience out of the box.

## Preloading manifests

Coming Soon.

## Splitting manifests

Icechunk stores chunk references in a chunk manifest file stored in `manifests/`.
For very large arrays (millions of chunks), these files can get quite large.
By default, Icechunk stores all chunk references in a single manifest file per array.
Requesting even a single chunk requires downloading the entire manifest.
In some cases, this can result in a slow time-to-first-byte or large memory usage.

!!! note

Note that the chunk sizes in the following examples are tiny for demonstration purposes.

To avoid that, Icechunk lets you split the manifest files by specifying a ``ManifestSplittingConfig``.

```python exec="on" session="perf" source="material-block"
import icechunk as ic
from icechunk import ManifestSplitCondition, ManifestSplittingConfig, ManifestSplitDimCondition

split_config = ManifestSplittingConfig.from_dict(
{
ManifestSplitCondition.AnyArray(): {
ManifestSplitDimCondition.DimensionName("time"): 365 * 24
}
}
)
repo_config = ic.RepositoryConfig(manifest=ic.ManifestConfig(splitting=split_config))
```

Then pass the config to `Repository.open` or `Repository.create`
```python
repo = ic.Repository.open(..., config=repo_config)
```

This particular example splits manifests so that each manifest contains `365 * 24` chunks along the time dimension, and every chunk along every other dimension in a single file.

Options for specifying the arrays whose manifest you want to split are:

1. [`ManifestSplitCondition.name_matches`](./reference.md#icechunk.ManifestSplitCondition.name_matches) takes a regular expression used to match an array's name;
2. [`ManifestSplitCondition.path_matches`](./reference.md#icechunk.ManifestSplitCondition.path_matches) takes a regular expression used to match an array's path;
3. [`ManifestSplitCondition.and_conditions`](./reference.md#icechunk.ManifestSplitCondition.and_conditions) to combine (1), (2), and (4) together; and
4. [`ManifestSplitCondition.or_conditions`](./reference.md#icechunk.ManifestSplitCondition.or_conditions) to combine (1), (2), and (3) together.


`And` and `Or` may be used to combine multiple path and/or name matches. For example,
```python exec="on" session="perf" source="material-block"
array_condition = ManifestSplitCondition.or_conditions(
[
ManifestSplitCondition.name_matches("temperature"),
ManifestSplitCondition.name_matches("salinity"),
]
)
sconfig = ManifestSplittingConfig.from_dict(
{array_condition: {ManifestSplitDimCondition.DimensionName("longitude"): 3}}
)
```

Options for specifying how to split along a specific axis or dimension are:

1. [`ManifestSplitDimCondition.Axis`](./reference.md#icechunk.ManifestSplitDimCondition.Axis) takes an integer axis;
2. [`ManifestSplitDimCondition.DimensionName`](./reference.md#icechunk.ManifestSplitDimCondition.DimensionName) takes a regular expression used to match the dimension names of the array;
3. [`ManifestSplitDimCondition.Any`](./reference.md#icechunk.ManifestSplitDimCondition.Any) matches any _remaining_ dimension name or axis.


For example, for an array with dimensions `time, latitude, longitude`, the following config
```python exec="on" session="perf" source="material-block"
from icechunk import ManifestSplitDimCondition

{
ManifestSplitDimCondition.DimensionName("longitude"): 3,
ManifestSplitDimCondition.Axis(1): 2,
ManifestSplitDimCondition.Any(): 1,
}
```
will result in splitting manifests so that each manifest contains (3 longitude chunks x 2 latitude chunks x 1 time chunk) = 6 chunks per manifest file.


!!! note

Python dictionaries preserve insertion order, so the first condition encountered takes priority.
1 change: 1 addition & 0 deletions docs/mkdocs.yml
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,7 @@ nav:
- Quickstart: icechunk-python/quickstart.md
- Configuration: icechunk-python/configuration.md
- Storage: icechunk-python/storage.md
- Performance: icechunk-python/performance.md
- FAQ: icechunk-python/faq.md
- Xarray: icechunk-python/xarray.md
- Parallel Writes: icechunk-python/parallel.md
Expand Down
45 changes: 32 additions & 13 deletions icechunk-python/benchmarks/conftest.py
Original file line number Diff line number Diff line change
@@ -1,14 +1,13 @@
import copy
from typing import cast

import pytest

from benchmarks import helpers
from benchmarks.datasets import (
ERA5,
ERA5_ARCO,
ERA5_SINGLE,
GB_8MB_CHUNKS,
GB_128MB_CHUNKS,
LARGE_1D,
LARGE_MANIFEST_SHARDED,
LARGE_MANIFEST_UNSHARDED,
PANCAKE_WRITES,
SIMPLE_1D,
TEST_BUCKETS,
Expand All @@ -18,15 +17,22 @@
)
from icechunk import Repository, local_filesystem_storage

try:
from icechunk import ManifestSplittingConfig # noqa: F401

no_splitting = False
except ImportError:
no_splitting = True


def request_to_dataset(request, moar_prefix: str = "") -> Dataset:
extra_prefix = request.config.getoption("--icechunk-prefix") + moar_prefix
where = request.config.getoption("--where")
ds = request.param
ds = copy.deepcopy(request.param)
if where == "local" and ds.skip_local:
pytest.skip()
# for some reason, this gets run multiple times so we apply the prefix repeatedly
# if we don't catch that :(
# this gets run multiple times because the fixture scope is 'function'
# so we need this `force_idempotent` ugliness
ds.storage_config = ds.storage_config.with_overwrite(
**TEST_BUCKETS[where]
).with_extra(prefix=extra_prefix, force_idempotent=True)
Expand All @@ -50,13 +56,26 @@ def simple_write_dataset(request) -> BenchmarkWriteDataset:
return cast(BenchmarkWriteDataset, ds)


@pytest.fixture(params=[pytest.param(LARGE_1D, id="large-1d")])
def large_write_dataset(request) -> BenchmarkWriteDataset:
moar_prefix = helpers.rdms()
ds = request_to_dataset(request, moar_prefix=moar_prefix)
return cast(BenchmarkWriteDataset, ds)


@pytest.fixture(
params=[
pytest.param(GB_8MB_CHUNKS, id="gb-8mb"),
pytest.param(GB_128MB_CHUNKS, id="gb-128mb"),
pytest.param(ERA5_SINGLE, id="era5-single"),
pytest.param(ERA5, id="era5-weatherbench"),
pytest.param(ERA5_ARCO, id="era5-arco"),
# pytest.param(GB_8MB_CHUNKS, id="gb-8mb"),
# pytest.param(GB_128MB_CHUNKS, id="gb-128mb"),
# pytest.param(ERA5_SINGLE, id="era5-single"),
# pytest.param(ERA5, id="era5-weatherbench"),
# pytest.param(ERA5_ARCO, id="era5-arco"),
pytest.param(LARGE_MANIFEST_UNSHARDED, id="large-manifest-no-split"),
pytest.param(
LARGE_MANIFEST_SHARDED,
id="large-manifest-split",
marks=pytest.mark.skipif(no_splitting, reason="no splitting"),
),
],
)
def synth_dataset(request) -> BenchmarkReadDataset:
Expand Down
Loading