[ https://issues.apache.org/jira/browse/ARROW-2814?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Wes McKinney updated ARROW-2814: -------------------------------- Description: Example, setup: {code} import pandas as pd s = pd.Series([{'data': {'document_id': None, 'document_type': None, 'master_customer_id': None, 'message': 'User Login Request', 'policy_id': None, 'sequence_no': 14, 'user_name': None}, 'header': {'actor_id': None, 'actor_type': None, 'brand_code': 'ES', 'event_origin': None, 'event_timestamp': '2018-01-01T18:25:43.511Z', 'event_type': 'LOGIN', 'master_customer_id': '14', 'source': 'CUSTOMER_AUTH_SERVICE', 'source_id': None, 'source_version': None}, 'payload_version': '1', 'status': {'status_code': 100, 'status_message': 'Success'}}]) {code} This works: {code} In [24]: pa.array(list(s)) Out[24]: <pyarrow.lib.StructArray object at 0x7f8435b09c28> [ {'data': {'document_id': None, 'document_type': None, 'master_customer_id': None, 'message': 'User Login Request', 'policy_id': None, 'sequence_no': 14, 'user_name': None}, 'header': {'actor_id': None, 'actor_type': None, 'brand_code': 'ES', 'event_origin': None, 'event_timestamp': '2018-01-01T18:25:43.511Z', 'event_type': 'LOGIN', 'master_customer_id': '14', 'source': 'CUSTOMER_AUTH_SERVICE', 'source_id': None, 'source_version': None}, 'payload_version': '1', 'status': {'status_code': 100, 'status_message': 'Success'}} ] {code} This does not: {code} In [23]: pa.array(s) --------------------------------------------------------------------------- ArrowInvalid Traceback (most recent call last) <ipython-input-23-eba23a1638b7> in <module>() ----> 1 pa.array(s) ~/code/arrow/python/pyarrow/array.pxi in pyarrow.lib.array() 175 values, type = pdcompat.get_datetimetz_type(values, obj.dtype, 176 type) --> 177 return _ndarray_to_array(values, mask, type, from_pandas, pool) 178 else: 179 if mask is not None: ~/code/arrow/python/pyarrow/array.pxi in pyarrow.lib._ndarray_to_array() 75 76 with nogil: ---> 77 check_status(NdarrayToArrow(pool, values, mask, 78 use_pandas_null_sentinels, 79 c_type, &chunked_out)) ~/code/arrow/python/pyarrow/error.pxi in pyarrow.lib.check_status() 79 message = frombytes(status.message()) 80 if status.IsInvalid(): ---> 81 raise ArrowInvalid(message) 82 elif status.IsIOError(): 83 raise ArrowIOError(message) ArrowInvalid: ../src/arrow/python/numpy_to_arrow.cc:1742 code: converter.Convert() Error inferring Arrow type for Python object array. Got Python object of type dict but can only handle these types: string, bool, float, int, date, time, decimal, bytearray, list, array {code} was: There is a problem when trying to run pa.Table.from_pandas() on a parquet file that has a json string in it. I have attached the file to this ticket that is the source of the problem and the code below will show the error. h2. Reproducible code import pandas as pd import pyarrow as pa import pyarrow.parquet as pq pd.options.display.max_colwidth = 10000 pq_table = pq.read_table("part-00000-8f03690f-736d-43a9-9287-6db9e228d59c.c000.gz.parquet") panda_table = pq_table.to_pandas() orginal_count = len(panda_table) h2. Fails table_output = pa.Table.from_pandas(panda_table) del panda_table['payload'] h2. Works table_output = pa.Table.from_pandas(panda_table) h2. Payload is the faulty column. Print out data pq_table = pq.read_table("part-00000-8f03690f-736d-43a9-9287-6db9e228d59c.c000.gz.parquet") panda_table = pq_table.to_pandas() orginal_count = len(panda_table) table_output = pa.Table.from_pandas(panda_table[['payload']]) panda_table[['payload']] > [Python] Struct type inference and conversion works for lists but not NumPy > arrays with dtype object > ---------------------------------------------------------------------------------------------------- > > Key: ARROW-2814 > URL: https://issues.apache.org/jira/browse/ARROW-2814 > Project: Apache Arrow > Issue Type: Bug > Components: Python > Affects Versions: 0.9.0 > Reporter: rob > Assignee: Wes McKinney > Priority: Blocker > Fix For: 0.10.0 > > Attachments: > part-00000-8f03690f-736d-43a9-9287-6db9e228d59c.c000.gz.parquet > > > Example, setup: > {code} > import pandas as pd > s = pd.Series([{'data': {'document_id': None, > 'document_type': None, > 'master_customer_id': None, > 'message': 'User Login Request', > 'policy_id': None, > 'sequence_no': 14, > 'user_name': None}, > 'header': {'actor_id': None, > 'actor_type': None, > 'brand_code': 'ES', > 'event_origin': None, > 'event_timestamp': '2018-01-01T18:25:43.511Z', > 'event_type': 'LOGIN', > 'master_customer_id': '14', > 'source': 'CUSTOMER_AUTH_SERVICE', > 'source_id': None, > 'source_version': None}, > 'payload_version': '1', > 'status': {'status_code': 100, 'status_message': 'Success'}}]) > {code} > This works: > {code} > In [24]: pa.array(list(s)) > Out[24]: > <pyarrow.lib.StructArray object at 0x7f8435b09c28> > [ > {'data': {'document_id': None, 'document_type': None, 'master_customer_id': > None, 'message': 'User Login Request', 'policy_id': None, 'sequence_no': 14, > 'user_name': None}, 'header': {'actor_id': None, 'actor_type': None, > 'brand_code': 'ES', 'event_origin': None, 'event_timestamp': > '2018-01-01T18:25:43.511Z', 'event_type': 'LOGIN', 'master_customer_id': > '14', 'source': 'CUSTOMER_AUTH_SERVICE', 'source_id': None, 'source_version': > None}, 'payload_version': '1', 'status': {'status_code': 100, > 'status_message': 'Success'}} > ] > {code} > This does not: > {code} > In [23]: pa.array(s) > --------------------------------------------------------------------------- > ArrowInvalid Traceback (most recent call last) > <ipython-input-23-eba23a1638b7> in <module>() > ----> 1 pa.array(s) > ~/code/arrow/python/pyarrow/array.pxi in pyarrow.lib.array() > 175 values, type = pdcompat.get_datetimetz_type(values, > obj.dtype, > 176 type) > --> 177 return _ndarray_to_array(values, mask, type, from_pandas, > pool) > 178 else: > 179 if mask is not None: > ~/code/arrow/python/pyarrow/array.pxi in pyarrow.lib._ndarray_to_array() > 75 > 76 with nogil: > ---> 77 check_status(NdarrayToArrow(pool, values, mask, > 78 use_pandas_null_sentinels, > 79 c_type, &chunked_out)) > ~/code/arrow/python/pyarrow/error.pxi in pyarrow.lib.check_status() > 79 message = frombytes(status.message()) > 80 if status.IsInvalid(): > ---> 81 raise ArrowInvalid(message) > 82 elif status.IsIOError(): > 83 raise ArrowIOError(message) > ArrowInvalid: ../src/arrow/python/numpy_to_arrow.cc:1742 code: > converter.Convert() > Error inferring Arrow type for Python object array. Got Python object of type > dict but can only handle these types: string, bool, float, int, date, time, > decimal, bytearray, list, array > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)