Virtual Datasets#
While Icechunk works wonderfully with native chunks managed by Zarr, there is lots of archival data out there in other formats already. To interoperate with such data, Icechunk supports "Virtual" chunks, where any number of chunks in a given dataset may reference external data in existing archival formats, such as netCDF, HDF, GRIB, or TIFF. Virtual chunks are loaded directly from the original source without copying or modifying the original achival data files. This enables Icechunk to manage large datasets from existing data without needing that data to be in Zarr format already.
Note
The concept of a "virtual Zarr dataset" originates from the Kerchunk project, which preceded and inspired VirtualiZarr. Like VirtualiZarr, the kerchunk package provides functionality to scan metadata of existing data files and combine these references into larger virtual datasets, but unlike VirtualiZarr the Kerchunk package currently has no facility for writing to Icechunk stores. If you previously were interested in "Kerchunking" your data, you can now achieve a similar result by using VirtualiZarr to create virtual datasets and write them to icechunk.
VirtualiZarr lets users ingest existing data files into virtual datasets using various different tools under the hood, including kerchunk, xarray, zarr, and now icechunk. It does so by creating virtual references to existing data that can be combined and manipulated to create larger virtual datasets using xarray. These datasets can then be exported to kerchunk reference format or to an Icechunk repository, without ever copying or moving the existing data files.
Note
Currently, Icechunk support virtual references to data stored in s3 compatible,gcs, http/https, and local storage backends. Support for azure is on the roadmap.
Security considerations with virtual chunks
Virtual chunks let Icechunk point to external locations (s3://, http://, file://, etc.), which means a malicious repo could try to trick your code into reading sensitive data from your machine or other sources.
To protect you, Icechunk is safe by default: it won't read from these locations unless you explicitly allow it. This requires (1) defining trusted virtual chunk containers when writing data, and (2) passing authorize_virtual_chunk_access when opening a repo, so you stay in control of what external paths get accessed.
Creating a virtual dataset with VirtualiZarr#
We are going to create a virtual dataset pointing to all of the OISST data for August 2024. This data is distributed publicly as netCDF files on AWS S3, with one netCDF file containing the Sea Surface Temperature (SST) data for each day of the month. We are going to use VirtualiZarr to combine all of these files into a single virtual dataset spanning the entire month, then write that dataset to Icechunk for use in analysis.
Before we get started, we need to install virtualizarr, and icechunk. We also need to install fsspec and s3fs for working with data on s3.
First, we need to find all of the files we are interested in, we will do this with fsspec using a glob expression to find every netcdf file in the August 2024 folder in the bucket:
import fsspec
fs = fsspec.filesystem('s3', anon=True)
oisst_files = fs.glob('s3://noaa-cdr-sea-surface-temp-optimum-interpolation-pds/data/v2.1/avhrr/202408/oisst-avhrr-v02r01.*.nc')
oisst_files = sorted(['s3://'+f for f in oisst_files])
#['s3://noaa-cdr-sea-surface-temp-optimum-interpolation-pds/data/v2.1/avhrr/201001/oisst-avhrr-v02r01.20100101.nc',
# 's3://noaa-cdr-sea-surface-temp-optimum-interpolation-pds/data/v2.1/avhrr/201001/oisst-avhrr-v02r01.20100102.nc',
# 's3://noaa-cdr-sea-surface-temp-optimum-interpolation-pds/data/v2.1/avhrr/201001/oisst-avhrr-v02r01.20100103.nc',
# 's3://noaa-cdr-sea-surface-temp-optimum-interpolation-pds/data/v2.1/avhrr/201001/oisst-avhrr-v02r01.20100104.nc',
#...
#]
Now that we have the filenames of the data we need, we can create virtual datasets with VirtualiZarr. This may take a minute.
from virtualizarr import open_virtual_dataset
virtual_datasets =[
open_virtual_dataset(url, indexes={})
for url in oisst_files
]
We can now use xarray to combine these virtual datasets into one large virtual dataset (For more details on this operation see VirtualiZarr's documentation). We know that each of our files share the same structure but with a different date. So we are going to concatenate these datasets on the time dimension.
import xarray as xr
virtual_ds = xr.concat(
virtual_datasets,
dim='time',
coords='minimal',
compat='override',
combine_attrs='override'
)
#<xarray.Dataset> Size: 257MB
#Dimensions: (time: 31, zlev: 1, lat: 720, lon: 1440)
#Coordinates:
# time (time) float32 124B ManifestArray<shape=(31,), dtype=float32, ch...
# lat (lat) float32 3kB ManifestArray<shape=(720,), dtype=float32, chu...
# zlev (zlev) float32 4B ManifestArray<shape=(1,), dtype=float32, chunk...
# lon (lon) float32 6kB ManifestArray<shape=(1440,), dtype=float32, ch...
#Data variables:
# sst (time, zlev, lat, lon) int16 64MB ManifestArray<shape=(31, 1, 72...
# anom (time, zlev, lat, lon) int16 64MB ManifestArray<shape=(31, 1, 72...
# ice (time, zlev, lat, lon) int16 64MB ManifestArray<shape=(31, 1, 72...
# err (time, zlev, lat, lon) int16 64MB ManifestArray<shape=(31, 1, 72...
We have a virtual dataset with 31 timestamps! One hint that this worked correctly is that the readout shows the variables and coordinates as ManifestArray instances, the representation that VirtualiZarr uses for virtual arrays. Let's create an Icechunk repo to write this dataset to in the oisst directory on our local filesystem.
Note
Take note of the VirtualChunkContainer passed into the RepositoryConfig when creating the store. We specify the storage configuration necessary to access the anonymous S3 bucket that holds the OISST netCDF files, along with credentials that match. This creates a mapping between the s3 virtual chunk container and the credentials passed for the s3 namespace. For more configuration options, see the configuration page.
import icechunk
storage = icechunk.local_filesystem_storage(
path='oisst',
)
config = icechunk.RepositoryConfig.default()
config.set_virtual_chunk_container(icechunk.VirtualChunkContainer("s3://mybucket/my/data/", icechunk.s3_store(region="us-east-1")))
credentials = icechunk.containers_credentials({"s3://mybucket/my/data/": icechunk.s3_credentials(anonymous=True)})
repo = icechunk.Repository.create(storage, config, credentials)
With the repo created, and the virtual chunk container added, lets write our virtual dataset to Icechunk with VirtualiZarr!
The refs are written so lets save our progress by committing to the store.
Note
Your commit hash will be different! For more on the version control features of Icechunk, see the version control page.
Now we can read the dataset from the store using xarray to confirm everything went as expected. xarray reads directly from the Icechunk store because it is a fully compliant zarr Store instance.
ds = xr.open_zarr(
session.store,
zarr_version=3,
consolidated=False,
chunks={},
)
#<xarray.Dataset> Size: 1GB
#Dimensions: (lon: 1440, time: 31, zlev: 1, lat: 720)
#Coordinates:
# * lon (lon) float32 6kB 0.125 0.375 0.625 0.875 ... 359.4 359.6 359.9
# * zlev (zlev) float32 4B 0.0
# * time (time) datetime64[ns] 248B 2024-08-01T12:00:00 ... 2024-08-31T12...
# * lat (lat) float32 3kB -89.88 -89.62 -89.38 -89.12 ... 89.38 89.62 89.88
#Data variables:
# sst (time, zlev, lat, lon) float64 257MB dask.array<chunksize=(1, 1, 720, 1440), meta=np.ndarray>
# ice (time, zlev, lat, lon) float64 257MB dask.array<chunksize=(1, 1, 720, 1440), meta=np.ndarray>
# anom (time, zlev, lat, lon) float64 257MB dask.array<chunksize=(1, 1, 720, 1440), meta=np.ndarray>
# err (time, zlev, lat, lon) float64 257MB dask.array<chunksize=(1, 1, 720, 1440), meta=np.ndarray>
Success! We have created our full dataset with 31 timesteps spanning the month of august, all with virtual references to pre-existing data files in object store. This means we can now version control our dataset, allowing us to update it, and roll it back to a previous version without copying or moving any data from the original files.
Finally, let's make a plot of the sea surface temperature!

Note
Users of the repo will need to enable the virtual chunk container by passing the credentials argument to Repository.open. This way, the repo user, flags the container as authorized. credentials argument must be a dict using url prefixes as keys and optional credentials as values. If the container requires no credentials, None can be used as the value in the map. Failing to authorize a container, will generate an error when a chunk is fetched from it.
Virtual Reference API#
While VirtualiZarr is the easiest way to create virtual datasets with Icechunk, the Store API that it uses to create the datasets in Icechunk is public. IcechunkStore contains a set_virtual_ref method that specifies a virtual ref for a specified chunk.
Virtual Reference Storage Support#
Currently, Icechunk supports four types of storage for virtual references:
S3 Compatible#
References to files accessible via S3 compatible storage.
Example#
Here is how we can set the chunk at key c/0 to point to a file on an s3 bucket,mybucket, with the prefix my/data/file.nc:
config = icechunk.RepositoryConfig.default()
config.set_virtual_chunk_container(icechunk.VirtualChunkContainer("s3://mybucket/my/data", icechunk.s3_store(region="us-east-1")))
repo = icechunk.Repository.create(storage, config)
session = repo.writable_session("main")
session.store.set_virtual_ref('c/0', 's3://mybucket/my/data/file.nc', offset=1000, length=200)
Configuration#
S3 virtual references require configuring credential for the store to be able to access the specified s3 bucket. See the configuration docs for instructions.
GCS#
References to files accessible on Google Cloud Storage
Example#
Here is how we can set the chunk at key c/0 to point to a file on an s3 bucket,mybucket, with the prefix my/data/file.nc:
config = icechunk.RepositoryConfig.default()
config.set_virtual_chunk_container(icechunk.VirtualChunkContainer("gcs://mybucket/my/data", icechunk.gcs_store(options={})))
repo = icechunk.Repository.create(storage, config)
session = repo.writable_session("main")
session.store.set_virtual_ref('c/0', 'gcs://mybucket/my/data/file.nc', offset=1000, length=200)
HTTP#
References to files accessible via http(s) protocol
Example#
Here is how we can set the chunk at key c/0 to point to a file on myserver, with the prefix my/data/file.nc:
config = icechunk.RepositoryConfig.default()
config.set_virtual_chunk_container(icechunk.VirtualChunkContainer("https://myserver/my/data", icechunk.http_store(options={})))
repo = icechunk.Repository.create(storage, config)
session = repo.writable_session("main")
session.store.set_virtual_ref('c/0', 'https://myserver/my/data/file.nc', offset=1000, length=200)
Local Filesystem#
References to files accessible via local filesystem. This requires any file paths to be absolute at this time.
Example#
Here is how we can set the chunk at key c/0 to point to a file on my local filesystem located at /path/to/my/file.nc:
config = icechunk.RepositoryConfig.default()
config.set_virtual_chunk_container(icechunk.VirtualChunkContainer("s3://mybucket/my/data", icechunk.local_filesystem_store("/path/to/my")))
repo = icechunk.Repository.create(storage, config)
session = repo.writable_session("main")
session.store.set_virtual_ref('c/0', 'file:///path/to/my/file.nc', offset=20, length=100)
No extra configuration is necessary for local filesystem references.
Virtual Reference File Format Support#
Icechunk supports creating virtual references for any filetypes that VirtualiZarr supports. This currently includes HDF5, netcdf4, and netcdf3, and TIFF/GeoTIFF. Support for other filetypes is under development in the VirtualiZarr project, and VirtualiZarr has an extensible system of 3rd-party Parsers if you want to implement support for a new filetype yourself Below are some relevant docs and issues: