Skip to content

Commit

Permalink
add to_castra method to DataFrame
Browse files Browse the repository at this point in the history
This gives us a nice efficient store for DataFrames
  • Loading branch information
mrocklin committed Jul 7, 2015
1 parent 4373fbd commit ba40340
Show file tree
Hide file tree
Showing 2 changed files with 26 additions and 0 deletions.
16 changes: 16 additions & 0 deletions dask/dataframe/core.py
Expand Up @@ -629,6 +629,22 @@ def assign(self, **kwargs):

return elemwise(_assign, self, *pairs, columns=list(df2.columns))

def to_castra(self, fn=None, categories=None):
""" Write DataFrame to Castra on-disk store
See https://github.com/blosc/castra for details
See Also:
Castra.to_dask
"""
from castra import Castra
name = 'to-castra' + next(tokens)
dsk = {name: (Castra, fn, (self._name, 0), categories)}
dsk.update(dict(((name, i), (Castra.extend, name, (self._name, i)))
for i in range(self.npartitions)))
c, _ = get(merge(dsk, self.dask), [name, list(dsk.keys())])
return c


def _assign(df, *pairs):
kwargs = dict(partition(2, pairs))
Expand Down
10 changes: 10 additions & 0 deletions dask/dataframe/tests/test_io.py
Expand Up @@ -394,3 +394,13 @@ def test_from_dask_array_raises():
except Exception as e:
assert 'hello' in str(e)
assert '3' in str(e)


def test_to_castra():
pytest.importorskip('castra')
df = pd.DataFrame({'x': ['a', 'b', 'c', 'D'],
'y': [1, 2, 3, 4]})
a = dd.from_pandas(df, 2)

c = a.to_castra()
assert eq(a, c[:])

0 comments on commit ba40340

Please sign in to comment.