Skip to content

Commit 99d0695

Browse files
committed
Support Pandas 0.19.0
1 parent 3862b4d commit 99d0695

File tree

6 files changed

+30
-21
lines changed

6 files changed

+30
-21
lines changed

.travis.yml

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3,11 +3,11 @@ sudo: false
33

44
env:
55
matrix:
6-
- PYTHON=2.7 NUMPY=1.10.4 PANDAS=0.18.0 COVERAGE='true' XTRATESTARGS=
6+
- PYTHON=2.7 NUMPY=1.10.4 PANDAS=0.19.0 COVERAGE='true' XTRATESTARGS=
77
- PYTHON=2.7 NUMPY=1.11.0 PANDAS=0.18.1 COVERAGE='false' PYTHONOPTIMIZE=2 XTRATESTARGS=--ignore=dask/diagnostics
88
- PYTHON=3.3 NUMPY=1.9.2 PANDAS=0.18.1 COVERAGE='false' XTRATESTARGS=
99
- PYTHON=3.4 NUMPY=1.10.4 PANDAS=0.18.0 COVERAGE='false' PYTHONOPTIMIZE=2 XTRATESTARGS=--ignore=dask/diagnostics
10-
- PYTHON=3.5 NUMPY=1.11.0 PANDAS=0.18.1 COVERAGE='false' XTRATESTARGS=
10+
- PYTHON=3.5 NUMPY=1.11.0 PANDAS=0.19.0 COVERAGE='false' XTRATESTARGS=
1111

1212
addons:
1313
apt:

dask/dataframe/categorical.py

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -114,6 +114,10 @@ def strip_categories(df):
114114
else df.index)
115115

116116

117+
def iscategorical(dt):
118+
return isinstance(dt, pd.types.dtypes.CategoricalDtype)
119+
120+
117121
def get_categories(df):
118122
"""
119123
Get Categories of dataframe

dask/dataframe/core.py

Lines changed: 12 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22

33
from collections import Iterator
44
from copy import copy
5+
from distutils.version import LooseVersion
56
import operator
67
from operator import getitem, setitem
78
from pprint import pformat
@@ -1219,15 +1220,19 @@ def quantile(self, q=0.5, axis=0):
12191220
0 or 'index' for row-wise, 1 or 'columns' for column-wise
12201221
"""
12211222
axis = self._validate_axis(axis)
1222-
name = 'quantiles-concat--' + tokenize(self, q, axis)
1223+
keyname = 'quantiles-concat--' + tokenize(self, q, axis)
12231224

12241225
if axis == 1:
12251226
if isinstance(q, list):
12261227
# Not supported, the result will have current index as columns
12271228
raise ValueError("'q' must be scalar when axis=1 is specified")
1228-
meta = pd.Series([], dtype='f8')
1229+
if LooseVersion(pd.__version__) >= '0.19':
1230+
name = q
1231+
else:
1232+
name = None
1233+
meta = pd.Series([], dtype='f8', name=name)
12291234
return map_partitions(M.quantile, self, q, axis,
1230-
token=name, meta=meta)
1235+
token=keyname, meta=meta)
12311236
else:
12321237
meta = self._meta.quantile(q, axis=axis)
12331238
num = self._get_numeric_data()
@@ -1238,13 +1243,13 @@ def quantile(self, q=0.5, axis=0):
12381243
qnames = [(_q._name, 0) for _q in quantiles]
12391244

12401245
if isinstance(quantiles[0], Scalar):
1241-
dask[(name, 0)] = (pd.Series, (list, qnames), num.columns)
1246+
dask[(keyname, 0)] = (pd.Series, (list, qnames), num.columns)
12421247
divisions = (min(num.columns), max(num.columns))
1243-
return Series(dask, name, meta, divisions)
1248+
return Series(dask, keyname, meta, divisions)
12441249
else:
12451250
from .multi import _pdconcat
1246-
dask[(name, 0)] = (_pdconcat, (list, qnames), 1)
1247-
return DataFrame(dask, name, meta, quantiles[0].divisions)
1251+
dask[(keyname, 0)] = (_pdconcat, (list, qnames), 1)
1252+
return DataFrame(dask, keyname, meta, quantiles[0].divisions)
12481253

12491254
@derived_from(pd.DataFrame)
12501255
def describe(self):

dask/dataframe/indexing.py

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,6 @@ def _maybe_partial_time_string(index, indexer, kind):
8080
Convert indexer for partial string selection
8181
if data has DatetimeIndex/PeriodIndex
8282
"""
83-
8483
# do not pass dd.Index
8584
assert isinstance(index, pd.Index)
8685

dask/dataframe/io.py

Lines changed: 11 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -139,16 +139,16 @@ def from_pandas(data, npartitions=None, chunksize=None, sort=True, name=None):
139139
... index=pd.date_range(start='20100101', periods=6))
140140
>>> ddf = from_pandas(df, npartitions=3)
141141
>>> ddf.divisions # doctest: +NORMALIZE_WHITESPACE
142-
(Timestamp('2010-01-01 00:00:00', offset='D'),
143-
Timestamp('2010-01-03 00:00:00', offset='D'),
144-
Timestamp('2010-01-05 00:00:00', offset='D'),
145-
Timestamp('2010-01-06 00:00:00', offset='D'))
142+
(Timestamp('2010-01-01 00:00:00', freq='D'),
143+
Timestamp('2010-01-03 00:00:00', freq='D'),
144+
Timestamp('2010-01-05 00:00:00', freq='D'),
145+
Timestamp('2010-01-06 00:00:00', freq='D'))
146146
>>> ddf = from_pandas(df.a, npartitions=3) # Works with Series too!
147147
>>> ddf.divisions # doctest: +NORMALIZE_WHITESPACE
148-
(Timestamp('2010-01-01 00:00:00', offset='D'),
149-
Timestamp('2010-01-03 00:00:00', offset='D'),
150-
Timestamp('2010-01-05 00:00:00', offset='D'),
151-
Timestamp('2010-01-06 00:00:00', offset='D'))
148+
(Timestamp('2010-01-01 00:00:00', freq='D'),
149+
Timestamp('2010-01-03 00:00:00', freq='D'),
150+
Timestamp('2010-01-05 00:00:00', freq='D'),
151+
Timestamp('2010-01-06 00:00:00', freq='D'))
152152
153153
Raises
154154
------
@@ -554,7 +554,8 @@ def to_hdf(df, path_or_buf, key, mode='a', append=False, get=None,
554554

555555

556556
def _read_single_hdf(path, key, start=0, stop=None, columns=None,
557-
chunksize=int(1e6), sorted_index=False, lock=None, mode=None):
557+
chunksize=int(1e6), sorted_index=False, lock=None,
558+
mode='a'):
558559
"""
559560
Read a single hdf file into a dask.dataframe. Used for each file in
560561
read_hdf.
@@ -652,7 +653,7 @@ def _pd_read_hdf(path, key, lock, kwargs):
652653

653654
@wraps(pd.read_hdf)
654655
def read_hdf(pattern, key, start=0, stop=None, columns=None,
655-
chunksize=1000000, sorted_index=False, lock=True, mode=None):
656+
chunksize=1000000, sorted_index=False, lock=True, mode='a'):
656657
"""
657658
Read hdf files into a dask dataframe. Like pandas.read_hdf, except it we
658659
can read multiple files, and read multiple keys from the same file by using

dask/dataframe/tests/test_io.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1264,7 +1264,7 @@ def test_to_csv_simple():
12641264
df.to_csv(dir)
12651265
assert os.listdir(dir)
12661266
result = dd.read_csv(os.path.join(dir, '*')).compute()
1267-
assert (result.x == df0.x).all()
1267+
assert (result.x.values == df0.x.values).all()
12681268

12691269

12701270
def test_to_csv_series():

0 commit comments

Comments
 (0)