[jira] [Created] (ARROW-7486) Allow HDFS FileSystem to be created without Hadoop present

2019-12-31 Thread Matthew Rocklin (Jira)
Matthew Rocklin created ARROW-7486:
--

 Summary: Allow HDFS FileSystem to be created without Hadoop present
 Key: ARROW-7486
 URL: https://issues.apache.org/jira/browse/ARROW-7486
 Project: Apache Arrow
  Issue Type: Improvement
  Components: Python
Reporter: Matthew Rocklin


I would like to be able to construct an HDFS FileSystem object on a machine 
without Hadoop installed.  I don't need it to be able to actually do anything.  
I just need creating it to not fail.

This would enable Dask users to run computations on an HDFS enabled cluster 
from outside of that cluster.  This almost works today.  We send a small 
computation to a worker (which has HDFS access) to generate the task graph for 
loading data, and then we bring that task graph back to the local machine, 
continue building on it, and then finally submit everything off to the workers 
for execution.

The flaw here is when we bring back the task graph from the worker back to the 
client.  It contains a reference to a PyArrow HDFSFileSystem object, which upon 
de-serialization calls _maybe_set_hadoop_classpath().  I suspect that if this 
was allowed to fail that things would work out ok for us.  

Downstream issue originally reported here: 
https://github.com/dask/dask/issues/5758



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (ARROW-6926) [Python] Support __sizeof__ protocol for Python objects

2019-11-15 Thread Matthew Rocklin (Jira)


[ 
https://issues.apache.org/jira/browse/ARROW-6926?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16975268#comment-16975268
 ] 

Matthew Rocklin commented on ARROW-6926:


Someone ended up contributing these to Dask (we have a diispatch mechanism to 
work around these not being implemented upstream).  Obviously it would have 
been nicer for this code to be implemented in Arrow originally, but I thought 
I'd point to it here in case it's helpful to others.

https://github.com/dask/dask/blob/539d1e27a8ccce01de5f3d49f1748057c27552f2/dask/sizeof.py#L115-L145

> [Python] Support __sizeof__ protocol for Python objects
> ---
>
> Key: ARROW-6926
> URL: https://issues.apache.org/jira/browse/ARROW-6926
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Matthew Rocklin
>Priority: Minor
> Fix For: 1.0.0
>
>
> It would be helpful if PyArrow objects implemented the `__sizeof__` protocol 
> to give other libraries hints about how much data they have allocated.  This 
> helps systems like Dask, which have to make judgements about whether or not 
> something is cheap to move or taking up a large amount of space.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (ARROW-6926) Support __sizeof__ protocol for Python objects

2019-10-17 Thread Matthew Rocklin (Jira)
Matthew Rocklin created ARROW-6926:
--

 Summary: Support __sizeof__ protocol for Python objects
 Key: ARROW-6926
 URL: https://issues.apache.org/jira/browse/ARROW-6926
 Project: Apache Arrow
  Issue Type: Improvement
  Components: Python
Reporter: Matthew Rocklin


It would be helpful if PyArrow objects implemented the `__sizeof__` protocol to 
give other libraries hints about how much data they have allocated.  This helps 
systems like Dask, which have to make judgements about whether or not something 
is cheap to move or taking up a large amount of space.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (ARROW-5144) [Python] ParquetDataset and ParquetPiece not serializable

2019-04-22 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-5144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16823161#comment-16823161
 ] 

Matthew Rocklin commented on ARROW-5144:


That would be helpful, yes.  We're currently raising an error in master telling 
people to downgrade.  We get bug reports about this issue most days it seems.

> [Python] ParquetDataset and ParquetPiece not serializable
> -
>
> Key: ARROW-5144
> URL: https://issues.apache.org/jira/browse/ARROW-5144
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: parquet, Python
>Affects Versions: 0.13.0
> Environment: osx python36/conda cloudpickle 0.8.1
> arrow-cpp 0.13.0   py36ha71616b_0conda-forge
> pyarrow   0.13.0   py36hb37e6aa_0conda-forge
>Reporter: Martin Durant
>Assignee: Krisztian Szucs
>Priority: Critical
>  Labels: pull-request-available
> Fix For: 0.14.0
>
> Attachments: part.0.parquet
>
>  Time Spent: 3h 50m
>  Remaining Estimate: 0h
>
> Since 0.13.0, parquet instances are no longer serialisable, which means that 
> dask.distributed cannot pass them between processes in order to load parquet 
> in parallel.
> Example:
> ```
> >>> import cloudpickle
> >>> import pyarrow.parquet as pq
> >>> pf = pq.ParquetDataset('nation.impala.parquet')
> >>> cloudpickle.dumps(pf)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dumps(obj, protocol)
> 893 try:
> 894 cp = CloudPickler(file, protocol=protocol)
> --> 895 cp.dump(obj)
> 896 return file.getvalue()
> 897 finally:
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dump(self, obj)
> 266 self.inject_addons()
> 267 try:
> --> 268 return Pickler.dump(self, obj)
> 269 except RuntimeError as e:
> 270 if 'recursion' in e.args[0]:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in dump(self, obj)
> 407 if self.proto >= 4:
> 408 self.framer.start_framing()
> --> 409 self.save(obj)
> 410 self.write(STOP)
> 411 self.framer.end_framing()
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 519
> 520 # Save the reduce() output and finally memoize the object
> --> 521 self.save_reduce(obj=obj, *rv)
> 522
> 523 def persistent_id(self, obj):
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_reduce(self, func, args, 
> state, listitems, dictitems, obj)
> 632
> 633 if state is not None:
> --> 634 save(state)
> 635 write(BUILD)
> 636
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 474 f = self.dispatch.get(t)
> 475 if f is not None:
> --> 476 f(self, obj) # Call unbound method with explicit self
> 477 return
> 478
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_dict(self, obj)
> 819
> 820 self.memoize(obj)
> --> 821 self._batch_setitems(obj.items())
> 822
> 823 dispatch[dict] = save_dict
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in _batch_setitems(self, items)
> 845 for k, v in tmp:
> 846 save(k)
> --> 847 save(v)
> 848 write(SETITEMS)
> 849 elif n:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 494 reduce = getattr(obj, "__reduce_ex__", None)
> 495 if reduce is not None:
> --> 496 rv = reduce(self.proto)
> 497 else:
> 498 reduce = getattr(obj, "__reduce__", None)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/pyarrow/_parquet.cpython-36m-darwin.so
>  in pyarrow._parquet.ParquetSchema.__reduce_cython__()
> TypeError: no default __reduce__ due to non-trivial __cinit__
> ```
> The indicated schema instance is also referenced by the ParquetDatasetPiece s.
> ref: https://github.com/dask/distributed/issues/2597



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-4139) [Python] Cast Parquet column statistics to unicode if UTF8 ConvertedType is set

2019-04-19 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-4139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16822085#comment-16822085
 ] 

Matthew Rocklin commented on ARROW-4139:


I don't have strong thoughts about the API.  I mostly care that downstream 
projects don't have to special case a variety of types.  I expect that that 
special casing will be quite brittle and break over time.  Ideally we wouldn't 
be special casing these things in Arrow either.  

My guess is that there is already code somewhere in Arrow that knows how to 
convert these values in a consistent way.  Hopefully that same code would be 
engaged for statistics as well, that way we would have confidence that things 
wouldn't drift in the future.


> [Python] Cast Parquet column statistics to unicode if UTF8 ConvertedType is 
> set
> ---
>
> Key: ARROW-4139
> URL: https://issues.apache.org/jira/browse/ARROW-4139
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Reporter: Matthew Rocklin
>Priority: Minor
>  Labels: parquet, python
> Fix For: 0.14.0
>
>
> When writing Pandas data to Parquet format and reading it back again I find 
> that that statistics of text columns are stored as byte arrays rather than as 
> unicode text. 
> I'm not sure if this is a bug in Arrow, PyArrow, or just in my understanding 
> of how best to manage statistics.  (I'd be quite happy to learn that it was 
> the latter).
> Here is a minimal example
> {code:python}
> import pandas as pd
> df = pd.DataFrame({'x': ['a']})
> df.to_parquet('df.parquet')
> import pyarrow.parquet as pq
> pf = pq.ParquetDataset('df.parquet')
> piece = pf.pieces[0]
> rg = piece.row_group(0)
> md = piece.get_metadata(pq.ParquetFile)
> rg = md.row_group(0)
> c = rg.column(0)
> >>> c
> 
>   file_offset: 63
>   file_path: 
>   physical_type: BYTE_ARRAY
>   num_values: 1
>   path_in_schema: x
>   is_stats_set: True
>   statistics:
> 
>   has_min_max: True
>   min: b'a'
>   max: b'a'
>   null_count: 0
>   distinct_count: 0
>   num_values: 1
>   physical_type: BYTE_ARRAY
>   compression: SNAPPY
>   encodings: ('PLAIN_DICTIONARY', 'PLAIN', 'RLE')
>   has_dictionary_page: True
>   dictionary_page_offset: 4
>   data_page_offset: 25
>   total_compressed_size: 59
>   total_uncompressed_size: 55
> >>> type(c.statistics.min)
> bytes
> {code}
> My guess is that we would want to store a logical type in the statistics like 
> UNICODE, though I don't have enough experience with Parquet data types to 
> know if this is a good idea or possible.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-5144) [Python] ParquetDataset and ParquetPiece not serializable

2019-04-16 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-5144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16819024#comment-16819024
 ] 

Matthew Rocklin commented on ARROW-5144:


Most objects in Python are serializable by default.  My guess is that this 
object recently gained a non-serializable attribute, perhaps like an open file?

> [Python] ParquetDataset and ParquetPiece not serializable
> -
>
> Key: ARROW-5144
> URL: https://issues.apache.org/jira/browse/ARROW-5144
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Affects Versions: 0.13.0
> Environment: osx python36/conda cloudpickle 0.8.1
> arrow-cpp 0.13.0   py36ha71616b_0conda-forge
> pyarrow   0.13.0   py36hb37e6aa_0conda-forge
>Reporter: Martin Durant
>Assignee: Krisztian Szucs
>Priority: Critical
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> Since 0.13.0, parquet instances are no longer serialisable, which means that 
> dask.distributed cannot pass them between processes in order to load parquet 
> in parallel.
> Example:
> ```
> >>> import cloudpickle
> >>> import pyarrow.parquet as pq
> >>> pf = pq.ParquetDataset('nation.impala.parquet')
> >>> cloudpickle.dumps(pf)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dumps(obj, protocol)
> 893 try:
> 894 cp = CloudPickler(file, protocol=protocol)
> --> 895 cp.dump(obj)
> 896 return file.getvalue()
> 897 finally:
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dump(self, obj)
> 266 self.inject_addons()
> 267 try:
> --> 268 return Pickler.dump(self, obj)
> 269 except RuntimeError as e:
> 270 if 'recursion' in e.args[0]:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in dump(self, obj)
> 407 if self.proto >= 4:
> 408 self.framer.start_framing()
> --> 409 self.save(obj)
> 410 self.write(STOP)
> 411 self.framer.end_framing()
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 519
> 520 # Save the reduce() output and finally memoize the object
> --> 521 self.save_reduce(obj=obj, *rv)
> 522
> 523 def persistent_id(self, obj):
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_reduce(self, func, args, 
> state, listitems, dictitems, obj)
> 632
> 633 if state is not None:
> --> 634 save(state)
> 635 write(BUILD)
> 636
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 474 f = self.dispatch.get(t)
> 475 if f is not None:
> --> 476 f(self, obj) # Call unbound method with explicit self
> 477 return
> 478
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_dict(self, obj)
> 819
> 820 self.memoize(obj)
> --> 821 self._batch_setitems(obj.items())
> 822
> 823 dispatch[dict] = save_dict
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in _batch_setitems(self, items)
> 845 for k, v in tmp:
> 846 save(k)
> --> 847 save(v)
> 848 write(SETITEMS)
> 849 elif n:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 494 reduce = getattr(obj, "__reduce_ex__", None)
> 495 if reduce is not None:
> --> 496 rv = reduce(self.proto)
> 497 else:
> 498 reduce = getattr(obj, "__reduce__", None)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/pyarrow/_parquet.cpython-36m-darwin.so
>  in pyarrow._parquet.ParquetSchema.__reduce_cython__()
> TypeError: no default __reduce__ due to non-trivial __cinit__
> ```
> The indicated schema instance is also referenced by the ParquetDatasetPiece s.
> ref: https://github.com/dask/distributed/issues/2597



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-1983) [Python] Add ability to write parquet `_metadata` file

2019-04-15 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-1983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16818319#comment-16818319
 ] 

Matthew Rocklin commented on ARROW-1983:


My understanding is that there is already a standard around using a "_metadata" 
file that presumably is expected to have certain data laid out in a certain 
way.  It may be that [~mdurant] can provide a nice reference to the 
expectations.

It also looks like PyArrow has a nice reader for this information.  If I open 
up a Parquet Dataset that has a `_metadata` file I find that my object has all 
of the right information, so that might also be a good place to look.

> [Python] Add ability to write parquet `_metadata` file
> --
>
> Key: ARROW-1983
> URL: https://issues.apache.org/jira/browse/ARROW-1983
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: C++, Python
>Reporter: Jim Crist
>Priority: Major
>  Labels: beginner, parquet
> Fix For: 0.14.0
>
>
> Currently {{pyarrow.parquet}} can only write the {{_common_metadata}} file 
> (mostly just schema information). It would be useful to add the ability to 
> write a {{_metadata}} file as well. This should include information about 
> each row group in the dataset, including summary statistics. Having this 
> summary file would allow filtering of row groups without needing to access 
> each file beforehand.
> This would require that the user is able to get the written RowGroups out of 
> a {{pyarrow.parquet.write_table}} call and then give these objects as a list 
> to new function that then passes them on as C++ objects to {{parquet-cpp}} 
> that generates the respective {{_metadata}} file.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-5144) ParquetDataset and CloudParuqtePiece not serializable

2019-04-10 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-5144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16814535#comment-16814535
 ] 

Matthew Rocklin commented on ARROW-5144:


Hi everyone, 

This is pretty critical for Dask usage.  Anyone trying to use PyArrow 0.13 in a 
Dask workflow will break pretty hard here.  This isn't something that we can 
work around easily on our side.  It would be great to know if this is likely to 
be resolved quickly, or if we should warn users strongly away from 0.13.

In general, I recommend serialization tests for any project looking to interact 
with distributed computing libraries in Python.  Often this consists of tests 
like the following for any type that you think a parallel computing framework 
might want to interact with.

{code}
def test_serialization():
obj = MyObj()
obj2 = pickle.loads(pickle.dumps(obj))

assert obj == obj2
{code}

> ParquetDataset and CloudParuqtePiece not serializable
> -
>
> Key: ARROW-5144
> URL: https://issues.apache.org/jira/browse/ARROW-5144
> Project: Apache Arrow
>  Issue Type: Bug
>Affects Versions: 0.13.0
> Environment: osx python36/conda cloudpickle 0.8.1
> arrow-cpp 0.13.0   py36ha71616b_0conda-forge
> pyarrow   0.13.0   py36hb37e6aa_0conda-forge
>Reporter: Martin Durant
>Priority: Major
>
> Since 0.13.0, parquet instances are no longer serialisable, which means that 
> dask.distributed cannot pass them between processes in order to load parquet 
> in parallel.
> Example:
> ```
> >>> import cloudpickle
> >>> import pyarrow.parquet as pq
> >>> pf = pq.ParquetDataset('nation.impala.parquet')
> >>> cloudpickle.dumps(pf)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dumps(obj, protocol)
> 893 try:
> 894 cp = CloudPickler(file, protocol=protocol)
> --> 895 cp.dump(obj)
> 896 return file.getvalue()
> 897 finally:
> ~/anaconda/envs/py36/lib/python3.6/site-packages/cloudpickle/cloudpickle.py 
> in dump(self, obj)
> 266 self.inject_addons()
> 267 try:
> --> 268 return Pickler.dump(self, obj)
> 269 except RuntimeError as e:
> 270 if 'recursion' in e.args[0]:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in dump(self, obj)
> 407 if self.proto >= 4:
> 408 self.framer.start_framing()
> --> 409 self.save(obj)
> 410 self.write(STOP)
> 411 self.framer.end_framing()
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 519
> 520 # Save the reduce() output and finally memoize the object
> --> 521 self.save_reduce(obj=obj, *rv)
> 522
> 523 def persistent_id(self, obj):
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_reduce(self, func, args, 
> state, listitems, dictitems, obj)
> 632
> 633 if state is not None:
> --> 634 save(state)
> 635 write(BUILD)
> 636
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 474 f = self.dispatch.get(t)
> 475 if f is not None:
> --> 476 f(self, obj) # Call unbound method with explicit self
> 477 return
> 478
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save_dict(self, obj)
> 819
> 820 self.memoize(obj)
> --> 821 self._batch_setitems(obj.items())
> 822
> 823 dispatch[dict] = save_dict
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in _batch_setitems(self, items)
> 845 for k, v in tmp:
> 846 save(k)
> --> 847 save(v)
> 848 write(SETITEMS)
> 849 elif n:
> ~/anaconda/envs/py36/lib/python3.6/pickle.py in save(self, obj, 
> save_persistent_id)
> 494 reduce = getattr(obj, "__reduce_ex__", None)
> 495 if reduce is not None:
> --> 496 rv = reduce(self.proto)
> 497 else:
> 498 reduce = getattr(obj, "__reduce__", None)
> ~/anaconda/envs/py36/lib/python3.6/site-packages/pyarrow/_parquet.cpython-36m-darwin.so
>  in pyarrow._parquet.ParquetSchema.__reduce_cython__()
> TypeError: no default __reduce__ due to non-trivial __cinit__
> ```
> The indicated schema instance is also referenced by the ParquetDatasetPiece s.
> ref: https://github.com/dask/distributed/issues/2597



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-4139) [Python] Cast Parquet column statistics to unicode if UTF8 ConvertedType is set

2019-03-26 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-4139?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16802291#comment-16802291
 ] 

Matthew Rocklin commented on ARROW-4139:


Perhaps relatedly, it would be useful to get physical types for statistics of 
other types.  Here is another issue I ran into with timestamps

{code}
In [1]: import pandas as pd

In [2]: df = pd.util.testing.makeTimeDataFrame()

In [3]: df.head()
Out[3]:
   A B C D
2000-01-03  1.255856 -1.092558 -1.454595  0.898535
2000-01-04 -1.006590  0.640467 -2.249877  0.068293
2000-01-05 -1.525559  0.567070  1.039230 -0.967301
2000-01-06 -0.773395 -1.565619  0.025786  0.106949
2000-01-07 -0.079000  0.367165  1.746211 -0.097441

In [4]: df.to_parquet('foo.parquet')

In [5]: import pyarrow.parquet as pq

In [6]: p = pq.ParquetDataset('foo.parquet')

In [7]: piece = p.pieces[0]

In [8]: md = piece.get_metadata(open_file_func=lambda fn: open(fn, mode='rb'))

In [9]: rg = md.row_group(0)

In [10]: rg.column(4)
Out[10]:

  file_offset: 1913
  file_path:
  physical_type: INT64
  num_values: 30
  path_in_schema: __index_level_0__
  is_stats_set: True
  statistics:

  has_min_max: True
  min: 94685760
  max: 95022720
  null_count: 0
  distinct_count: 0
  num_values: 30
  physical_type: INT64
  compression: SNAPPY
  encodings: ('PLAIN_DICTIONARY', 'PLAIN', 'RLE')
  has_dictionary_page: True
  dictionary_page_offset: 1635
  data_page_offset: 1842
  total_compressed_size: 278
  total_uncompressed_size: 325

In [11]: rg.column(4).statistics.min  # I want this to be some sort of 
timestamp object
Out[11]: 94685760
{code}

Somewhat unrelatedly, there is a lot of boiler plate to get down to that 
information.  If there are nicer ways to get to statistics I'd be interested in 
hearing about them.

> [Python] Cast Parquet column statistics to unicode if UTF8 ConvertedType is 
> set
> ---
>
> Key: ARROW-4139
> URL: https://issues.apache.org/jira/browse/ARROW-4139
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Reporter: Matthew Rocklin
>Priority: Minor
>  Labels: parquet, python
> Fix For: 0.14.0
>
>
> When writing Pandas data to Parquet format and reading it back again I find 
> that that statistics of text columns are stored as byte arrays rather than as 
> unicode text. 
> I'm not sure if this is a bug in Arrow, PyArrow, or just in my understanding 
> of how best to manage statistics.  (I'd be quite happy to learn that it was 
> the latter).
> Here is a minimal example
> {code:python}
> import pandas as pd
> df = pd.DataFrame({'x': ['a']})
> df.to_parquet('df.parquet')
> import pyarrow.parquet as pq
> pf = pq.ParquetDataset('df.parquet')
> piece = pf.pieces[0]
> rg = piece.row_group(0)
> md = piece.get_metadata(pq.ParquetFile)
> rg = md.row_group(0)
> c = rg.column(0)
> >>> c
> 
>   file_offset: 63
>   file_path: 
>   physical_type: BYTE_ARRAY
>   num_values: 1
>   path_in_schema: x
>   is_stats_set: True
>   statistics:
> 
>   has_min_max: True
>   min: b'a'
>   max: b'a'
>   null_count: 0
>   distinct_count: 0
>   num_values: 1
>   physical_type: BYTE_ARRAY
>   compression: SNAPPY
>   encodings: ('PLAIN_DICTIONARY', 'PLAIN', 'RLE')
>   has_dictionary_page: True
>   dictionary_page_offset: 4
>   data_page_offset: 25
>   total_compressed_size: 59
>   total_uncompressed_size: 55
> >>> type(c.statistics.min)
> bytes
> {code}
> My guess is that we would want to store a logical type in the statistics like 
> UNICODE, though I don't have enough experience with Parquet data types to 
> know if this is a good idea or possible.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3244) [Python] Multi-file parquet loading without scan

2019-03-26 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16802274#comment-16802274
 ] 

Matthew Rocklin commented on ARROW-3244:


Is this resolved today by the ParquetDataset.metadata attribute?

{code}
pq.ParquetDataset(path).metadata
{code}

> [Python] Multi-file parquet loading without scan
> 
>
> Key: ARROW-3244
> URL: https://issues.apache.org/jira/browse/ARROW-3244
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Martin Durant
>Priority: Major
>  Labels: parquet
> Fix For: 0.14.0
>
>
> A number of mechanism are possible to avoid having to access and read the 
> parquet footers in a data set consisting of a number of files. In the case of 
> a large number of data files (perhaps split with directory partitioning) and 
> remote storage, this can be a significant overhead. This is significant from 
> the point of view of Dask, which must have the metadata available in the 
> client before setting up computational graphs.
>  
> Here are some suggestions of what could be done.
>  
>  * some parquet writing frameworks include a `_metadata` file, which contains 
> all the information from the footers of the various files. If this file is 
> present, then this data can be read from one place, with a single file 
> access. For a large number of files, parsing the thrift information may, by 
> itself, be a non-negligible overhead≥
>  * the schema (dtypes) can be found in a `_common_metadata`, or from any one 
> of the data-files, then the schema could be assumed (perhaps at the user's 
> option) to be the same for all of the files. However, the information about 
> the directory partitioning would not be available. Although Dask may infer 
> the information from the filenames, it would be preferable to go through the 
> machinery with parquet-cpp, and view the whole data-set as a single object. 
> Note that the files will still need to have the footer read to access the 
> data, for the bytes offsets, but from Dask's point of view, this would be 
> deferred to tasks running in parallel.
> (please forgive that some of this has already been mentioned elsewhere; this 
> is one of the entries in the list at 
> [https://github.com/dask/fastparquet/issues/374] as a feature that is useful 
> in fastparquet)
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-1983) [Python] Add ability to write parquet `_metadata` file

2019-03-01 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-1983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16782163#comment-16782163
 ] 

Matthew Rocklin commented on ARROW-1983:


Hi all, thought I would check in here.  I'll likely start planning work around 
Dask Parquet reader/writer functionality soon, and am curious is there is any 
timeline on this issue.  "Nope" is a totally fine answer, just looking for 
information for planning purposes. 

> [Python] Add ability to write parquet `_metadata` file
> --
>
> Key: ARROW-1983
> URL: https://issues.apache.org/jira/browse/ARROW-1983
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Jim Crist
>Assignee: Robbie Gruener
>Priority: Major
>  Labels: beginner, parquet
> Fix For: 0.13.0
>
>
> Currently {{pyarrow.parquet}} can only write the {{_common_metadata}} file 
> (mostly just schema information). It would be useful to add the ability to 
> write a {{_metadata}} file as well. This should include information about 
> each row group in the dataset, including summary statistics. Having this 
> summary file would allow filtering of row groups without needing to access 
> each file beforehand.
> This would require that the user is able to get the written RowGroups out of 
> a {{pyarrow.parquet.write_table}} call and then give these objects as a list 
> to new function that then passes them on as C++ objects to {{parquet-cpp}} 
> that generates the respective {{_metadata}} file.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-1983) [Python] Add ability to write parquet `_metadata` file

2019-01-24 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-1983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16751500#comment-16751500
 ] 

Matthew Rocklin commented on ARROW-1983:


In https://github.com/dask/dask/issues/4410 we learn that metadata information 
can grow to be large in the case where there are many columns and many 
partitions.  There is some value to ensuring that the metadata results are 
somewhat compact in memory, though I also wouldn't spend a ton of effort 
optimizing here.

> [Python] Add ability to write parquet `_metadata` file
> --
>
> Key: ARROW-1983
> URL: https://issues.apache.org/jira/browse/ARROW-1983
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Jim Crist
>Assignee: Robert Gruener
>Priority: Major
>  Labels: beginner, parquet
> Fix For: 0.13.0
>
>
> Currently {{pyarrow.parquet}} can only write the {{_common_metadata}} file 
> (mostly just schema information). It would be useful to add the ability to 
> write a {{_metadata}} file as well. This should include information about 
> each row group in the dataset, including summary statistics. Having this 
> summary file would allow filtering of row groups without needing to access 
> each file beforehand.
> This would require that the user is able to get the written RowGroups out of 
> a {{pyarrow.parquet.write_table}} call and then give these objects as a list 
> to new function that then passes them on as C++ objects to {{parquet-cpp}} 
> that generates the respective {{_metadata}} file.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-4266) [Python][CI] Disable ORC tests in dask integration test

2019-01-15 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-4266?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16743163#comment-16743163
 ] 

Matthew Rocklin commented on ARROW-4266:


cc [~jcrist] who has handled ORC previously

> [Python][CI] Disable ORC tests in dask integration test
> ---
>
> Key: ARROW-4266
> URL: https://issues.apache.org/jira/browse/ARROW-4266
> Project: Apache Arrow
>  Issue Type: Task
>Reporter: Krisztian Szucs
>Assignee: Krisztian Szucs
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> https://issues.apache.org/jira/browse/ARROW-3910 changed the default value of 
> to_pandas: to_pandas(date_as_object=True) which breaks dask's ORC tests 
> [https://github.com/dask/dask/blob/e48aca49af9005c938ff4773aa05ca8b20e2e1b1/dask/dataframe/io/orc.py#L19]
>  
> cc [~mrocklin]
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ARROW-4139) Parquet Statistics on unicode text files have byte array type

2018-12-31 Thread Matthew Rocklin (JIRA)
Matthew Rocklin created ARROW-4139:
--

 Summary: Parquet Statistics on unicode text files have byte array 
type
 Key: ARROW-4139
 URL: https://issues.apache.org/jira/browse/ARROW-4139
 Project: Apache Arrow
  Issue Type: Bug
Reporter: Matthew Rocklin


When writing Pandas data to Parquet format and reading it back again I find 
that that statistics of text columns are stored as byte arrays rather than as 
unicode text. 

I'm not sure if this is a bug in Arrow, PyArrow, or just in my understanding of 
how best to manage statistics.  (I'd be quite happy to learn that it was the 
latter).

Here is a minimal example

{code:python}
import pandas as pd
df = pd.DataFrame({'x': ['a']})
df.to_parquet('df.parquet')
import pyarrow.parquet as pq
pf = pq.ParquetDataset('df.parquet')
piece = pf.pieces[0]
rg = piece.row_group(0)
md = piece.get_metadata(pq.ParquetFile)
rg = md.row_group(0)
c = rg.column(0)

>>> c

  file_offset: 63
  file_path: 
  physical_type: BYTE_ARRAY
  num_values: 1
  path_in_schema: x
  is_stats_set: True
  statistics:

  has_min_max: True
  min: b'a'
  max: b'a'
  null_count: 0
  distinct_count: 0
  num_values: 1
  physical_type: BYTE_ARRAY
  compression: SNAPPY
  encodings: ('PLAIN_DICTIONARY', 'PLAIN', 'RLE')
  has_dictionary_page: True
  dictionary_page_offset: 4
  data_page_offset: 25
  total_compressed_size: 59
  total_uncompressed_size: 55

>>> type(c.statistics.min)
bytes
{code}

My guess is that we would want to store a logical type in the statistics like 
UNICODE, though I don't have enough experience with Parquet data types to know 
if this is a good idea or possible.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-1989) [Python] Better UX on timestamp conversion to Pandas

2018-12-31 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-1989?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16731491#comment-16731491
 ] 

Matthew Rocklin commented on ARROW-1989:


I would find this fix useful.  

In particular I think that it might be useful to point users towards possible 
solutions, like the `allow_truncated_timestamps=True` option.

> [Python] Better UX on timestamp conversion to Pandas
> 
>
> Key: ARROW-1989
> URL: https://issues.apache.org/jira/browse/ARROW-1989
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Uwe L. Korn
>Priority: Major
> Fix For: 0.13.0
>
>
> Converting timestamp columns to Pandas, users often have the problem that 
> they have dates that are larger than Pandas can represent with their 
> nanosecond representation. Currently they simply see an Arrow exception and 
> think that this problem is caused by Arrow. We should try to change the error 
> from
> {code}
> ArrowInvalid: Casting from timestamp[ns] to timestamp[us] would lose data: XX
> {code}
> to something along the lines of 
> {code}
> ArrowInvalid: Casting from timestamp[ns] to timestamp[us] would lose data: 
> XX. This conversion is needed as Pandas does only support nanosecond 
> timestamps. Your data is likely out of the range that can be represented with 
> nanosecond resolution.
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3652) [Python] CategoricalIndex is lost after reading back

2018-12-31 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16731485#comment-16731485
 ] 

Matthew Rocklin commented on ARROW-3652:


Right, I've just done a search on this topic and found a few other issues.  
Happy to move conversation to one of them if desired.

In this particular case we already have pandas-specific metadata telling us 
that we have a categorical column (this is in the `pandas_type` entry for the 
column).  As you say we don't know the category-mapping, but we could still 
create a Categorical column and just let Pandas decide the mapping.  I suspect 
that it sorts the entries as a sane default..  This wouldn't necessarily 
roundtrip the right category-to-code mapping, but we can at least roundtrip the 
fact that these should be represented as categoricals.

Speaking from a Dask Dataframe perspective this would be desirable.  We're 
accustomed to having to remap mismatched categorical columns to make things 
match up.  Pandas generally handles this case well today.  However, I can 
understand if Arrow wants to play things safe here and resist the temptation to 
let Pandas be magical.

 

So, the concrete ask here is that when converting a table to a Pandas dataframe 
and using the `use_pandas_metadata=` keyword that we convert columns that 
Pandas marked with `pandas_type='category'` to be categorical columns.

> [Python] CategoricalIndex is lost after reading back
> 
>
> Key: ARROW-3652
> URL: https://issues.apache.org/jira/browse/ARROW-3652
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Affects Versions: 0.11.1
>Reporter: Armin Berres
>Priority: Major
>  Labels: parquet
>
> When a {{CategoricalIndex}} is written and read back the resulting index is 
> not more categorical.
> {code}
> df = pd.DataFrame([['a', 'b'], ['c', 'd']], columns=['c1', 'c2'])
> df['c1'] = df['c1'].astype('category')
> df = df.set_index(['c1'])
> table = pa.Table.from_pandas(df)
> pq.write_table(table, 'test.parquet')
> ref_df = pq.read_pandas('test.parquet').to_pandas()
> print(df.index)
> # CategoricalIndex(['a', 'c'], categories=['a', 'c'], ordered=False, 
> name='c1', dtype='category')
> print(ref_df.index)
> # Index(['a', 'c'], dtype='object', name='c1')
> {code}
> In the metadata the information is correctly contained:
> {code:java}
> {"name": "c1", "field_name": "c1", "p'
> b'andas_type": "categorical", "numpy_type": "int8", "metadata": 
> {"'
> b'num_categories": 2, "ordered": false}
> {code}
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3652) [Python] CategoricalIndex is lost after reading back

2018-12-31 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16731477#comment-16731477
 ] 

Matthew Rocklin commented on ARROW-3652:


This is lost for categorical columns generally, not only the index


{code:python}
import pandas as pd
df = pd.DataFrame({'x': pd.Categorical(['a'])})
df.to_parquet('df.parquet')

import pyarrow.parquet as pq
table = pq.read_table('df.parquet', use_pandas_metadata=True)

>>> df.dtypes
xcategory
dtype: object

>>> table.to_pandas().dtypes
xobject
dtype: object

>>> table
pyarrow.Table
x: string
__index_level_0__: int64
metadata

{b'pandas': b'{"index_columns": ["__index_level_0__"], "column_indexes": [{"na'
b'me": null, "field_name": null, "pandas_type": "unicode", "numpy_'
b'type": "object", "metadata": {"encoding": "UTF-8"}}], "columns":'
b' [{"name": "x", "field_name": "x", "pandas_type": "categorical",'
b' "numpy_type": "int8", "metadata": {"num_categories": 1, "ordere'
b'd": false}}, {"name": null, "field_name": "__index_level_0__", "'
b'pandas_type": "int64", "numpy_type": "int64", "metadata": null}]'
b', "pandas_version": "0.23.4"}'}
{code}


> [Python] CategoricalIndex is lost after reading back
> 
>
> Key: ARROW-3652
> URL: https://issues.apache.org/jira/browse/ARROW-3652
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Affects Versions: 0.11.1
>Reporter: Armin Berres
>Priority: Major
>  Labels: parquet
>
> When a {{CategoricalIndex}} is written and read back the resulting index is 
> not more categorical.
> {code}
> df = pd.DataFrame([['a', 'b'], ['c', 'd']], columns=['c1', 'c2'])
> df['c1'] = df['c1'].astype('category')
> df = df.set_index(['c1'])
> table = pa.Table.from_pandas(df)
> pq.write_table(table, 'test.parquet')
> ref_df = pq.read_pandas('test.parquet').to_pandas()
> print(df.index)
> # CategoricalIndex(['a', 'c'], categories=['a', 'c'], ordered=False, 
> name='c1', dtype='category')
> print(ref_df.index)
> # Index(['a', 'c'], dtype='object', name='c1')
> {code}
> In the metadata the information is correctly contained:
> {code:java}
> {"name": "c1", "field_name": "c1", "p'
> b'andas_type": "categorical", "numpy_type": "int8", "metadata": 
> {"'
> b'num_categories": 2, "ordered": false}
> {code}
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-1983) [Python] Add ability to write parquet `_metadata` file

2018-12-31 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-1983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16731467#comment-16731467
 ] 

Matthew Rocklin commented on ARROW-1983:


>  I'm planning to move more of the multifile dataset handling into C++ because 
> we also need it in Ruby and R, so would make sense to maintain one 
> implementation for the 3 languages

Makes sense to me.  No pressure, but is there a time in particular when you're 
planning to do this?  This will help me with planning on the Dask side.  I'm 
also happy to help with things on the Python Arrow side near term if they come 
up.  

For context see https://github.com/dask/dask/pull/4336#issuecomment-450686100

> [Python] Add ability to write parquet `_metadata` file
> --
>
> Key: ARROW-1983
> URL: https://issues.apache.org/jira/browse/ARROW-1983
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Jim Crist
>Assignee: Robert Gruener
>Priority: Major
>  Labels: beginner, parquet
> Fix For: 0.13.0
>
>
> Currently {{pyarrow.parquet}} can only write the {{_common_metadata}} file 
> (mostly just schema information). It would be useful to add the ability to 
> write a {{_metadata}} file as well. This should include information about 
> each row group in the dataset, including summary statistics. Having this 
> summary file would allow filtering of row groups without needing to access 
> each file beforehand.
> This would require that the user is able to get the written RowGroups out of 
> a {{pyarrow.parquet.write_table}} call and then give these objects as a list 
> to new function that then passes them on as C++ objects to {{parquet-cpp}} 
> that generates the respective {{_metadata}} file.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-1983) [Python] Add ability to write parquet `_metadata` file

2018-12-29 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-1983?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16730844#comment-16730844
 ] 

Matthew Rocklin commented on ARROW-1983:


> If I understand correctly, we need to combine all of the row group metadata 
> for all files in a directory.

Yes.  Ideally when writing a row group we would get some metadata object in 
memory. We would then collect all of those objects and hand them to some 
`write_metadata` function afterwards.

> When a new file is written, does this file have to be updated?
 
Yes, or it can be removed/invalidated.
 
As a side note, this is probably one of a small number of issues that stop Dask 
Dataframe from using PyArrow by default.  Metadata files with full row group 
information are especially valuable for us, particularly with remote/cloud 
storage.  (I'm going through Dask's parquet handling now)

> [Python] Add ability to write parquet `_metadata` file
> --
>
> Key: ARROW-1983
> URL: https://issues.apache.org/jira/browse/ARROW-1983
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Jim Crist
>Assignee: Robert Gruener
>Priority: Major
>  Labels: beginner, parquet
> Fix For: 0.13.0
>
>
> Currently {{pyarrow.parquet}} can only write the {{_common_metadata}} file 
> (mostly just schema information). It would be useful to add the ability to 
> write a {{_metadata}} file as well. This should include information about 
> each row group in the dataset, including summary statistics. Having this 
> summary file would allow filtering of row groups without needing to access 
> each file beforehand.
> This would require that the user is able to get the written RowGroups out of 
> a {{pyarrow.parquet.write_table}} call and then give these objects as a list 
> to new function that then passes them on as C++ objects to {{parquet-cpp}} 
> that generates the respective {{_metadata}} file.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3280) [Python] Difficulty running tests after conda install

2018-09-21 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16623810#comment-16623810
 ] 

Matthew Rocklin commented on ARROW-3280:


Yeah, given the error I thought that as well, but updating didn't seem to do 
much good.
{code:java}
mrocklin@carbon:~/workspace/arrow/python$ python -c "import setuptools; 
print(setuptools.__version__)"
40.4.1
{code}

> [Python] Difficulty running tests after conda install
> -
>
> Key: ARROW-3280
> URL: https://issues.apache.org/jira/browse/ARROW-3280
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Affects Versions: 0.10.0
> Environment: conda create -n test-arrow pytest ipython pandas nomkl 
> pyarrow -c conda-forge
> Ubuntu 16.04
>Reporter: Matthew Rocklin
>Priority: Minor
>  Labels: python
>
> I install PyArrow from conda-forge, and then try running tests (or import 
> generally)
> {code:java}
> conda create -n test-arrow pytest ipython pandas nomkl pyarrow -c conda-forge 
> {code}
> {code:java}
> mrocklin@carbon:~/workspace/arrow/python$ py.test 
> pyarrow/tests/test_parquet.py 
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 328, in _getconftestmodules
> return self._path2confmods[path]
> KeyError: 
> local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/test_parquet.py')During
>  handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 328, in _getconftestmodules
> return self._path2confmods[path]
> KeyError: local('/home/mrocklin/workspace/arrow/python/pyarrow/tests')During 
> handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 359, in _importconftest
> return self._conftestpath2mod[conftestpath]
> KeyError: 
> local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py')During
>  handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 365, in _importconftest
> mod = conftestpath.pyimport()
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/py/_path/local.py",
>  line 668, in pyimport
> __import__(modname)
> File "/home/mrocklin/workspace/arrow/python/pyarrow/__init__.py", line 54, in 
> 
> from pyarrow.lib import cpu_count, set_cpu_count
> ModuleNotFoundError: No module named 'pyarrow.lib'
> ERROR: could not load 
> /home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py{code}
> Probably this is something wrong with my environment, but I thought I'd 
> report it as a usability bug



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3280) [Python] Difficulty running tests after conda install

2018-09-20 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16622941#comment-16622941
 ] 

Matthew Rocklin commented on ARROW-3280:


I got as far as the following.  My apologies for the rambling issue.  Feel free 
to close.  I'm also happy to reopen something else a bit cleaner.
{code:java}
mrocklin@carbon:~/workspace/arrow/python$ python setup.py build_ext 
--build-type=$ARROW_BUILD_TYPE \
> --with-parquet --with-plasma --inplace
your setuptools is too old (<12)
setuptools_scm functionality is degraded
Traceback (most recent call last):
File "setup.py", line 589, in 
url="https://arrow.apache.org/;
File 
"/home/mrocklin/Software/anaconda/envs/pyarrow-dev/lib/python3.6/site-packages/setuptools/__init__.py",
 line 140, in setup
return distutils.core.setup(**attrs)
File 
"/home/mrocklin/Software/anaconda/envs/pyarrow-dev/lib/python3.6/distutils/core.py",
 line 108, in setup
_setup_distribution = dist = klass(attrs)
File 
"/home/mrocklin/Software/anaconda/envs/pyarrow-dev/lib/python3.6/site-packages/setuptools/dist.py",
 line 370, in __init__
k: v for k, v in attrs.items()
File 
"/home/mrocklin/Software/anaconda/envs/pyarrow-dev/lib/python3.6/distutils/dist.py",
 line 281, in __init__
self.finalize_options()
File 
"/home/mrocklin/Software/anaconda/envs/pyarrow-dev/lib/python3.6/site-packages/setuptools/dist.py",
 line 529, in finalize_options
ep.load()(self, ep.name, value)
File 
"/home/mrocklin/workspace/arrow/python/.eggs/setuptools_scm-1.15.1rc1-py3.6.egg/setuptools_scm/integration.py",
 line 19, in version_keyword
File 
"/home/mrocklin/workspace/arrow/python/.eggs/setuptools_scm-1.15.1rc1-py3.6.egg/setuptools_scm/__init__.py",
 line 117, in get_version
File 
"/home/mrocklin/workspace/arrow/python/.eggs/setuptools_scm-1.15.1rc1-py3.6.egg/setuptools_scm/__init__.py",
 line 69, in _do_parse
File "setup.py", line 519, in parse_git
return parse(root, **kwargs)
File 
"/home/mrocklin/workspace/arrow/python/.eggs/setuptools_scm-1.15.1rc1-py3.6.egg/setuptools_scm/git.py",
 line 99, in parse
ValueError: invalid literal for int() with base 10: 'ge2c4b09d'
{code}

> [Python] Difficulty running tests after conda install
> -
>
> Key: ARROW-3280
> URL: https://issues.apache.org/jira/browse/ARROW-3280
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Affects Versions: 0.10.0
> Environment: conda create -n test-arrow pytest ipython pandas nomkl 
> pyarrow -c conda-forge
> Ubuntu 16.04
>Reporter: Matthew Rocklin
>Priority: Minor
>  Labels: python
>
> I install PyArrow from conda-forge, and then try running tests (or import 
> generally)
> {code:java}
> conda create -n test-arrow pytest ipython pandas nomkl pyarrow -c conda-forge 
> {code}
> {code:java}
> mrocklin@carbon:~/workspace/arrow/python$ py.test 
> pyarrow/tests/test_parquet.py 
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 328, in _getconftestmodules
> return self._path2confmods[path]
> KeyError: 
> local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/test_parquet.py')During
>  handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 328, in _getconftestmodules
> return self._path2confmods[path]
> KeyError: local('/home/mrocklin/workspace/arrow/python/pyarrow/tests')During 
> handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 359, in _importconftest
> return self._conftestpath2mod[conftestpath]
> KeyError: 
> local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py')During
>  handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 365, in _importconftest
> mod = conftestpath.pyimport()
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/py/_path/local.py",
>  line 668, in pyimport
> __import__(modname)
> File "/home/mrocklin/workspace/arrow/python/pyarrow/__init__.py", line 54, in 
> 
> from pyarrow.lib import cpu_count, set_cpu_count
> ModuleNotFoundError: No module named 'pyarrow.lib'
> ERROR: could not load 
> /home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py{code}
> Probably this is something wrong with my environment, but I thought I'd 
> report it as a usability bug



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3280) [Python] Difficulty running tests after conda install

2018-09-20 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16622938#comment-16622938
 ] 

Matthew Rocklin commented on ARROW-3280:


Actually, let me back up.  I wrote a test and wanted to run it.  I tried 
running py.test from that directory and got the error that I list above.  
That's probably reasonable, given that Python is probably confused about paths 
given that I'm in a directory named arrow.  

I googled online for arrow developer notes, and eventually found that I was 
supposed to look in the directory for a README file.  That file didn't have 
anything about testing in it explicitly.  

I see now that it has a "Build from source" section that links to external 
docs.  I'll go and try that and see what happens.

> [Python] Difficulty running tests after conda install
> -
>
> Key: ARROW-3280
> URL: https://issues.apache.org/jira/browse/ARROW-3280
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Affects Versions: 0.10.0
> Environment: conda create -n test-arrow pytest ipython pandas nomkl 
> pyarrow -c conda-forge
> Ubuntu 16.04
>Reporter: Matthew Rocklin
>Priority: Minor
>  Labels: python
>
> I install PyArrow from conda-forge, and then try running tests (or import 
> generally)
> {code:java}
> conda create -n test-arrow pytest ipython pandas nomkl pyarrow -c conda-forge 
> {code}
> {code:java}
> mrocklin@carbon:~/workspace/arrow/python$ py.test 
> pyarrow/tests/test_parquet.py 
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 328, in _getconftestmodules
> return self._path2confmods[path]
> KeyError: 
> local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/test_parquet.py')During
>  handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 328, in _getconftestmodules
> return self._path2confmods[path]
> KeyError: local('/home/mrocklin/workspace/arrow/python/pyarrow/tests')During 
> handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 359, in _importconftest
> return self._conftestpath2mod[conftestpath]
> KeyError: 
> local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py')During
>  handling of the above exception, another exception occurred:
> Traceback (most recent call last):
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
>  line 365, in _importconftest
> mod = conftestpath.pyimport()
> File 
> "/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/py/_path/local.py",
>  line 668, in pyimport
> __import__(modname)
> File "/home/mrocklin/workspace/arrow/python/pyarrow/__init__.py", line 54, in 
> 
> from pyarrow.lib import cpu_count, set_cpu_count
> ModuleNotFoundError: No module named 'pyarrow.lib'
> ERROR: could not load 
> /home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py{code}
> Probably this is something wrong with my environment, but I thought I'd 
> report it as a usability bug



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Created] (ARROW-3280) [Python] Difficulty running tests after conda install

2018-09-20 Thread Matthew Rocklin (JIRA)
Matthew Rocklin created ARROW-3280:
--

 Summary: [Python] Difficulty running tests after conda install
 Key: ARROW-3280
 URL: https://issues.apache.org/jira/browse/ARROW-3280
 Project: Apache Arrow
  Issue Type: Bug
  Components: Python
Affects Versions: 0.10.0
 Environment: conda create -n test-arrow pytest ipython pandas nomkl 
pyarrow -c conda-forge

Ubuntu 16.04
Reporter: Matthew Rocklin


I install PyArrow from conda-forge, and then try running tests (or import 
generally)
{code:java}
conda create -n test-arrow pytest ipython pandas nomkl pyarrow -c conda-forge 
{code}
{code:java}
mrocklin@carbon:~/workspace/arrow/python$ py.test pyarrow/tests/test_parquet.py 
Traceback (most recent call last):
File 
"/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
 line 328, in _getconftestmodules
return self._path2confmods[path]
KeyError: 
local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/test_parquet.py')During
 handling of the above exception, another exception occurred:
Traceback (most recent call last):
File 
"/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
 line 328, in _getconftestmodules
return self._path2confmods[path]
KeyError: local('/home/mrocklin/workspace/arrow/python/pyarrow/tests')During 
handling of the above exception, another exception occurred:
Traceback (most recent call last):
File 
"/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
 line 359, in _importconftest
return self._conftestpath2mod[conftestpath]
KeyError: 
local('/home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py')During 
handling of the above exception, another exception occurred:
Traceback (most recent call last):
File 
"/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/_pytest/config.py",
 line 365, in _importconftest
mod = conftestpath.pyimport()
File 
"/home/mrocklin/Software/anaconda/lib/python3.6/site-packages/py/_path/local.py",
 line 668, in pyimport
__import__(modname)
File "/home/mrocklin/workspace/arrow/python/pyarrow/__init__.py", line 54, in 

from pyarrow.lib import cpu_count, set_cpu_count
ModuleNotFoundError: No module named 'pyarrow.lib'
ERROR: could not load 
/home/mrocklin/workspace/arrow/python/pyarrow/tests/conftest.py{code}
Probably this is something wrong with my environment, but I thought I'd report 
it as a usability bug



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3245) [Python] Infer index and/or filtering from parquet column statistics

2018-09-20 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16622921#comment-16622921
 ] 

Matthew Rocklin commented on ARROW-3245:


After some fooling around this worked for me

{{import pyarrow.parquet as pq}}
{{import pandas as pd}}
{{df = pd.DataFrame(\{'a': [1, 0]})}}
{{df.to_parquet('out.parq', engine='pyarrow')}}
{{pf = pq.ParquetDataset('out.parq')}}
{{piece = pf.pieces[0]}}
{{import functools}}
{{piece.get_metadata(functools.partial(open, mode='rb'))}}

I had to dive into the source a bit to figure out how to interpret the 
docstring.

> [Python] Infer index and/or filtering from parquet column statistics
> 
>
> Key: ARROW-3245
> URL: https://issues.apache.org/jira/browse/ARROW-3245
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Martin Durant
>Priority: Major
>  Labels: parquet
>
> The metadata included in parquet generally gives the min/max of data for each 
> chunk of each column. This allows early filtering out of whole chunks if they 
> do not meet some criterion, and can greatly reduce reading burden in some 
> circumstances. In Dask, we care about this for setting an index and its 
> "divisions" (start/stop values for each data partition) and for directly 
> avoiding including some chunks in the graph of tasks to be processed. 
> Similarly, filtering may be applied on the values of fields defined by the 
> directory partitioning.
> Currently, dask using the fastparquet backend is able to infer possible 
> columns to use as an index, perform filtering on that index and do general 
> filtering on any column which has statistical or partitioning information. It 
> would be very helpful to have such facilities via pyarrow also.
>  This is probably the most important of the requests from Dask.
> (please forgive that some of this has already been mentioned elsewhere; this 
> is one of the entries in the list at 
> [https://github.com/dask/fastparquet/issues/374] as a feature that is useful 
> in fastparquet)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-3244) [Python] Multi-file parquet loading without scan

2018-09-20 Thread Matthew Rocklin (JIRA)


[ 
https://issues.apache.org/jira/browse/ARROW-3244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16622917#comment-16622917
 ] 

Matthew Rocklin commented on ARROW-3244:


What happens today when someone reads a multi-file parquet dataset with dask 
dataframe?  We read a single file to get the schema and then just build tasks 
for everything else?  Or do we need to read through each of the files in order 
to find out how many row blocks are in each?

On the Arrow side is this in scope?  Is this already implemented?  Are there 
mechanisms to construct the metadata files from within Arrow?  If not, and if 
this is in scope then what is the right way / place to add this behavior?

> [Python] Multi-file parquet loading without scan
> 
>
> Key: ARROW-3244
> URL: https://issues.apache.org/jira/browse/ARROW-3244
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Martin Durant
>Priority: Major
>  Labels: parquet
>
> A number of mechanism are possible to avoid having to access and read the 
> parquet footers in a data set consisting of a number of files. In the case of 
> a large number of data files (perhaps split with directory partitioning) and 
> remote storage, this can be a significant overhead. This is significant from 
> the point of view of Dask, which must have the metadata available in the 
> client before setting up computational graphs.
>  
> Here are some suggestions of what could be done.
>  
>  * some parquet writing frameworks include a `_metadata` file, which contains 
> all the information from the footers of the various files. If this file is 
> present, then this data can be read from one place, with a single file 
> access. For a large number of files, parsing the thrift information may, by 
> itself, be a non-negligible overhead≥
>  * the schema (dtypes) can be found in a `_common_metadata`, or from any one 
> of the data-files, then the schema could be assumed (perhaps at the user's 
> option) to be the same for all of the files. However, the information about 
> the directory partitioning would not be available. Although Dask may infer 
> the information from the filenames, it would be preferable to go through the 
> machinery with parquet-cpp, and view the whole data-set as a single object. 
> Note that the files will still need to have the footer read to access the 
> data, for the bytes offsets, but from Dask's point of view, this would be 
> deferred to tasks running in parallel.
> (please forgive that some of this has already been mentioned elsewhere; this 
> is one of the entries in the list at 
> [https://github.com/dask/fastparquet/issues/374] as a feature that is useful 
> in fastparquet)
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (ARROW-1784) [Python] Read and write pandas.DataFrame in pyarrow.serialize by decomposing the BlockManager rather than coercing to Arrow format

2017-11-09 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-1784?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16246134#comment-16246134
 ] 

Matthew Rocklin commented on ARROW-1784:


Note that matching the BlockManager itself is not important for us.  I could 
imagine doing things columnwise as well if that feels cleaner semantically or 
more future-proof.  

> [Python] Read and write pandas.DataFrame in pyarrow.serialize by decomposing 
> the BlockManager rather than coercing to Arrow format
> --
>
> Key: ARROW-1784
> URL: https://issues.apache.org/jira/browse/ARROW-1784
> Project: Apache Arrow
>  Issue Type: New Feature
>  Components: Python
>Reporter: Wes McKinney
> Fix For: 0.8.0
>
>
> See discussion in https://github.com/dask/distributed/pull/931
> This will permit zero-copy reads for DataFrames not containing Python 
> objects. In the event of an {{ObjectBlock}} these arrays will not be worse 
> than pickle to reconstruct on the receiving side



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (ARROW-596) [Python] Add convenience function to convert pandas.DataFrame to pyarrow.Buffer containing a file or stream representation

2017-03-03 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894489#comment-15894489
 ] 

Matthew Rocklin commented on ARROW-596:
---

I've never had to construct one myself.  I just grab `my_numpy_array.data` and 
pass that around.  I'll ask Antoine Pitrou to chime in here.  I suspect that he 
would have a better understanding.

> [Python] Add convenience function to convert pandas.DataFrame to 
> pyarrow.Buffer containing a file or stream representation
> --
>
> Key: ARROW-596
> URL: https://issues.apache.org/jira/browse/ARROW-596
> Project: Apache Arrow
>  Issue Type: New Feature
>  Components: Python
>Reporter: Wes McKinney
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (ARROW-596) [Python] Add convenience function to convert pandas.DataFrame to pyarrow.Buffer containing a file or stream representation

2017-03-03 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-596?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894477#comment-15894477
 ] 

Matthew Rocklin commented on ARROW-596:
---

For network applications a nice interface is something that we can pass to 
`socket.send`.  This might be something like a bytes, bytearray, memoryview, or 
sequence of those.

> [Python] Add convenience function to convert pandas.DataFrame to 
> pyarrow.Buffer containing a file or stream representation
> --
>
> Key: ARROW-596
> URL: https://issues.apache.org/jira/browse/ARROW-596
> Project: Apache Arrow
>  Issue Type: New Feature
>  Components: Python
>Reporter: Wes McKinney
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (ARROW-376) Python: Convert non-range Pandas indices (optionally) to Arrow

2017-03-03 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-376?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15894266#comment-15894266
 ] 

Matthew Rocklin commented on ARROW-376:
---

> I think we would see good speed boosts on communication heavy workloads like 
> shuffles.

I need to walk back from this statement a bit.  I implemented a crude solution 
using straight numpy that works in simple cases to see what I could expect from 
a full Arrow solution.  I did not see as much improvement as I expected.  Still 
trying to identify my current bottleneck.

> Python: Convert non-range Pandas indices (optionally) to Arrow
> --
>
> Key: ARROW-376
> URL: https://issues.apache.org/jira/browse/ARROW-376
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Uwe L. Korn
>Assignee: Wes McKinney
>Priority: Minor
>  Labels: newbie
> Fix For: 0.3.0
>
>
> Currently the indices of a Pandas DataFrame are totally ignored on the Pandas 
> to Arrow conversion. We should add an option to also convert the index to an 
> Arrow column if they are not a simple range index.
> The condition for a simple index should be {{isinstance(df.index, 
> pd.RangeIndex) && (df.index._start == 0) && (df.index._stop == len(df.index)) 
> && (df.index._step == 1)}}. In this case, we can always skip the index 
> conversion. Otherwise, a new column in the Arrow table shall be created using 
> the index' name as the name of the column. Additionally there should be some 
> metadata annotation of that column that it is derived of an Pandas Index, so 
> that for roundtrips, we'll use it again as the index of a DataFrame.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (ARROW-452) [C++/Python] Merge "Feather" file format implementation

2017-03-02 Thread Matthew Rocklin (JIRA)

 [ 
https://issues.apache.org/jira/browse/ARROW-452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Matthew Rocklin reassigned ARROW-452:
-

Assignee: (was: Matthew Rocklin)

> [C++/Python] Merge "Feather" file format implementation
> ---
>
> Key: ARROW-452
> URL: https://issues.apache.org/jira/browse/ARROW-452
> Project: Apache Arrow
>  Issue Type: New Feature
>  Components: C++, Python
>Reporter: Wes McKinney
>
> See https://github.com/wesm/feather/tree/master/cpp -- this will assist with 
> code consolidation and reconciling metadata requirements for Python and R 
> users, with the goal of eventually using the Arrow IPC format for everything 
> and deprecating the less-flexible Feather format / metadata. 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Commented] (ARROW-376) Python: Convert non-range Pandas indices (optionally) to Arrow

2017-03-02 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-376?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15893208#comment-15893208
 ] 

Matthew Rocklin commented on ARROW-376:
---

I would love to see this issue get higher priority.  I would like to experiment 
with using Arrow as Dask's network serialization format for pandas dataframes 
if it were implemented.  I think we would see good speed boosts on 
communication heavy workloads like shuffles.  This would be fun to write about 
afterwards.

> Python: Convert non-range Pandas indices (optionally) to Arrow
> --
>
> Key: ARROW-376
> URL: https://issues.apache.org/jira/browse/ARROW-376
> Project: Apache Arrow
>  Issue Type: Improvement
>  Components: Python
>Reporter: Uwe L. Korn
>Assignee: Jim Ahn
>Priority: Minor
>  Labels: newbie
>
> Currently the indices of a Pandas DataFrame are totally ignored on the Pandas 
> to Arrow conversion. We should add an option to also convert the index to an 
> Arrow column if they are not a simple range index.
> The condition for a simple index should be {{isinstance(df.index, 
> pd.RangeIndex) && (df.index._start == 0) && (df.index._stop == len(df.index)) 
> && (df.index._step == 1)}}. In this case, we can always skip the index 
> conversion. Otherwise, a new column in the Arrow table shall be created using 
> the index' name as the name of the column. Additionally there should be some 
> metadata annotation of that column that it is derived of an Pandas Index, so 
> that for roundtrips, we'll use it again as the index of a DataFrame.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Assigned] (ARROW-452) [C++/Python] Merge "Feather" file format implementation

2017-03-02 Thread Matthew Rocklin (JIRA)

 [ 
https://issues.apache.org/jira/browse/ARROW-452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Matthew Rocklin reassigned ARROW-452:
-

Assignee: Matthew Rocklin  (was: Wes McKinney)

> [C++/Python] Merge "Feather" file format implementation
> ---
>
> Key: ARROW-452
> URL: https://issues.apache.org/jira/browse/ARROW-452
> Project: Apache Arrow
>  Issue Type: New Feature
>  Components: C++, Python
>Reporter: Wes McKinney
>Assignee: Matthew Rocklin
>
> See https://github.com/wesm/feather/tree/master/cpp -- this will assist with 
> code consolidation and reconciling metadata requirements for Python and R 
> users, with the goal of eventually using the Arrow IPC format for everything 
> and deprecating the less-flexible Feather format / metadata. 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Comment Edited] (ARROW-504) [Python] Add adapter to write pandas.DataFrame in user-selected chunk size to streaming format

2017-01-23 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-504?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15834694#comment-15834694
 ] 

Matthew Rocklin edited comment on ARROW-504 at 1/23/17 2:53 PM:


At the moment I don't have any active use cases for this.  We tend to handle 
pandas dataframes as atomic blocks of data.

However generally I agree that streaming chunks in a more granular way is 
probably a better way to go.  Non-blocking IO quickly becomes blocking IO if 
data starts overflowing local buffers.  This is the sort of technology that 
might influence future design decisions.

>From a pure Dask perspective my ideal serialization interface is Python object 
>-> iterator of memoryview objects.  


was (Author: mrocklin):
At the moment I don't have any active use cases for this.  We tend to handle 
pandas dataframes as atomic blocks of data.

However generally I agree that streaming chunks in a more granular way is 
probably a better way to go.  Non-blocking IO quickly becomes blocking IO if 
data starts overflows local buffers.  This is the sort of technology that might 
influence future design decisions.

>From a pure Dask perspective my ideal serialization interface is Python object 
>-> iterator of memoryview objects.  

> [Python] Add adapter to write pandas.DataFrame in user-selected chunk size to 
> streaming format
> --
>
> Key: ARROW-504
> URL: https://issues.apache.org/jira/browse/ARROW-504
> Project: Apache Arrow
>  Issue Type: New Feature
>Reporter: Wes McKinney
>
> While we can convert a {{pandas.DataFrame}} to a single (arbitrarily large) 
> {{arrow::RecordBatch}}, it is not easy to create multiple small record 
> batches -- we could do so in a streaming fashion and immediately write them 
> into an {{arrow::io::OutputStream}}.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (ARROW-434) Segfaults and encoding issues in Python Parquet reads

2016-12-22 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15770242#comment-15770242
 ] 

Matthew Rocklin edited comment on ARROW-434 at 12/22/16 3:22 PM:
-

Cool, verified that it works on my end.  The taxi data on s3fs is still failing 
with an encoding error.  I've been having difficulty managing permissions on S3 
to make this publicly available (just ignorance on my part).  In the mean time, 
here's the status of the files in the "parquet compatibility project"

{code}
In [1]: import pyarrow.parquet

In [2]: from glob import glob

In [3]: filenames = sorted(glob('*.parquet'))

In [4]: filenames
Out[4]: 
['customer.impala.parquet',
 'foo.parquet',
 'gzip-nation.impala.parquet',
 'nation.dict.parquet',
 'nation.impala.parquet',
 'nation.plain.parquet',
 'snappy-nation.impala.parquet',
 'test-converted-type-null.parquet',
 'test-null-dictionary.parquet',
 'test-null.parquet',
 'test.parquet']

In [5]: for fn in filenames:
   ...: try:
   ...: t = pyarrow.parquet.read_table(fn)
   ...: except Exception as e:
   ...: print('Failed on', fn, e)
   ...: else:
   ...: print("Succeeded on", fn)
   ...: 
   ...: 
Succeeded on customer.impala.parquet
Succeeded on foo.parquet
Succeeded on gzip-nation.impala.parquet
Failed on nation.dict.parquet IOError: Unexpected end of stream.
Succeeded on nation.impala.parquet
Succeeded on nation.plain.parquet
Succeeded on snappy-nation.impala.parquet
Succeeded on test-converted-type-null.parquet
Succeeded on test-null-dictionary.parquet
Succeeded on test-null.parquet
Succeeded on test.parquet

In [6]: pyarrow.parquet.read_table('nation.dict.parquet')
---
ArrowExceptionTraceback (most recent call last)
 in ()
> 1 pyarrow.parquet.read_table('nation.dict.parquet')

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/parquet.pyx
 in pyarrow.parquet.read_table 
(/feedstock_root/build_artefacts/work/arrow-268ffbeffb1cd0617e52d381d500a2d10f61124c/python/build/temp.linux-x86_64-3.5/parquet.cxx:2907)()

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/parquet.pyx
 in pyarrow.parquet.ParquetReader.read_all 
(/feedstock_root/build_artefacts/work/arrow-268ffbeffb1cd0617e52d381d500a2d10f61124c/python/build/temp.linux-x86_64-3.5/parquet.cxx:2275)()

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/error.pyx
 in pyarrow.error.check_status 
(/feedstock_root/build_artefacts/work/arrow-268ffbeffb1cd0617e52d381d500a2d10f61124c/python/build/temp.linux-x86_64-3.5/error.cxx:1197)()

ArrowException: IOError: Unexpected end of stream.
{code}


was (Author: mrocklin):
Cool, verified that it works on my end.  The airlines data on s3fs is still 
failing with an encoding error.  I've been having difficulty managing 
permissions on S3 to make this publicly available (just ignorance on my part).  
In the mean time, here's the status of the files in the "parquet compatibility 
project"

{code}
In [1]: import pyarrow.parquet

In [2]: from glob import glob

In [3]: filenames = sorted(glob('*.parquet'))

In [4]: filenames
Out[4]: 
['customer.impala.parquet',
 'foo.parquet',
 'gzip-nation.impala.parquet',
 'nation.dict.parquet',
 'nation.impala.parquet',
 'nation.plain.parquet',
 'snappy-nation.impala.parquet',
 'test-converted-type-null.parquet',
 'test-null-dictionary.parquet',
 'test-null.parquet',
 'test.parquet']

In [5]: for fn in filenames:
   ...: try:
   ...: t = pyarrow.parquet.read_table(fn)
   ...: except Exception as e:
   ...: print('Failed on', fn, e)
   ...: else:
   ...: print("Succeeded on", fn)
   ...: 
   ...: 
Succeeded on customer.impala.parquet
Succeeded on foo.parquet
Succeeded on gzip-nation.impala.parquet
Failed on nation.dict.parquet IOError: Unexpected end of stream.
Succeeded on nation.impala.parquet
Succeeded on nation.plain.parquet
Succeeded on snappy-nation.impala.parquet
Succeeded on test-converted-type-null.parquet
Succeeded on test-null-dictionary.parquet
Succeeded on test-null.parquet
Succeeded on test.parquet

In [6]: pyarrow.parquet.read_table('nation.dict.parquet')
---
ArrowExceptionTraceback (most recent call last)
 in ()
> 1 pyarrow.parquet.read_table('nation.dict.parquet')

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/parquet.pyx
 in pyarrow.parquet.read_table 
(/feedstock_root/build_artefacts/work/arrow-268ffbeffb1cd0617e52d381d500a2d10f61124c/python/build/temp.linux-x86_64-3.5/parquet.cxx:2907)()

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/parquet.pyx
 in 

[jira] [Commented] (ARROW-434) Segfaults and encoding issues in Python Parquet reads

2016-12-22 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-434?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15770242#comment-15770242
 ] 

Matthew Rocklin commented on ARROW-434:
---

Cool, verified that it works on my end.  The airlines data on s3fs is still 
failing with an encoding error.  I've been having difficulty managing 
permissions on S3 to make this publicly available (just ignorance on my part).  
In the mean time, here's the status of the files in the "parquet compatibility 
project"

{code}
In [1]: import pyarrow.parquet

In [2]: from glob import glob

In [3]: filenames = sorted(glob('*.parquet'))

In [4]: filenames
Out[4]: 
['customer.impala.parquet',
 'foo.parquet',
 'gzip-nation.impala.parquet',
 'nation.dict.parquet',
 'nation.impala.parquet',
 'nation.plain.parquet',
 'snappy-nation.impala.parquet',
 'test-converted-type-null.parquet',
 'test-null-dictionary.parquet',
 'test-null.parquet',
 'test.parquet']

In [5]: for fn in filenames:
   ...: try:
   ...: t = pyarrow.parquet.read_table(fn)
   ...: except Exception as e:
   ...: print('Failed on', fn, e)
   ...: else:
   ...: print("Succeeded on", fn)
   ...: 
   ...: 
Succeeded on customer.impala.parquet
Succeeded on foo.parquet
Succeeded on gzip-nation.impala.parquet
Failed on nation.dict.parquet IOError: Unexpected end of stream.
Succeeded on nation.impala.parquet
Succeeded on nation.plain.parquet
Succeeded on snappy-nation.impala.parquet
Succeeded on test-converted-type-null.parquet
Succeeded on test-null-dictionary.parquet
Succeeded on test-null.parquet
Succeeded on test.parquet

In [6]: pyarrow.parquet.read_table('nation.dict.parquet')
---
ArrowExceptionTraceback (most recent call last)
 in ()
> 1 pyarrow.parquet.read_table('nation.dict.parquet')

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/parquet.pyx
 in pyarrow.parquet.read_table 
(/feedstock_root/build_artefacts/work/arrow-268ffbeffb1cd0617e52d381d500a2d10f61124c/python/build/temp.linux-x86_64-3.5/parquet.cxx:2907)()

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/parquet.pyx
 in pyarrow.parquet.ParquetReader.read_all 
(/feedstock_root/build_artefacts/work/arrow-268ffbeffb1cd0617e52d381d500a2d10f61124c/python/build/temp.linux-x86_64-3.5/parquet.cxx:2275)()

/home/mrocklin/Software/anaconda/envs/arrow-test/lib/python3.5/site-packages/pyarrow/error.pyx
 in pyarrow.error.check_status 
(/feedstock_root/build_artefacts/work/arrow-268ffbeffb1cd0617e52d381d500a2d10f61124c/python/build/temp.linux-x86_64-3.5/error.cxx:1197)()

ArrowException: IOError: Unexpected end of stream.
{code}

> Segfaults and encoding issues in Python Parquet reads
> -
>
> Key: ARROW-434
> URL: https://issues.apache.org/jira/browse/ARROW-434
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
> Environment: Ubuntu, Python 3.5, installed pyarrow from conda-forge
>Reporter: Matthew Rocklin
>Assignee: Wes McKinney
>Priority: Minor
>  Labels: parquet, python
>
> I've conda installed pyarrow and am trying to read data from the 
> parquet-compatibility project.  I haven't explicitly built parquet-cpp or 
> anything and may or may not have old versions lying around, so please take 
> this issue with some salt:
> {code:python}
> In [1]: import pyarrow.parquet
> In [2]: t = pyarrow.parquet.read_table('nation.plain.parquet')
> ---
> ArrowExceptionTraceback (most recent call last)
>  in ()
> > 1 t = pyarrow.parquet.read_table('nation.plain.parquet')
> /home/mrocklin/Software/anaconda/lib/python3.5/site-packages/pyarrow/parquet.pyx
>  in pyarrow.parquet.read_table 
> (/feedstock_root/build_artefacts/work/arrow-79344b335849c2eb43954b0751018051814019d6/python/build/temp.linux-x86_64-3.5/parquet.cxx:2783)()
> /home/mrocklin/Software/anaconda/lib/python3.5/site-packages/pyarrow/parquet.pyx
>  in pyarrow.parquet.ParquetReader.read_all 
> (/feedstock_root/build_artefacts/work/arrow-79344b335849c2eb43954b0751018051814019d6/python/build/temp.linux-x86_64-3.5/parquet.cxx:2200)()
> /home/mrocklin/Software/anaconda/lib/python3.5/site-packages/pyarrow/error.pyx
>  in pyarrow.error.check_status 
> (/feedstock_root/build_artefacts/work/arrow-79344b335849c2eb43954b0751018051814019d6/python/build/temp.linux-x86_64-3.5/error.cxx:1185)()
> ArrowException: NotImplemented: list<: uint8>
> {code}
> Additionally I tried to read data from a Python file-like object pointing to 
> data on S3.  Let me know if you'd prefer a separate issue.
> {code:python}
> In [1]: import s3fs
> In [2]: fs 

[jira] [Commented] (ARROW-346) Python API Documentation

2016-11-24 Thread Matthew Rocklin (JIRA)

[ 
https://issues.apache.org/jira/browse/ARROW-346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15693626#comment-15693626
 ] 

Matthew Rocklin commented on ARROW-346:
---

Read the docs also provides experimental support for conda environments.




> Python API Documentation
> 
>
> Key: ARROW-346
> URL: https://issues.apache.org/jira/browse/ARROW-346
> Project: Apache Arrow
>  Issue Type: Bug
>  Components: Python
>Reporter: Matthew Rocklin
>Assignee: Uwe L. Korn
>Priority: Minor
>
> It would be useful to have API documentation somewhere.  A minimal page on 
> readthedocs might suffice and be cheap to maintain.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (ARROW-346) Python API Documentation

2016-10-20 Thread Matthew Rocklin (JIRA)
Matthew Rocklin created ARROW-346:
-

 Summary: Python API Documentation
 Key: ARROW-346
 URL: https://issues.apache.org/jira/browse/ARROW-346
 Project: Apache Arrow
  Issue Type: Bug
  Components: Python
Reporter: Matthew Rocklin
Priority: Minor


It would be useful to have API documentation somewhere.  A minimal page on 
readthedocs might suffice and be cheap to maintain.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)