Kerchunk and GRIB2: A Case Study using NOAA’s High-Resolution Rapid Refresh (HRRR) Dataset

HRRR GRIB2

Overview

Within this notebook, we will cover:

  1. Generating a list of GRIB2 files on a remote filesystem using fsspec

  2. How to create reference files of GRIB2 files using Kerchunk

  3. Combining multiple Kerchunk reference files using MultiZarrToZarr

  4. Reading the output with Xarray and Intake

This notebook shares many similarities with the Multi-File Datasets with Kerchunk and the NetCDF/HDF5 Argentinian Weather Dataset Case Study, however this case studies examines another data format and uses kerchunk.scan_grib to create reference files.

This notebook borrows heavily from this GIST created by Peter Marsh.

Prerequisites

Concepts

Importance

Notes

Kerchunk Basics

Required

Core

Multiple Files and Kerchunk

Required

Core

Kerchunk and Dask

Required

Core

Introduction to Xarray

Required

IO/Visualization

Intake Introduction

Recommended

IO

  • Time to learn: 45 minutes


Motivation

Kerchunk supports multiple input file formats. One of these is GRIB2(GRIdded Information in Binary form), which is a binary file format primary used in meteorology and weather datasets. Similar to NetCDF/HDF5, GRIB2 does not support efficient, parallel access. Using Kerchunk, we can read this legacy format as if it were an ARCO (Analysis-Ready, Cloud-Optimized) data format such as Zarr.

About the Dataset

The HRRR is a NOAA real-time 3-km resolution, hourly updated, cloud-resolving, convection-allowing atmospheric model, initialized by 3km grids with 3km radar assimilation. Radar data is assimilated in the HRRR every 15 min over a 1-h period adding further detail to that provided by the hourly data assimilation from the 13km radar-enhanced Rapid Refresh. NOAA releases a copy of this dataset via the AWS Registry of Open Data.

Flags

In the section below, set the subset flag to be True (default) or False depending if you want this notebook to process the full file list. If set to True, then a subset of the file list will be processed (Recommended)

subset_flag = True

Imports

import datetime as dt
import glob
import logging
from tempfile import TemporaryDirectory

import dask
import fsspec
import ujson
import xarray as xr
from distributed import Client
from kerchunk.combine import MultiZarrToZarr
from kerchunk.grib2 import scan_grib
from tqdm import tqdm

Create Input File List

Here we create fsspec files-systems for reading remote files and writing local reference files. Next we are using fsspec.glob to retrieve a list of file paths and appending the s3:// prefix to them.

# Initiate fsspec filesystems for reading and writing
fs_read = fsspec.filesystem("s3", anon=True, skip_instance_cache=True)

# retrieve list of available days in archive
days_available = fs_read.glob("s3://noaa-hrrr-bdp-pds/hrrr.*")

# Read HRRR GRIB2 files from latest day
files = fs_read.glob(f"s3://{days_available[-1]}/conus/*wrfsfcf01.grib2")

# Append s3 prefix for filelist
files = sorted(["s3://" + f for f in files])

# If the subset_flag == True (default), the list of input files will be subset to speed up the processing
if subset_flag:
    files = files[0:2]
files[0]
's3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2'
"s3://noaa-hrrr-bdp-pds/hrrr.20230419/conus/hrrr.t00z.wrfsfcf01.grib2"
"s3://noaa-hrrr-bdp-pds/hrrr.20230419/conus/hrrr.t00z.wrfsfcf01.grib2"
's3://noaa-hrrr-bdp-pds/hrrr.20230419/conus/hrrr.t00z.wrfsfcf01.grib2'

Start a Dask Client

To parallelize the creation of our reference files, we will use Dask. For a detailed guide on how to use Dask and Kerchunk, see the Foundations notebook: Kerchunk and Dask.

client = Client(n_workers=8, silence_logs=logging.ERROR)
client

Client

Client-0fcc4d81-21bd-11ee-8df8-0022481ea464

Connection method: Cluster object Cluster type: distributed.LocalCluster
Dashboard: http://127.0.0.1:8787/status

Cluster Info

Iterate through list of files and create Kerchunk indicies as .json reference files

Each input GRIB2 file contains mutiple “messages”, each a measure of some variable on a grid, but with grid dimensions not necessarily compatible with one-another. The filter we create in the first line selects only certain types of messages, and indicated that heightAboveGround will be a coordinate of interest.

We also write a separate JSON for each of the selected message, since these are the basic component data sets (see the loop over out).

# Note: scan_grib does not require a filter and will happily create a reference file for each available grib message. However when combining the grib messages using MultiZarrToZarr it is neccassary for the messages to share a coordinate system. Thus to make our lives easier and ensure all reference outputs from scan_grib share a coordinate system we pass a filter argument.
afilter = {"typeOfLevel": "heightAboveGround", "level": [2, 10]}
so = {"anon": True}

# We are creating a temporary directory to store the .json reference files
# Alternately, you could write these to cloud storage.
td = TemporaryDirectory()
temp_dir = td.name
temp_dir
'/tmp/tmpwjzcs7uj'
def make_json_name(
    file_url, message_number
):  # create a unique name for each reference file
    date = file_url.split("/")[3].split(".")[1]
    name = file_url.split("/")[5].split(".")[1:3]
    return f"{temp_dir}/{date}_{name[0]}_{name[1]}_message{message_number}.json"


def gen_json(file_url):
    out = scan_grib(
        file_url, storage_options=so, inline_threshold=100, filter=afilter
    )  # create the reference using scan_grib
    for i, message in enumerate(
        out
    ):  # scan_grib outputs a list containing one reference per grib message
        out_file_name = make_json_name(file_url, i)  # get name
        with open(out_file_name, "w") as f:
            f.write(ujson.dumps(message))  # write to file


# Generate Dask Delayed objects
tasks = [dask.delayed(gen_json)(fil) for fil in files]
# Start parallel processing
import warnings

warnings.filterwarnings("ignore")
dask.compute(tasks)
([None, None],)

Combine Kerchunk reference .json files

We know that four coordinates are identical for every one of our component datasets - they are not functions of valid_time.

# Create a list of reference json files
output_files = glob.glob(f"{temp_dir}/*.json")

# Combine individual references into single consolidated reference
mzz = MultiZarrToZarr(
    output_files,
    concat_dims=["valid_time"],
    identical_dims=["latitude", "longitude", "heightAboveGround", "step"],
)
multi_kerchunk = mzz.translate()
multi_kerchunk
{'version': 1,
 'refs': {'.zgroup': '{"zarr_format":2}',
  'valid_time/.zarray': '{\n    "chunks": [\n        2\n    ],\n    "compressor": null,\n    "dtype": "<i8",\n    "fill_value": null,\n    "filters": null,\n    "order": "C",\n    "shape": [\n        2\n    ],\n    "zarr_format": 2\n}',
  'valid_time/0': 'base64:kEyvZAAAAACgWq9kAAAAAA==',
  'valid_time/.zattrs': '{\n    "_ARRAY_DIMENSIONS": [\n        "valid_time"\n    ],\n    "calendar": "proleptic_gregorian",\n    "long_name": "time",\n    "standard_name": "time",\n    "units": "seconds since 1970-01-01T00:00:00"\n}',
  '.zattrs': '{"centre":"kwbc","centreDescription":"US National Weather Service - NCEP","edition":2,"subCentre":0}',
  'unknown/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"unknown"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'unknown/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"unknown","cfVarName":"unknown","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"unknown","numberOfPoints":1905141,"paramId":0,"shortName":"unknown","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"unknown"}',
  'unknown/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   55318613,
   1308094],
  'heightAboveGround/.zarray': '{"chunks":[1],"compressor":null,"dtype":"<i8","fill_value":null,"filters":null,"order":"C","shape":[1],"zarr_format":2}',
  'heightAboveGround/0': '\x02\x00\x00\x00\x00\x00\x00\x00',
  'heightAboveGround/.zattrs': '{"_ARRAY_DIMENSIONS":["heightAboveGround"],"long_name":"height above the surface","positive":"up","standard_name":"height","units":"m"}',
  'latitude/.zarray': '{"chunks":[1059,1799],"compressor":null,"dtype":"<f8","fill_value":null,"filters":[{"dtype":"float64","id":"grib","var":"latitude"}],"order":"C","shape":[1059,1799],"zarr_format":2}',
  'latitude/0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   33217465,
   64451],
  'latitude/.zattrs': '{"_ARRAY_DIMENSIONS":["y","x"],"long_name":"latitude","standard_name":"latitude","units":"degrees_north"}',
  'longitude/.zarray': '{"chunks":[1059,1799],"compressor":null,"dtype":"<f8","fill_value":null,"filters":[{"dtype":"float64","id":"grib","var":"longitude"}],"order":"C","shape":[1059,1799],"zarr_format":2}',
  'longitude/0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   33217465,
   64451],
  'longitude/.zattrs': '{"_ARRAY_DIMENSIONS":["y","x"],"long_name":"longitude","standard_name":"longitude","units":"degrees_east"}',
  'step/.zarray': '{"chunks":[1],"compressor":null,"dtype":"<f8","fill_value":null,"filters":null,"order":"C","shape":[1],"zarr_format":2}',
  'step/0': 'base64:AAAAAAAA8D8=',
  'step/.zattrs': '{"_ARRAY_DIMENSIONS":["step"],"long_name":"time since forecast_reference_time","standard_name":"forecast_period","units":"hours"}',
  'time/.zarray': '{"chunks":[1],"compressor":null,"dtype":"<i8","fill_value":null,"filters":null,"order":"C","shape":[1],"zarr_format":2}',
  'time/0': 'base64:gD6vZAAAAAA=',
  'time/.zattrs': '{"_ARRAY_DIMENSIONS":["time"],"calendar":"proleptic_gregorian","long_name":"initial time of forecast","standard_name":"forecast_reference_time","units":"seconds since 1970-01-01T00:00:00"}',
  't2m/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"t2m"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  't2m/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"air_temperature","cfVarName":"t2m","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"2 metre temperature","numberOfPoints":1905141,"paramId":167,"shortName":"2t","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"K"}',
  't2m/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   41989886,
   1246750],
  'pt/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"pt"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'pt/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"unknown","cfVarName":"pt","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"Potential temperature","numberOfPoints":1905141,"paramId":3,"shortName":"pt","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"K"}',
  'pt/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   43236636,
   1120958],
  'sh2/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"sh2"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'sh2/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"specific_humidity","cfVarName":"sh2","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"2 metre specific humidity","numberOfPoints":1905141,"paramId":174096,"shortName":"2sh","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"kg kg**-1"}',
  'sh2/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   44357594,
   1641762],
  'd2m/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"d2m"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'd2m/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"unknown","cfVarName":"d2m","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"2 metre dewpoint temperature","numberOfPoints":1905141,"paramId":168,"shortName":"2d","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"K"}',
  'd2m/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   45999356,
   1197583],
  'r2/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"r2"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'r2/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"relative_humidity","cfVarName":"r2","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"2 metre relative humidity","numberOfPoints":1905141,"paramId":260242,"shortName":"2r","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"%"}',
  'r2/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   47196939,
   1521015],
  'u10/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"u10"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'u10/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"eastward_wind","cfVarName":"u10","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"10 metre U wind component","numberOfPoints":1905141,"paramId":165,"shortName":"10u","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"m s**-1"}',
  'u10/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   49429793,
   2381615],
  'v10/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"v10"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'v10/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"northward_wind","cfVarName":"v10","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"10 metre V wind component","numberOfPoints":1905141,"paramId":166,"shortName":"10v","stepType":"instant","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"m s**-1"}',
  'v10/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   51811408,
   2381615],
  'si10/.zarray': '{"chunks":[1,1059,1799],"compressor":null,"dtype":"<f8","fill_value":3.4028234663852886e+38,"filters":[{"dtype":"float64","id":"grib","var":"si10"}],"order":"C","shape":[2,1059,1799],"zarr_format":2}',
  'si10/.zattrs': '{"NV":0,"_ARRAY_DIMENSIONS":["valid_time","y","x"],"cfName":"unknown","cfVarName":"si10","dataDate":20230713,"dataTime":0,"dataType":"fc","endStep":1,"gridDefinitionDescription":"Lambert Conformal can be secant or tangent, conical or bipolar","gridType":"lambert","missingValue":3.4028234663852886e+38,"name":"10 metre wind speed","numberOfPoints":1905141,"paramId":207,"shortName":"10si","stepType":"max","stepUnits":1,"typeOfLevel":"heightAboveGround","units":"m s**-1"}',
  'si10/0.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t00z.wrfsfcf01.grib2',
   54193023,
   1125590],
  'unknown/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   55195249,
   1287241],
  't2m/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   41728319,
   1237617],
  'pt/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   42965936,
   1134674],
  'sh2/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   44100610,
   1631008],
  'd2m/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   45731618,
   1184874],
  'r2/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   46916492,
   1520681],
  'u10/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   49302333,
   2381615],
  'v10/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   51683948,
   2381615],
  'si10/1.0.0': ['s3://noaa-hrrr-bdp-pds/hrrr.20230713/conus/hrrr.t01z.wrfsfcf01.grib2',
   54065563,
   1129686]}}

Write combined Kerchunk reference file to .json

# Write Kerchunk .json record
output_fname = "HRRR_combined.json"
with open(f"{output_fname}", "wb") as f:
    f.write(ujson.dumps(multi_kerchunk).encode())

Load Kerchunked dataset

# open dataset as zarr object using fsspec reference file system and Xarray
fs = fsspec.filesystem(
    "reference", fo=multi_kerchunk, remote_protocol="s3", remote_options={"anon": True}
)
m = fs.get_mapper("")
ds = xr.open_dataset(
    m, engine="zarr", backend_kwargs=dict(consolidated=False), chunks={"valid_time": 1}
)
ds
<xarray.Dataset>
Dimensions:            (valid_time: 2, y: 1059, x: 1799, heightAboveGround: 1,
                        step: 1, time: 1)
Coordinates:
  * heightAboveGround  (heightAboveGround) int64 2
  * step               (step) timedelta64[ns] 01:00:00
  * time               (time) datetime64[ns] 2023-07-13
  * valid_time         (valid_time) datetime64[ns] 2023-07-13T01:00:00 2023-0...
Dimensions without coordinates: y, x
Data variables:
    d2m                (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    latitude           (y, x) float64 dask.array<chunksize=(1059, 1799), meta=np.ndarray>
    longitude          (y, x) float64 dask.array<chunksize=(1059, 1799), meta=np.ndarray>
    pt                 (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    r2                 (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    sh2                (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    si10               (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    t2m                (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    u10                (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    unknown            (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
    v10                (valid_time, y, x) float64 dask.array<chunksize=(1, 1059, 1799), meta=np.ndarray>
Attributes:
    centre:             kwbc
    centreDescription:  US National Weather Service - NCEP
    edition:            2
    subCentre:          0

Plot a slice of the dataset

Here we are using Xarray to select a single time slice of the dataset and plot a temperature map of CONUS.

ds["t2m"][-1].plot()
<matplotlib.collections.QuadMesh at 0x7fa01b785a20>
../../_images/bf1c2197784c57ddae6a2ffa7a04b8c7ca239eba7dbfd9db2cbb12637eb58bdb.png
ds["t2m"][:, 500, 500].plot()
[<matplotlib.lines.Line2D at 0x7fa01b597310>]
../../_images/96908c833d93a2a87ba80680a93a26700dd5cf7fc872c598e3dd4c26c70509e7.png