forked from apache/arrow
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy patharray.pxi
More file actions
1753 lines (1420 loc) · 51.6 KB
/
array.pxi
File metadata and controls
1753 lines (1420 loc) · 51.6 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
cdef _sequence_to_array(object sequence, object mask, object size,
DataType type, CMemoryPool* pool, c_bool from_pandas):
cdef int64_t c_size
cdef PyConversionOptions options
if type is not None:
options.type = type.sp_type
if size is not None:
options.size = size
options.pool = pool
options.from_pandas = from_pandas
cdef shared_ptr[CChunkedArray] out
with nogil:
check_status(ConvertPySequence(sequence, mask, options, &out))
if out.get().num_chunks() == 1:
return pyarrow_wrap_array(out.get().chunk(0))
else:
return pyarrow_wrap_chunked_array(out)
cdef inline _is_array_like(obj):
if isinstance(obj, np.ndarray):
return True
return pandas_api._have_pandas_internal() and pandas_api.is_array_like(obj)
def _ndarray_to_arrow_type(object values, DataType type):
return pyarrow_wrap_data_type(_ndarray_to_type(values, type))
cdef shared_ptr[CDataType] _ndarray_to_type(object values,
DataType type) except *:
cdef shared_ptr[CDataType] c_type
dtype = values.dtype
if type is None and dtype != object:
with nogil:
check_status(NumPyDtypeToArrow(dtype, &c_type))
if type is not None:
c_type = type.sp_type
return c_type
cdef _ndarray_to_array(object values, object mask, DataType type,
c_bool from_pandas, c_bool safe, CMemoryPool* pool):
cdef:
shared_ptr[CChunkedArray] chunked_out
shared_ptr[CDataType] c_type = _ndarray_to_type(values, type)
CCastOptions cast_options = CCastOptions(safe)
with nogil:
check_status(NdarrayToArrow(pool, values, mask, from_pandas,
c_type, cast_options, &chunked_out))
if chunked_out.get().num_chunks() > 1:
return pyarrow_wrap_chunked_array(chunked_out)
else:
return pyarrow_wrap_array(chunked_out.get().chunk(0))
def _handle_arrow_array_protocol(obj, type, mask, size):
if mask is not None or size is not None:
raise ValueError(
"Cannot specify a mask or a size when passing an object that is "
"converted with the __arrow_array__ protocol.")
res = obj.__arrow_array__(type=type)
if not isinstance(res, Array):
raise TypeError("The object's __arrow_array__ method does not "
"return a pyarrow Array.")
return res
def array(object obj, type=None, mask=None, size=None, from_pandas=None,
bint safe=True, MemoryPool memory_pool=None):
"""
Create pyarrow.Array instance from a Python object
Parameters
----------
obj : sequence, iterable, ndarray or Series
If both type and size are specified may be a single use iterable. If
not strongly-typed, Arrow type will be inferred for resulting array
type : pyarrow.DataType
Explicit type to attempt to coerce to, otherwise will be inferred from
the data
mask : array (boolean), optional
Indicate which values are null (True) or not null (False).
size : int64, optional
Size of the elements. If the imput is larger than size bail at this
length. For iterators, if size is larger than the input iterator this
will be treated as a "max size", but will involve an initial allocation
of size followed by a resize to the actual size (so if you know the
exact size specifying it correctly will give you better performance).
from_pandas : boolean, default None
Use pandas's semantics for inferring nulls from values in
ndarray-like data. If passed, the mask tasks precendence, but
if a value is unmasked (not-null), but still null according to
pandas semantics, then it is null. Defaults to False if not
passed explicitly by user, or True if a pandas object is
passed in
safe : boolean, default True
Check for overflows or other unsafe conversions
memory_pool : pyarrow.MemoryPool, optional
If not passed, will allocate memory from the currently-set default
memory pool
Notes
-----
Localized timestamps will currently be returned as UTC (pandas's native
representation). Timezone-naive data will be implicitly interpreted as
UTC.
Examples
--------
>>> import pandas as pd
>>> import pyarrow as pa
>>> pa.array(pd.Series([1, 2]))
<pyarrow.array.Int64Array object at 0x7f674e4c0e10>
[
1,
2
]
>>> import numpy as np
>>> pa.array(pd.Series([1, 2]), np.array([0, 1],
... dtype=bool))
<pyarrow.array.Int64Array object at 0x7f9019e11208>
[
1,
null
]
Returns
-------
array : pyarrow.Array or pyarrow.ChunkedArray (if object data
overflowed binary storage)
"""
cdef:
CMemoryPool* pool = maybe_unbox_memory_pool(memory_pool)
bint is_pandas_object = False
bint c_from_pandas
type = ensure_type(type, allow_none=True)
if from_pandas is None:
c_from_pandas = False
else:
c_from_pandas = from_pandas
if hasattr(obj, '__arrow_array__'):
return _handle_arrow_array_protocol(obj, type, mask, size)
elif _is_array_like(obj):
if mask is not None:
# out argument unused
mask = get_series_values(mask, &is_pandas_object)
values = get_series_values(obj, &is_pandas_object)
if is_pandas_object and from_pandas is None:
c_from_pandas = True
if isinstance(values, np.ma.MaskedArray):
if mask is not None:
raise ValueError("Cannot pass a numpy masked array and "
"specify a mask at the same time")
else:
mask = values.mask
values = values.data
if hasattr(values, '__arrow_array__'):
return _handle_arrow_array_protocol(values, type, mask, size)
elif pandas_api.is_categorical(values):
return DictionaryArray.from_arrays(
values.codes, values.categories.values,
mask=mask, ordered=values.ordered,
from_pandas=True, safe=safe,
memory_pool=memory_pool)
else:
if pandas_api.have_pandas:
values, type = pandas_api.compat.get_datetimetz_type(
values, obj.dtype, type)
return _ndarray_to_array(values, mask, type, c_from_pandas, safe,
pool)
else:
# ConvertPySequence does strict conversion if type is explicitly passed
return _sequence_to_array(obj, mask, size, type, pool, c_from_pandas)
def asarray(values, type=None):
"""
Convert to pyarrow.Array, inferring type if not provided. Attempt to cast
if indicated type is different
Parameters
----------
values : array-like
This can be a sequence, numpy.ndarray, pyarrow.Array or
pyarrow.ChunkedArray. If a ChunkedArray is passed, the output will be
a ChunkedArray, otherwise the output will be a Array.
type : string or DataType
Returns
-------
arr : Array or ChunkedArray
"""
if isinstance(values, (Array, ChunkedArray)):
if type is not None and not values.type.equals(type):
values = values.cast(type)
return values
else:
return array(values, type=type)
def infer_type(values, mask=None, from_pandas=False):
"""
Attempt to infer Arrow data type that can hold the passed Python
sequence type in an Array object
Parameters
----------
values : array-like
Sequence to infer type from
mask : ndarray (bool type), optional
Optional exclusion mask where True marks null, False non-null
from_pandas : boolean, default False
Use pandas's NA/null sentinel values for type inference
Returns
-------
type : DataType
"""
cdef:
shared_ptr[CDataType] out
c_bool use_pandas_sentinels = from_pandas
if mask is not None and not isinstance(mask, np.ndarray):
mask = np.array(mask, dtype=bool)
check_status(InferArrowType(values, mask, use_pandas_sentinels, &out))
return pyarrow_wrap_data_type(out)
def _normalize_slice(object arrow_obj, slice key):
cdef:
Py_ssize_t start, stop, step
Py_ssize_t n = len(arrow_obj)
start = key.start or 0
if start < 0:
start += n
if start < 0:
start = 0
elif start >= n:
start = n
stop = key.stop if key.stop is not None else n
if stop < 0:
stop += n
if stop < 0:
stop = 0
elif stop >= n:
stop = n
step = key.step or 1
if step != 1:
raise IndexError('only slices with step 1 supported')
else:
return arrow_obj.slice(start, stop - start)
cdef Py_ssize_t _normalize_index(Py_ssize_t index,
Py_ssize_t length) except -1:
if index < 0:
index += length
if index < 0:
raise IndexError("index out of bounds")
elif index >= length:
raise IndexError("index out of bounds")
return index
cdef class _FunctionContext:
cdef:
unique_ptr[CFunctionContext] ctx
def __cinit__(self):
self.ctx.reset(new CFunctionContext(c_default_memory_pool()))
cdef _FunctionContext _global_ctx = _FunctionContext()
cdef CFunctionContext* _context() nogil:
return _global_ctx.ctx.get()
cdef wrap_datum(const CDatum& datum):
if datum.kind() == DatumType_ARRAY:
return pyarrow_wrap_array(MakeArray(datum.array()))
elif datum.kind() == DatumType_CHUNKED_ARRAY:
return pyarrow_wrap_chunked_array(datum.chunked_array())
elif datum.kind() == DatumType_SCALAR:
return pyarrow_wrap_scalar(datum.scalar())
else:
raise ValueError("Unable to wrap Datum in a Python object")
cdef _append_array_buffers(const CArrayData* ad, list res):
"""
Recursively append Buffer wrappers from *ad* and its children.
"""
cdef size_t i, n
assert ad != NULL
n = ad.buffers.size()
for i in range(n):
buf = ad.buffers[i]
res.append(pyarrow_wrap_buffer(buf)
if buf.get() != NULL else None)
n = ad.child_data.size()
for i in range(n):
_append_array_buffers(ad.child_data[i].get(), res)
cdef _reduce_array_data(const CArrayData* ad):
"""
Recursively dissect ArrayData to (pickable) tuples.
"""
cdef size_t i, n
assert ad != NULL
n = ad.buffers.size()
buffers = []
for i in range(n):
buf = ad.buffers[i]
buffers.append(pyarrow_wrap_buffer(buf)
if buf.get() != NULL else None)
children = []
n = ad.child_data.size()
for i in range(n):
children.append(_reduce_array_data(ad.child_data[i].get()))
return pyarrow_wrap_data_type(ad.type), ad.length, ad.null_count, \
ad.offset, buffers, children
cdef shared_ptr[CArrayData] _reconstruct_array_data(data):
"""
Reconstruct CArrayData objects from the tuple structure generated
by _reduce_array_data.
"""
cdef:
int64_t length, null_count, offset, i
DataType dtype
Buffer buf
vector[shared_ptr[CBuffer]] c_buffers
vector[shared_ptr[CArrayData]] c_children
dtype, length, null_count, offset, buffers, children = data
for i in range(len(buffers)):
buf = buffers[i]
if buf is None:
c_buffers.push_back(shared_ptr[CBuffer]())
else:
c_buffers.push_back(buf.buffer)
for i in range(len(children)):
c_children.push_back(_reconstruct_array_data(children[i]))
return CArrayData.MakeWithChildren(
dtype.sp_type,
length,
c_buffers,
c_children,
null_count,
offset)
def _restore_array(data):
"""
Reconstruct an Array from pickled ArrayData.
"""
cdef shared_ptr[CArrayData] ad = _reconstruct_array_data(data)
return pyarrow_wrap_array(MakeArray(ad))
cdef class _PandasConvertible:
def to_pandas(
self,
memory_pool=None,
categories=None,
bint strings_to_categorical=False,
bint zero_copy_only=False,
bint integer_object_nulls=False,
bint date_as_object=True,
bint use_threads=True,
bint deduplicate_objects=True,
bint ignore_metadata=False
):
"""
Convert to a pandas-compatible NumPy array or DataFrame, as appropriate
Parameters
----------
memory_pool : MemoryPool, default None
Arrow MemoryPool to use for allocations. Uses the default memory
pool is not passed
strings_to_categorical : boolean, default False
Encode string (UTF8) and binary types to pandas.Categorical
categories: list, default empty
List of fields that should be returned as pandas.Categorical. Only
applies to table-like data structures
zero_copy_only : boolean, default False
Raise an ArrowException if this function call would require copying
the underlying data
integer_object_nulls : boolean, default False
Cast integers with nulls to objects
date_as_object : boolean, default False
Cast dates to objects
use_threads: boolean, default True
Whether to parallelize the conversion using multiple threads
deduplicate_objects : boolean, default False
Do not create multiple copies Python objects when created, to save
on memory use. Conversion will be slower
ignore_metadata : boolean, default False
If True, do not use the 'pandas' metadata to reconstruct the
DataFrame index, if present
Returns
-------
pandas.Series or pandas.DataFrame depending on type of object
"""
options = dict(
pool=memory_pool,
strings_to_categorical=strings_to_categorical,
zero_copy_only=zero_copy_only,
integer_object_nulls=integer_object_nulls,
date_as_object=date_as_object,
use_threads=use_threads,
deduplicate_objects=deduplicate_objects
)
return self._to_pandas(options, categories=categories,
ignore_metadata=ignore_metadata)
cdef PandasOptions _convert_pandas_options(dict options):
cdef PandasOptions result
result.pool = maybe_unbox_memory_pool(options['pool'])
result.strings_to_categorical = options['strings_to_categorical']
result.zero_copy_only = options['zero_copy_only']
result.integer_object_nulls = options['integer_object_nulls']
result.date_as_object = options['date_as_object']
result.use_threads = options['use_threads']
result.deduplicate_objects = options['deduplicate_objects']
return result
cdef class Array(_PandasConvertible):
"""
The base class for all Arrow arrays.
"""
def __init__(self):
raise TypeError("Do not call {}'s constructor directly, use one of "
"the `pyarrow.Array.from_*` functions instead."
.format(self.__class__.__name__))
cdef void init(self, const shared_ptr[CArray]& sp_array) except *:
self.sp_array = sp_array
self.ap = sp_array.get()
self.type = pyarrow_wrap_data_type(self.sp_array.get().type())
def __eq__(self, other):
raise NotImplementedError('Comparisons with pyarrow.Array are not '
'implemented')
def _debug_print(self):
with nogil:
check_status(DebugPrint(deref(self.ap), 0))
def diff(self, Array other):
"""
Return string containing the result of arrow::Diff comparing contents
of this array against the other array
"""
cdef c_string result
with nogil:
result = self.ap.Diff(deref(other.ap))
return frombytes(result)
def cast(self, object target_type, bint safe=True):
"""
Cast array values to another data type.
Example
-------
>>> from datetime import datetime
>>> import pyarrow as pa
>>> arr = pa.array([datetime(2010, 1, 1), datetime(2015, 1, 1)])
>>> arr.type
TimestampType(timestamp[us])
You can use ``pyarrow.DataType`` objects to specify the target type:
>>> arr.cast(pa.timestamp('ms'))
<pyarrow.lib.TimestampArray object at 0x10420eb88>
[
1262304000000,
1420070400000
]
>>> arr.cast(pa.timestamp('ms')).type
TimestampType(timestamp[ms])
Alternatively, it is also supported to use the string aliases for these
types:
>>> arr.cast('timestamp[ms]')
<pyarrow.lib.TimestampArray object at 0x10420eb88>
[
1262304000000,
1420070400000
]
>>> arr.cast('timestamp[ms]').type
TimestampType(timestamp[ms])
Parameters
----------
target_type : DataType
Type to cast to
safe : boolean, default True
Check for overflows or other unsafe conversions
Returns
-------
casted : Array
"""
cdef:
CCastOptions options = CCastOptions(safe)
DataType type = ensure_type(target_type)
shared_ptr[CArray] result
with nogil:
check_status(Cast(_context(), self.ap[0], type.sp_type,
options, &result))
return pyarrow_wrap_array(result)
def view(self, object target_type):
"""Return zero-copy "view" of array as another data type. The data
types must have compatible columnar buffer layouts
Parameters
----------
target_type : DataType
Type to construct view as
Returns
-------
view : Array
"""
cdef DataType type = ensure_type(target_type)
cdef shared_ptr[CArray] result
with nogil:
check_status(self.ap.View(type.sp_type, &result))
return pyarrow_wrap_array(result)
def sum(self):
"""
Sum the values in a numerical array.
"""
cdef CDatum out
with nogil:
check_status(Sum(_context(), CDatum(self.sp_array), &out))
return wrap_datum(out)
def unique(self):
"""
Compute distinct elements in array
"""
cdef shared_ptr[CArray] result
with nogil:
check_status(Unique(_context(), CDatum(self.sp_array), &result))
return pyarrow_wrap_array(result)
def dictionary_encode(self):
"""
Compute dictionary-encoded representation of array
"""
cdef CDatum out
with nogil:
check_status(DictionaryEncode(_context(), CDatum(self.sp_array),
&out))
return wrap_datum(out)
@staticmethod
def from_pandas(obj, mask=None, type=None, bint safe=True,
MemoryPool memory_pool=None):
"""
Convert pandas.Series to an Arrow Array, using pandas's semantics about
what values indicate nulls. See pyarrow.array for more general
conversion from arrays or sequences to Arrow arrays.
Parameters
----------
sequence : ndarray, Inded Series
mask : array (boolean), optional
Indicate which values are null (True) or not null (False)
type : pyarrow.DataType
Explicit type to attempt to coerce to, otherwise will be inferred
from the data
safe : boolean, default True
Check for overflows or other unsafe conversions
memory_pool : pyarrow.MemoryPool, optional
If not passed, will allocate memory from the currently-set default
memory pool
Notes
-----
Localized timestamps will currently be returned as UTC (pandas's native
representation). Timezone-naive data will be implicitly interpreted as
UTC.
Returns
-------
array : pyarrow.Array or pyarrow.ChunkedArray (if object data
overflows binary buffer)
"""
return array(obj, mask=mask, type=type, safe=safe, from_pandas=True,
memory_pool=memory_pool)
def __reduce__(self):
return _restore_array, \
(_reduce_array_data(self.sp_array.get().data().get()),)
@staticmethod
def from_buffers(DataType type, length, buffers, null_count=-1, offset=0,
children=None):
"""
Construct an Array from a sequence of buffers. The concrete type
returned depends on the datatype.
Parameters
----------
type : DataType
The value type of the array
length : int
The number of values in the array
buffers: List[Buffer]
The buffers backing this array
null_count : int, default -1
offset : int, default 0
The array's logical offset (in values, not in bytes) from the
start of each buffer
children : List[Array], default None
Nested type children with length matching type.num_children
Returns
-------
array : Array
"""
cdef:
Buffer buf
Array child
vector[shared_ptr[CBuffer]] c_buffers
vector[shared_ptr[CArrayData]] c_child_data
shared_ptr[CArrayData] array_data
children = children or []
if type.num_children != len(children):
raise ValueError("Type's expected number of children "
"({0}) did not match the passed number "
"({1}).".format(type.num_children, len(children)))
if type.num_buffers != len(buffers):
raise ValueError("Type's expected number of buffers "
"({0}) did not match the passed number "
"({1}).".format(type.num_buffers, len(buffers)))
for buf in buffers:
# None will produce a null buffer pointer
c_buffers.push_back(pyarrow_unwrap_buffer(buf))
for child in children:
c_child_data.push_back(child.ap.data())
array_data = CArrayData.MakeWithChildren(type.sp_type, length,
c_buffers, c_child_data,
null_count, offset)
cdef Array result = pyarrow_wrap_array(MakeArray(array_data))
result.validate()
return result
@property
def null_count(self):
return self.sp_array.get().null_count()
def __iter__(self):
for i in range(len(self)):
yield self.getitem(i)
def __repr__(self):
type_format = object.__repr__(self)
return '{0}\n{1}'.format(type_format, str(self))
def format(self, int indent=0, int window=10):
cdef:
c_string result
with nogil:
check_status(
PrettyPrint(
deref(self.ap),
PrettyPrintOptions(indent, window),
&result
)
)
return frombytes(result)
def __str__(self):
return self.format()
def equals(Array self, Array other):
return self.ap.Equals(deref(other.ap))
def __len__(self):
return self.length()
cdef int64_t length(self):
if self.sp_array.get():
return self.sp_array.get().length()
else:
return 0
def isnull(self):
raise NotImplemented
def __getitem__(self, index):
"""
Return the value at the given index.
Returns
-------
value : Scalar
"""
if PySlice_Check(index):
return _normalize_slice(self, index)
return self.getitem(_normalize_index(index, self.length()))
cdef getitem(self, int64_t i):
return box_scalar(self.type, self.sp_array, i)
def slice(self, offset=0, length=None):
"""
Compute zero-copy slice of this array
Parameters
----------
offset : int, default 0
Offset from start of array to slice
length : int, default None
Length of slice (default is until end of Array starting from
offset)
Returns
-------
sliced : RecordBatch
"""
cdef:
shared_ptr[CArray] result
if offset < 0:
raise IndexError('Offset must be non-negative')
if length is None:
result = self.ap.Slice(offset)
else:
result = self.ap.Slice(offset, length)
return pyarrow_wrap_array(result)
def take(self, Array indices):
"""
Take elements from an array.
The resulting array will be of the same type as the input array, with
elements taken from the input array at the given indices. If an index
is null then the taken element will be null.
Parameters
----------
indices : Array
The indices of the values to extract. Array needs to be of
integer type.
Returns
-------
Array
Examples
--------
>>> import pyarrow as pa
>>> arr = pa.array(["a", "b", "c", None, "e", "f"])
>>> indices = pa.array([0, None, 4, 3])
>>> arr.take(indices)
<pyarrow.lib.StringArray object at 0x7ffa4fc7d368>
[
"a",
null,
"e",
null
]
"""
cdef:
cdef CTakeOptions options
cdef CDatum out
with nogil:
check_status(Take(_context(), CDatum(self.sp_array),
CDatum(indices.sp_array), options, &out))
return wrap_datum(out)
def filter(self, Array mask):
"""
Filter the array with a boolean mask.
Parameters
----------
mask : Array
The boolean mask indicating which values to extract.
Returns
-------
Array
Examples
--------
>>> import pyarrow as pa
>>> arr = pa.array(["a", "b", "c", None, "e"])
>>> mask = pa.array([True, False, None, False, True])
>>> arr.filter(mask)
<pyarrow.lib.StringArray object at 0x7fa826df9200>
[
"a",
null,
"e"
]
"""
cdef:
cdef CDatum out
with nogil:
check_status(FilterKernel(_context(), CDatum(self.sp_array),
CDatum(mask.sp_array), &out))
return wrap_datum(out)
def _to_pandas(self, options, **kwargs):
cdef:
PyObject* out
PandasOptions c_options = _convert_pandas_options(options)
with nogil:
check_status(ConvertArrayToPandas(c_options, self.sp_array,
self, &out))
result = pandas_api.series(wrap_array_output(out), name=self._name)
if isinstance(self.type, TimestampType) and self.type.tz is not None:
from pyarrow.pandas_compat import make_tz_aware
result = make_tz_aware(result, self.type.tz)
return result
def __array__(self, dtype=None):
cdef:
PyObject* out
PandasOptions c_options
object values
with nogil:
check_status(ConvertArrayToPandas(c_options, self.sp_array,
self, &out))
# wrap_array_output uses pandas to convert to Categorical, here
# always convert to numpy array
values = PyObject_to_object(out)
if isinstance(values, dict):
values = np.take(values['dictionary'], values['indices'])
if dtype is None:
return values
return values.astype(dtype)
def to_numpy(self):
"""
Experimental: return a NumPy view of this array. Only primitive
arrays with the same memory layout as NumPy (i.e. integers,
floating point), without any nulls, are supported.
Returns
-------
array : numpy.ndarray
"""
if self.null_count:
raise NotImplementedError('NumPy array view is only supported '
'for arrays without nulls.')
if not is_primitive(self.type.id) or self.type.id == _Type_BOOL:
raise NotImplementedError('NumPy array view is only supported '
'for primitive types.')
buflist = self.buffers()
assert len(buflist) == 2
return np.frombuffer(buflist[-1], dtype=self.type.to_pandas_dtype())[
self.offset:self.offset + len(self)]
def to_pylist(self):
"""
Convert to a list of native Python objects.
Returns
-------
lst : list
"""
return [x.as_py() for x in self]
def validate(self):
"""
Perform any validation checks implemented by
arrow::Array::Validate(). Raises exception with error message if
array does not validate.
Raises
------
ArrowInvalid
"""
with nogil:
check_status(self.ap.Validate())
@property
def offset(self):
"""
A relative position into another array's data, to enable zero-copy
slicing. This value defaults to zero but must be applied on all
operations with the physical storage buffers.
"""
return self.sp_array.get().offset()
def buffers(self):
"""
Return a list of Buffer objects pointing to this array's physical
storage.
To correctly interpret these buffers, you need to also apply the offset
multiplied with the size of the stored data type.
"""
res = []
_append_array_buffers(self.sp_array.get().data().get(), res)
return res
cdef wrap_array_output(PyObject* output):