diff --git a/doc/time-series.rst b/doc/time-series.rst index 47742360f62..b716d6cbc24 100644 --- a/doc/time-series.rst +++ b/doc/time-series.rst @@ -309,13 +309,19 @@ For data indexed by a :py:class:`~xarray.CFTimeIndex` xarray currently supports: da.differentiate('time') -- And serialization: +- Serialization: .. ipython:: python da.to_netcdf('example-no-leap.nc') xr.open_dataset('example-no-leap.nc') +- And resampling along the time dimension for data indexed by a :py:class:`~xarray.CFTimeIndex`: + +.. ipython:: python + + da.resample(time='81T', closed='right', label='right', base=3).mean() + .. note:: While much of the time series functionality that is possible for standard @@ -323,17 +329,14 @@ For data indexed by a :py:class:`~xarray.CFTimeIndex` xarray currently supports: still some remaining important features that have yet to be implemented, for example: - - Resampling along the time dimension for data indexed by a - :py:class:`~xarray.CFTimeIndex` (:issue:`2191`, :issue:`2458`) - Built-in plotting of data with :py:class:`cftime.datetime` coordinate axes (:issue:`2164`). For some use-cases it may still be useful to convert from a :py:class:`~xarray.CFTimeIndex` to a :py:class:`pandas.DatetimeIndex`, - despite the difference in calendar types (e.g. to allow the use of some - forms of resample with non-standard calendars). The recommended way of - doing this is to use the built-in - :py:meth:`~xarray.CFTimeIndex.to_datetimeindex` method: + despite the difference in calendar types. The recommended way of doing this + is to use the built-in :py:meth:`~xarray.CFTimeIndex.to_datetimeindex` + method: .. ipython:: python :okwarning: @@ -343,8 +346,7 @@ For data indexed by a :py:class:`~xarray.CFTimeIndex` xarray currently supports: da datetimeindex = da.indexes['time'].to_datetimeindex() da['time'] = datetimeindex - da.resample(time='Y').mean('time') - + However in this case one should use caution to only perform operations which do not depend on differences between dates (e.g. differentiation, interpolation, or upsampling with resample), as these could introduce subtle diff --git a/doc/whats-new.rst b/doc/whats-new.rst index 169950813fd..142ef3a2e51 100644 --- a/doc/whats-new.rst +++ b/doc/whats-new.rst @@ -47,6 +47,10 @@ Enhancements report showing what exactly differs between the two objects (dimensions / coordinates / variables / attributes) (:issue:`1507`). By `Benoit Bovy `_. +- Resampling of standard and non-standard calendars indexed by + :py:class:`~xarray.CFTimeIndex` is now possible. (:issue:`2191`). + By `Jwen Fai Low `_ and + `Spencer Clark `_. - Add ``tolerance`` option to ``resample()`` methods ``bfill``, ``pad``, ``nearest``. (:issue:`2695`) By `Hauke Schulz `_. diff --git a/xarray/coding/cftime_offsets.py b/xarray/coding/cftime_offsets.py index a373aeff747..d21139995dd 100644 --- a/xarray/coding/cftime_offsets.py +++ b/xarray/coding/cftime_offsets.py @@ -358,29 +358,41 @@ def rollback(self, date): class Day(BaseCFTimeOffset): _freq = 'D' + def as_timedelta(self): + return timedelta(days=self.n) + def __apply__(self, other): - return other + timedelta(days=self.n) + return other + self.as_timedelta() class Hour(BaseCFTimeOffset): _freq = 'H' + def as_timedelta(self): + return timedelta(hours=self.n) + def __apply__(self, other): - return other + timedelta(hours=self.n) + return other + self.as_timedelta() class Minute(BaseCFTimeOffset): _freq = 'T' + def as_timedelta(self): + return timedelta(minutes=self.n) + def __apply__(self, other): - return other + timedelta(minutes=self.n) + return other + self.as_timedelta() class Second(BaseCFTimeOffset): _freq = 'S' + def as_timedelta(self): + return timedelta(seconds=self.n) + def __apply__(self, other): - return other + timedelta(seconds=self.n) + return other + self.as_timedelta() _FREQUENCIES = { @@ -427,6 +439,11 @@ def __apply__(self, other): _FREQUENCY_CONDITION) +# pandas defines these offsets as "Tick" objects, which for instance have +# distinct behavior from monthly or longer frequencies in resample. +CFTIME_TICKS = (Day, Hour, Minute, Second) + + def to_offset(freq): """Convert a frequency string to the appropriate subclass of BaseCFTimeOffset.""" diff --git a/xarray/core/common.py b/xarray/core/common.py index ff0c7034d70..30ea56f3496 100644 --- a/xarray/core/common.py +++ b/xarray/core/common.py @@ -756,23 +756,16 @@ def resample(self, indexer=None, skipna=None, closed=None, label=None, dim_coord = self[dim] if isinstance(self.indexes[dim_name], CFTimeIndex): - raise NotImplementedError( - 'Resample is currently not supported along a dimension ' - 'indexed by a CFTimeIndex. For certain kinds of downsampling ' - 'it may be possible to work around this by converting your ' - 'time index to a DatetimeIndex using ' - 'CFTimeIndex.to_datetimeindex. Use caution when doing this ' - 'however, because switching to a DatetimeIndex from a ' - 'CFTimeIndex with a non-standard calendar entails a change ' - 'in the calendar type, which could lead to subtle and silent ' - 'errors.' - ) - + from .resample_cftime import CFTimeGrouper + grouper = CFTimeGrouper(freq, closed, label, base, loffset) + else: + # TODO: to_offset() call required for pandas==0.19.2 + grouper = pd.Grouper(freq=freq, closed=closed, label=label, + base=base, + loffset=pd.tseries.frequencies.to_offset( + loffset)) group = DataArray(dim_coord, coords=dim_coord.coords, dims=dim_coord.dims, name=RESAMPLE_DIM) - # TODO: to_offset() call required for pandas==0.19.2 - grouper = pd.Grouper(freq=freq, closed=closed, label=label, base=base, - loffset=pd.tseries.frequencies.to_offset(loffset)) resampler = self._resample_cls(self, group=group, dim=dim_name, grouper=grouper, resample_dim=RESAMPLE_DIM) diff --git a/xarray/core/groupby.py b/xarray/core/groupby.py index e4577c3d593..1fa1c159fbc 100644 --- a/xarray/core/groupby.py +++ b/xarray/core/groupby.py @@ -258,12 +258,8 @@ def __init__(self, obj, group, squeeze=False, grouper=None, bins=None, if not index.is_monotonic: # TODO: sort instead of raising an error raise ValueError('index must be monotonic for resampling') - s = pd.Series(np.arange(index.size), index) - first_items = s.groupby(grouper).first() - _apply_loffset(grouper, first_items) - full_index = first_items.index - if first_items.isnull().any(): - first_items = first_items.dropna() + full_index, first_items = self._get_index_and_items( + index, grouper) sbins = first_items.values.astype(np.int64) group_indices = ([slice(i, j) for i, j in zip(sbins[:-1], sbins[1:])] + @@ -310,6 +306,19 @@ def __len__(self): def __iter__(self): return zip(self._unique_coord.values, self._iter_grouped()) + def _get_index_and_items(self, index, grouper): + from .resample_cftime import CFTimeGrouper + s = pd.Series(np.arange(index.size), index) + if isinstance(grouper, CFTimeGrouper): + first_items = grouper.first_items(index) + else: + first_items = s.groupby(grouper).first() + _apply_loffset(grouper, first_items) + full_index = first_items.index + if first_items.isnull().any(): + first_items = first_items.dropna() + return full_index, first_items + def _iter_grouped(self): """Iterate over each element in this group""" for indices in self._group_indices: diff --git a/xarray/core/resample_cftime.py b/xarray/core/resample_cftime.py new file mode 100644 index 00000000000..6b6d214768e --- /dev/null +++ b/xarray/core/resample_cftime.py @@ -0,0 +1,363 @@ +"""Resampling for CFTimeIndex. Does not support non-integer freq.""" +# The mechanisms for resampling CFTimeIndex was copied and adapted from +# the source code defined in pandas.core.resample +# +# For reference, here is a copy of the pandas copyright notice: +# +# BSD 3-Clause License +# +# Copyright (c) 2008-2012, AQR Capital Management, LLC, Lambda Foundry, Inc. +# and PyData Development Team +# All rights reserved. +# +# Redistribution and use in source and binary forms, with or without +# modification, are permitted provided that the following conditions are met: +# +# * Redistributions of source code must retain the above copyright notice, this +# list of conditions and the following disclaimer. +# +# * Redistributions in binary form must reproduce the above copyright notice, +# this list of conditions and the following disclaimer in the documentation +# and/or other materials provided with the distribution. +# +# * Neither the name of the copyright holder nor the names of its +# contributors may be used to endorse or promote products derived from +# this software without specific prior written permission. +# +# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +# AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +# IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +# ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE +# LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +# CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +# SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +# INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +# CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +# ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +# POSSIBILITY OF SUCH DAMAGE. + +from ..coding.cftimeindex import CFTimeIndex +from ..coding.cftime_offsets import (cftime_range, normalize_date, + Day, MonthEnd, YearEnd, + CFTIME_TICKS, to_offset) +import datetime +import numpy as np +import pandas as pd + + +class CFTimeGrouper(object): + """This is a simple container for the grouping parameters that implements a + single method, the only one required for resampling in xarray. It cannot + be used in a call to groupby like a pandas.Grouper object can.""" + + def __init__(self, freq, closed, label, base, loffset): + self.freq = to_offset(freq) + self.closed = closed + self.label = label + self.base = base + self.loffset = loffset + + if isinstance(self.freq, (MonthEnd, YearEnd)): + if self.closed is None: + self.closed = 'right' + if self.label is None: + self.label = 'right' + else: + if self.closed is None: + self.closed = 'left' + if self.label is None: + self.label = 'left' + + def first_items(self, index): + """Meant to reproduce the results of the following + + grouper = pandas.Grouper(...) + first_items = pd.Series(np.arange(len(index)), + index).groupby(grouper).first() + + with index being a CFTimeIndex instead of a DatetimeIndex. + """ + + datetime_bins, labels = _get_time_bins(index, self.freq, self.closed, + self.label, self.base) + if self.loffset is not None: + if isinstance(self.loffset, datetime.timedelta): + labels = labels + self.loffset + else: + labels = labels + to_offset(self.loffset) + + # check binner fits data + if index[0] < datetime_bins[0]: + raise ValueError("Value falls before first bin") + if index[-1] > datetime_bins[-1]: + raise ValueError("Value falls after last bin") + + integer_bins = np.searchsorted( + index, datetime_bins, side=self.closed)[:-1] + first_items = pd.Series(integer_bins, labels) + + # Mask duplicate values with NaNs, preserving the last values + non_duplicate = ~first_items.duplicated('last') + return first_items.where(non_duplicate) + + +def _get_time_bins(index, freq, closed, label, base): + """Obtain the bins and their respective labels for resampling operations. + + Parameters + ---------- + index : CFTimeIndex + Index object to be resampled (e.g., CFTimeIndex named 'time'). + freq : xarray.coding.cftime_offsets.BaseCFTimeOffset + The offset object representing target conversion a.k.a. resampling + frequency (e.g., 'MS', '2D', 'H', or '3T' with + coding.cftime_offsets.to_offset() applied to it). + closed : 'left' or 'right', optional + Which side of bin interval is closed. + The default is 'left' for all frequency offsets except for 'M' and 'A', + which have a default of 'right'. + label : 'left' or 'right', optional + Which bin edge label to label bucket with. + The default is 'left' for all frequency offsets except for 'M' and 'A', + which have a default of 'right'. + base : int, optional + For frequencies that evenly subdivide 1 day, the "origin" of the + aggregated intervals. For example, for '5min' frequency, base could + range from 0 through 4. Defaults to 0. + + Returns + ------- + datetime_bins : CFTimeIndex + Defines the edge of resampling bins by which original index values will + be grouped into. + labels : CFTimeIndex + Define what the user actually sees the bins labeled as. + """ + + if not isinstance(index, CFTimeIndex): + raise TypeError('index must be a CFTimeIndex, but got ' + 'an instance of %r' % type(index).__name__) + if len(index) == 0: + datetime_bins = labels = CFTimeIndex(data=[], name=index.name) + return datetime_bins, labels + + first, last = _get_range_edges(index.min(), index.max(), freq, + closed=closed, + base=base) + datetime_bins = labels = cftime_range(freq=freq, + start=first, + end=last, + name=index.name) + + datetime_bins, labels = _adjust_bin_edges(datetime_bins, freq, closed, + index, labels) + + if label == 'right': + labels = labels[1:] + else: + labels = labels[:-1] + + # TODO: when CFTimeIndex supports missing values, if the reference index + # contains missing values, insert the appropriate NaN value at the + # beginning of the datetime_bins and labels indexes. + + return datetime_bins, labels + + +def _adjust_bin_edges(datetime_bins, offset, closed, index, labels): + """This is required for determining the bin edges resampling with + daily frequencies greater than one day, month end, and year end + frequencies. + + Consider the following example. Let's say you want to downsample the + time series with the following coordinates to month end frequency: + + CFTimeIndex([2000-01-01 12:00:00, 2000-01-31 12:00:00, + 2000-02-01 12:00:00], dtype='object') + + Without this adjustment, _get_time_bins with month-end frequency will + return the following index for the bin edges (default closed='right' and + label='right' in this case): + + CFTimeIndex([1999-12-31 00:00:00, 2000-01-31 00:00:00, + 2000-02-29 00:00:00], dtype='object') + + If 2000-01-31 is used as a bound for a bin, the value on + 2000-01-31T12:00:00 (at noon on January 31st), will not be included in the + month of January. To account for this, pandas adds a day minus one worth + of microseconds to the bin edges generated by cftime range, so that we do + bin the value at noon on January 31st in the January bin. This results in + an index with bin edges like the following: + + CFTimeIndex([1999-12-31 23:59:59, 2000-01-31 23:59:59, + 2000-02-29 23:59:59], dtype='object') + + The labels are still: + + CFTimeIndex([2000-01-31 00:00:00, 2000-02-29 00:00:00], dtype='object') + + This is also required for daily frequencies longer than one day and + year-end frequencies. + """ + is_super_daily = (isinstance(offset, (MonthEnd, YearEnd)) or + (isinstance(offset, Day) and offset.n > 1)) + if is_super_daily: + if closed == 'right': + datetime_bins = datetime_bins + datetime.timedelta(days=1, + microseconds=-1) + if datetime_bins[-2] > index.max(): + datetime_bins = datetime_bins[:-1] + labels = labels[:-1] + + return datetime_bins, labels + + +def _get_range_edges(first, last, offset, closed='left', base=0): + """ Get the correct starting and ending datetimes for the resampled + CFTimeIndex range. + + Parameters + ---------- + first : cftime.datetime + Uncorrected starting datetime object for resampled CFTimeIndex range. + Usually the min of the original CFTimeIndex. + last : cftime.datetime + Uncorrected ending datetime object for resampled CFTimeIndex range. + Usually the max of the original CFTimeIndex. + offset : xarray.coding.cftime_offsets.BaseCFTimeOffset + The offset object representing target conversion a.k.a. resampling + frequency. Contains information on offset type (e.g. Day or 'D') and + offset magnitude (e.g., n = 3). + closed : 'left' or 'right', optional + Which side of bin interval is closed. Defaults to 'left'. + base : int, optional + For frequencies that evenly subdivide 1 day, the "origin" of the + aggregated intervals. For example, for '5min' frequency, base could + range from 0 through 4. Defaults to 0. + + Returns + ------- + first : cftime.datetime + Corrected starting datetime object for resampled CFTimeIndex range. + last : cftime.datetime + Corrected ending datetime object for resampled CFTimeIndex range. + """ + if isinstance(offset, CFTIME_TICKS): + first, last = _adjust_dates_anchored(first, last, offset, + closed=closed, base=base) + return first, last + else: + first = normalize_date(first) + last = normalize_date(last) + + if closed == 'left': + first = offset.rollback(first) + else: + first = first - offset + + last = last + offset + return first, last + + +def _adjust_dates_anchored(first, last, offset, closed='right', base=0): + """ First and last offsets should be calculated from the start day to fix + an error cause by resampling across multiple days when a one day period is + not a multiple of the frequency. + See https://github.com/pandas-dev/pandas/issues/8683 + + Parameters + ---------- + first : cftime.datetime + A datetime object representing the start of a CFTimeIndex range. + last : cftime.datetime + A datetime object representing the end of a CFTimeIndex range. + offset : xarray.coding.cftime_offsets.BaseCFTimeOffset + The offset object representing target conversion a.k.a. resampling + frequency. Contains information on offset type (e.g. Day or 'D') and + offset magnitude (e.g., n = 3). + closed : 'left' or 'right', optional + Which side of bin interval is closed. Defaults to 'right'. + base : int, optional + For frequencies that evenly subdivide 1 day, the "origin" of the + aggregated intervals. For example, for '5min' frequency, base could + range from 0 through 4. Defaults to 0. + + Returns + ------- + fresult : cftime.datetime + A datetime object representing the start of a date range that has been + adjusted to fix resampling errors. + lresult : cftime.datetime + A datetime object representing the end of a date range that has been + adjusted to fix resampling errors. + """ + + base = base % offset.n + start_day = normalize_date(first) + base_td = type(offset)(n=base).as_timedelta() + start_day += base_td + foffset = exact_cftime_datetime_difference( + start_day, first) % offset.as_timedelta() + loffset = exact_cftime_datetime_difference( + start_day, last) % offset.as_timedelta() + if closed == 'right': + if foffset.total_seconds() > 0: + fresult = first - foffset + else: + fresult = first - offset.as_timedelta() + + if loffset.total_seconds() > 0: + lresult = last + (offset.as_timedelta() - loffset) + else: + lresult = last + else: + if foffset.total_seconds() > 0: + fresult = first - foffset + else: + fresult = first + + if loffset.total_seconds() > 0: + lresult = last + (offset.as_timedelta() - loffset) + else: + lresult = last + offset.as_timedelta() + return fresult, lresult + + +def exact_cftime_datetime_difference(a, b): + """Exact computation of b - a + + Assumes: + + a = a_0 + a_m + b = b_0 + b_m + + Here a_0, and b_0 represent the input dates rounded + down to the nearest second, and a_m, and b_m represent + the remaining microseconds associated with date a and + date b. + + We can then express the value of b - a as: + + b - a = (b_0 + b_m) - (a_0 + a_m) = b_0 - a_0 + b_m - a_m + + By construction, we know that b_0 - a_0 must be a round number + of seconds. Therefore we can take the result of b_0 - a_0 using + ordinary cftime.datetime arithmetic and round to the nearest + second. b_m - a_m is the remainder, in microseconds, and we + can simply add this to the rounded timedelta. + + Parameters + ---------- + a : cftime.datetime + Input datetime + b : cftime.datetime + Input datetime + + Returns + ------- + datetime.timedelta + """ + seconds = b.replace(microsecond=0) - a.replace(microsecond=0) + seconds = int(round(seconds.total_seconds())) + microseconds = b.microsecond - a.microsecond + return datetime.timedelta(seconds=seconds, microseconds=microseconds) diff --git a/xarray/tests/test_cftimeindex.py b/xarray/tests/test_cftimeindex.py index 358c9df0497..c5cdf0a3fee 100644 --- a/xarray/tests/test_cftimeindex.py +++ b/xarray/tests/test_cftimeindex.py @@ -375,12 +375,6 @@ def test_groupby(da): assert_identical(result, expected) -@pytest.mark.skipif(not has_cftime, reason='cftime not installed') -def test_resample_error(da): - with pytest.raises(NotImplementedError, match='to_datetimeindex'): - da.resample(time='Y') - - SEL_STRING_OR_LIST_TESTS = { 'string': '0001', 'string-slice': slice('0001-01-01', '0001-12-30'), # type: ignore diff --git a/xarray/tests/test_cftimeindex_resample.py b/xarray/tests/test_cftimeindex_resample.py new file mode 100644 index 00000000000..0b56f1d1fc6 --- /dev/null +++ b/xarray/tests/test_cftimeindex_resample.py @@ -0,0 +1,87 @@ +import pytest + +import datetime +import numpy as np +import pandas as pd +import xarray as xr + +pytest.importorskip('cftime') +pytest.importorskip('pandas', minversion='0.24') + + +@pytest.fixture( + params=[ + dict(start='2004-01-01T12:07:01', periods=91, freq='3D'), + dict(start='1892-01-03T12:07:01', periods=15, freq='41987T'), + dict(start='2004-01-01T12:07:01', periods=31, freq='2MS'), + dict(start='1892-01-03T12:07:01', periods=10, freq='3AS-JUN') + ], + ids=['3D', '41987T', '2MS', '3AS_JUN'] +) +def time_range_kwargs(request): + return request.param + + +@pytest.fixture() +def datetime_index(time_range_kwargs): + return pd.date_range(**time_range_kwargs) + + +@pytest.fixture() +def cftime_index(time_range_kwargs): + return xr.cftime_range(**time_range_kwargs) + + +def da(index): + return xr.DataArray(np.arange(100., 100. + index.size), + coords=[index], dims=['time']) + + +@pytest.mark.parametrize('freq', [ + '700T', '8001T', + '12H', '8001H', + '3D', '8D', '8001D', + '2MS', '2M', '3MS', '3M', '4MS', '4M', + '3AS', '3A', '4AS', '4A']) +@pytest.mark.parametrize('closed', [None, 'left', 'right']) +@pytest.mark.parametrize('label', [None, 'left', 'right']) +@pytest.mark.parametrize('base', [17, 24]) +def test_resampler(freq, closed, label, base, + datetime_index, cftime_index): + # Fairly extensive testing for standard/proleptic Gregorian calendar + loffset = '12H' + try: + da_datetime = da(datetime_index).resample( + time=freq, closed=closed, label=label, base=base, + loffset=loffset).mean() + except ValueError: + with pytest.raises(ValueError): + da(cftime_index).resample( + time=freq, closed=closed, label=label, base=base, + loffset=loffset).mean() + else: + da_cftime = da(cftime_index).resample(time=freq, closed=closed, + label=label, base=base, + loffset=loffset).mean() + da_cftime['time'] = da_cftime.indexes['time'].to_datetimeindex() + xr.testing.assert_identical(da_cftime, da_datetime) + + +@pytest.mark.parametrize('calendar', ['gregorian', 'noleap', 'all_leap', + '360_day', 'julian']) +def test_calendars(calendar): + # Limited testing for non-standard calendars + freq, closed, label, base = '81T', None, None, 17 + loffset = datetime.timedelta(hours=12) + xr_index = xr.cftime_range(start='2004-01-01T12:07:01', periods=7, + freq='3D', calendar=calendar) + pd_index = pd.date_range(start='2004-01-01T12:07:01', periods=7, + freq='3D') + da_cftime = da(xr_index).resample( + time=freq, closed=closed, label=label, base=base, loffset=loffset + ).mean() + da_datetime = da(pd_index).resample( + time=freq, closed=closed, label=label, base=base, loffset=loffset + ).mean() + da_cftime['time'] = da_cftime.indexes['time'].to_datetimeindex() + xr.testing.assert_identical(da_cftime, da_datetime) diff --git a/xarray/tests/test_dataarray.py b/xarray/tests/test_dataarray.py index 9465ce2f84a..906ebb278cc 100644 --- a/xarray/tests/test_dataarray.py +++ b/xarray/tests/test_dataarray.py @@ -2299,16 +2299,6 @@ def func(arg1, arg2, arg3=0.): actual = da.resample(time='D').apply(func, args=(1.,), arg3=1.) assert_identical(actual, expected) - @requires_cftime - def test_resample_cftimeindex(self): - cftime = _import_cftime() - times = cftime.num2date(np.arange(12), units='hours since 0001-01-01', - calendar='noleap') - array = DataArray(np.arange(12), [('time', times)]) - - with raises_regex(NotImplementedError, 'to_datetimeindex'): - array.resample(time='6H').mean() - def test_resample_first(self): times = pd.date_range('2000-01-01', freq='6H', periods=10) array = DataArray(np.arange(10), [('time', times)])