forked from apache/arrow
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathtable.pxi
More file actions
1065 lines (864 loc) · 28.4 KB
/
table.pxi
File metadata and controls
1065 lines (864 loc) · 28.4 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.
import json
from collections import OrderedDict
try:
import pandas as pd
except ImportError:
# The pure-Python based API works without a pandas installation
pass
else:
import pyarrow.pandas_compat as pdcompat
cdef class ChunkedArray:
"""
Array backed via one or more memory chunks.
Warning
-------
Do not call this class's constructor directly.
"""
def __cinit__(self):
self.chunked_array = NULL
cdef void init(self, const shared_ptr[CChunkedArray]& chunked_array):
self.sp_chunked_array = chunked_array
self.chunked_array = chunked_array.get()
property type:
def __get__(self):
return pyarrow_wrap_data_type(self.sp_chunked_array.get().type())
cdef int _check_nullptr(self) except -1:
if self.chunked_array == NULL:
raise ReferenceError(
"{} object references a NULL pointer. Not initialized.".format(
type(self).__name__
)
)
return 0
def length(self):
self._check_nullptr()
return self.chunked_array.length()
def __len__(self):
return self.length()
@property
def null_count(self):
"""
Number of null entires
Returns
-------
int
"""
self._check_nullptr()
return self.chunked_array.null_count()
@property
def num_chunks(self):
"""
Number of underlying chunks
Returns
-------
int
"""
self._check_nullptr()
return self.chunked_array.num_chunks()
def chunk(self, i):
"""
Select a chunk by its index
Parameters
----------
i : int
Returns
-------
pyarrow.Array
"""
self._check_nullptr()
if i >= self.num_chunks or i < 0:
raise IndexError('Chunk index out of range.')
return pyarrow_wrap_array(self.chunked_array.chunk(i))
def iterchunks(self):
for i in range(self.num_chunks):
yield self.chunk(i)
def to_pylist(self):
"""
Convert to a list of native Python objects.
"""
result = []
for i in range(self.num_chunks):
result += self.chunk(i).to_pylist()
return result
cdef class Column:
"""
Named vector of elements of equal type.
Warning
-------
Do not call this class's constructor directly.
"""
def __cinit__(self):
self.column = NULL
cdef void init(self, const shared_ptr[CColumn]& column):
self.sp_column = column
self.column = column.get()
def __repr__(self):
from pyarrow.compat import StringIO
result = StringIO()
result.write(object.__repr__(self))
data = self.data
for i in range(len(data)):
result.write('\nchunk {0}: {1}'.format(i, repr(data.chunk(0))))
return result.getvalue()
@staticmethod
def from_array(object field_or_name, Array arr):
cdef Field boxed_field
if isinstance(field_or_name, Field):
boxed_field = field_or_name
if arr.type != boxed_field.type:
raise ValueError('Passed field type does not match array')
else:
boxed_field = field(field_or_name, arr.type)
cdef shared_ptr[CColumn] sp_column
sp_column.reset(new CColumn(boxed_field.sp_field, arr.sp_array))
return pyarrow_wrap_column(sp_column)
def to_pandas(self, strings_to_categorical=False, zero_copy_only=False):
"""
Convert the arrow::Column to a pandas.Series
Returns
-------
pandas.Series
"""
cdef:
PyObject* out
PandasOptions options
options = PandasOptions(
strings_to_categorical=strings_to_categorical,
zero_copy_only=zero_copy_only)
with nogil:
check_status(libarrow.ConvertColumnToPandas(options,
self.sp_column,
self, &out))
values = wrap_array_output(out)
result = pd.Series(values, name=self.name)
if isinstance(self.type, TimestampType):
if self.type.tz is not None:
result = (result.dt.tz_localize('utc')
.dt.tz_convert(self.type.tz))
return result
def equals(self, Column other):
"""
Check if contents of two columns are equal
Parameters
----------
other : pyarrow.Column
Returns
-------
are_equal : boolean
"""
cdef:
CColumn* my_col = self.column
CColumn* other_col = other.column
c_bool result
self._check_nullptr()
other._check_nullptr()
with nogil:
result = my_col.Equals(deref(other_col))
return result
def to_pylist(self):
"""
Convert to a list of native Python objects.
"""
return self.data.to_pylist()
cdef int _check_nullptr(self) except -1:
if self.column == NULL:
raise ReferenceError(
"{} object references a NULL pointer. Not initialized.".format(
type(self).__name__
)
)
return 0
def __len__(self):
return self.length()
def length(self):
self._check_nullptr()
return self.column.length()
@property
def shape(self):
"""
Dimensions of this columns
Returns
-------
(int,)
"""
self._check_nullptr()
return (self.length(),)
@property
def null_count(self):
"""
Number of null entires
Returns
-------
int
"""
self._check_nullptr()
return self.column.null_count()
@property
def name(self):
"""
Label of the column
Returns
-------
str
"""
return bytes(self.column.name()).decode('utf8')
@property
def type(self):
"""
Type information for this column
Returns
-------
pyarrow.DataType
"""
return pyarrow_wrap_data_type(self.column.type())
@property
def data(self):
"""
The underlying data
Returns
-------
pyarrow.ChunkedArray
"""
cdef ChunkedArray chunked_array = ChunkedArray()
chunked_array.init(self.column.data())
return chunked_array
cdef shared_ptr[const CKeyValueMetadata] unbox_metadata(dict metadata):
if metadata is None:
return <shared_ptr[const CKeyValueMetadata]> nullptr
cdef:
unordered_map[c_string, c_string] unordered_metadata = metadata
return (<shared_ptr[const CKeyValueMetadata]>
make_shared[CKeyValueMetadata](unordered_metadata))
cdef _schema_from_arrays(arrays, names, dict metadata,
shared_ptr[CSchema]* schema):
cdef:
Column col
c_string c_name
vector[shared_ptr[CField]] fields
shared_ptr[CDataType] type_
Py_ssize_t K = len(arrays)
if K == 0:
schema.reset(new CSchema(fields, unbox_metadata(metadata)))
return
fields.resize(K)
if isinstance(arrays[0], Column):
for i in range(K):
col = arrays[i]
type_ = col.sp_column.get().type()
c_name = tobytes(col.name)
fields[i].reset(new CField(c_name, type_, True))
else:
if names is None:
raise ValueError('Must pass names when constructing '
'from Array objects')
if len(names) != K:
raise ValueError("Length of names ({}) does not match "
"length of arrays ({})".format(len(names), K))
for i in range(K):
val = arrays[i]
if isinstance(val, (Array, ChunkedArray)):
type_ = (<DataType> val.type).sp_type
else:
raise TypeError(type(val))
if names[i] is None:
c_name = tobytes(u'None')
else:
c_name = tobytes(names[i])
fields[i].reset(new CField(c_name, type_, True))
schema.reset(new CSchema(fields, unbox_metadata(metadata)))
cdef class RecordBatch:
"""
Batch of rows of columns of equal length
Warning
-------
Do not call this class's constructor directly, use one of the ``from_*``
methods instead.
"""
def __cinit__(self):
self.batch = NULL
self._schema = None
cdef void init(self, const shared_ptr[CRecordBatch]& batch):
self.sp_batch = batch
self.batch = batch.get()
cdef int _check_nullptr(self) except -1:
if self.batch == NULL:
raise ReferenceError(
"{} object references a NULL pointer. Not initialized.".format(
type(self).__name__
)
)
return 0
def __len__(self):
self._check_nullptr()
return self.batch.num_rows()
def replace_schema_metadata(self, dict metadata=None):
"""
EXPERIMENTAL: Create shallow copy of record batch by replacing schema
key-value metadata with the indicated new metadata (which may be None,
which deletes any existing metadata
Parameters
----------
metadata : dict, default None
Returns
-------
shallow_copy : RecordBatch
"""
cdef shared_ptr[CKeyValueMetadata] c_meta
if metadata is not None:
convert_metadata(metadata, &c_meta)
cdef shared_ptr[CRecordBatch] new_batch
with nogil:
new_batch = self.batch.ReplaceSchemaMetadata(c_meta)
return pyarrow_wrap_batch(new_batch)
@property
def num_columns(self):
"""
Number of columns
Returns
-------
int
"""
self._check_nullptr()
return self.batch.num_columns()
@property
def num_rows(self):
"""
Number of rows
Due to the definition of a RecordBatch, all columns have the same
number of rows.
Returns
-------
int
"""
return len(self)
@property
def schema(self):
"""
Schema of the RecordBatch and its columns
Returns
-------
pyarrow.Schema
"""
cdef Schema schema
self._check_nullptr()
if self._schema is None:
schema = Schema()
schema.init_schema(self.batch.schema())
self._schema = schema
return self._schema
def column(self, i):
"""
Select single column from record batcha
Returns
-------
column : pyarrow.Array
"""
if not -self.num_columns <= i < self.num_columns:
raise IndexError(
'Record batch column index {:d} is out of range'.format(i)
)
return pyarrow_wrap_array(self.batch.column(i))
def __getitem__(self, key):
cdef:
Py_ssize_t start, stop
if isinstance(key, slice):
return _normalize_slice(self, key)
else:
return self.column(key)
def serialize(self, memory_pool=None):
"""
Write RecordBatch to Buffer as encapsulated IPC message
Parameters
----------
memory_pool : MemoryPool, default None
Uses default memory pool if not specified
Returns
-------
serialized : Buffer
"""
cdef:
shared_ptr[CBuffer] buffer
CMemoryPool* pool = maybe_unbox_memory_pool(memory_pool)
with nogil:
check_status(SerializeRecordBatch(deref(self.batch),
pool, &buffer))
return pyarrow_wrap_buffer(buffer)
def slice(self, offset=0, length=None):
"""
Compute zero-copy slice of this RecordBatch
Parameters
----------
offset : int, default 0
Offset from start of array to slice
length : int, default None
Length of slice (default is until end of batch starting from
offset)
Returns
-------
sliced : RecordBatch
"""
cdef shared_ptr[CRecordBatch] result
if offset < 0:
raise IndexError('Offset must be non-negative')
if length is None:
result = self.batch.Slice(offset)
else:
result = self.batch.Slice(offset, length)
return pyarrow_wrap_batch(result)
def equals(self, RecordBatch other):
cdef:
CRecordBatch* my_batch = self.batch
CRecordBatch* other_batch = other.batch
c_bool result
self._check_nullptr()
other._check_nullptr()
with nogil:
result = my_batch.Equals(deref(other_batch))
return result
def to_pydict(self):
"""
Converted the arrow::RecordBatch to an OrderedDict
Returns
-------
OrderedDict
"""
entries = []
for i in range(self.batch.num_columns()):
name = bytes(self.batch.column_name(i)).decode('utf8')
column = self[i].to_pylist()
entries.append((name, column))
return OrderedDict(entries)
def to_pandas(self, nthreads=None):
"""
Convert the arrow::RecordBatch to a pandas DataFrame
Returns
-------
pandas.DataFrame
"""
return Table.from_batches([self]).to_pandas(nthreads=nthreads)
@classmethod
def from_pandas(cls, df, Schema schema=None, bint preserve_index=True,
nthreads=None):
"""
Convert pandas.DataFrame to an Arrow RecordBatch
Parameters
----------
df: pandas.DataFrame
schema: pyarrow.Schema, optional
The expected schema of the RecordBatch. This can be used to
indicate the type of columns if we cannot infer it automatically.
preserve_index : bool, optional
Whether to store the index as an additional column in the resulting
``RecordBatch``.
nthreads : int, default None (may use up to system CPU count threads)
If greater than 1, convert columns to Arrow in parallel using
indicated number of threads
Returns
-------
pyarrow.RecordBatch
"""
names, arrays, metadata = pdcompat.dataframe_to_arrays(
df, schema, preserve_index, nthreads=nthreads
)
return cls.from_arrays(arrays, names, metadata)
@staticmethod
def from_arrays(list arrays, list names, dict metadata=None):
"""
Construct a RecordBatch from multiple pyarrow.Arrays
Parameters
----------
arrays: list of pyarrow.Array
column-wise data vectors
names: list of str
Labels for the columns
Returns
-------
pyarrow.RecordBatch
"""
cdef:
Array arr
c_string c_name
shared_ptr[CSchema] schema
shared_ptr[CRecordBatch] batch
vector[shared_ptr[CArray]] c_arrays
int64_t num_rows
int64_t i
int64_t number_of_arrays = len(arrays)
if len(arrays) > 0:
num_rows = len(arrays[0])
else:
num_rows = 0
_schema_from_arrays(arrays, names, metadata, &schema)
c_arrays.reserve(len(arrays))
for arr in arrays:
c_arrays.push_back(arr.sp_array)
batch.reset(new CRecordBatch(schema, num_rows, c_arrays))
return pyarrow_wrap_batch(batch)
def table_to_blocks(PandasOptions options, Table table, int nthreads,
MemoryPool memory_pool):
cdef:
PyObject* result_obj
shared_ptr[CTable] c_table = table.sp_table
CMemoryPool* pool
pool = maybe_unbox_memory_pool(memory_pool)
with nogil:
check_status(
libarrow.ConvertTableToPandas(
options, c_table, nthreads, pool, &result_obj
)
)
return PyObject_to_object(result_obj)
cdef class Table:
"""
A collection of top-level named, equal length Arrow arrays.
Warning
-------
Do not call this class's constructor directly, use one of the ``from_*``
methods instead.
"""
def __cinit__(self):
self.table = NULL
def __repr__(self):
return 'pyarrow.{}\n{}'.format(type(self).__name__, str(self.schema))
cdef void init(self, const shared_ptr[CTable]& table):
self.sp_table = table
self.table = table.get()
cdef int _check_nullptr(self) except -1:
if self.table == nullptr:
raise ReferenceError(
"Table object references a NULL pointer. Not initialized."
)
return 0
def replace_schema_metadata(self, dict metadata=None):
"""
EXPERIMENTAL: Create shallow copy of table by replacing schema
key-value metadata with the indicated new metadata (which may be None,
which deletes any existing metadata
Parameters
----------
metadata : dict, default None
Returns
-------
shallow_copy : Table
"""
cdef shared_ptr[CKeyValueMetadata] c_meta
if metadata is not None:
convert_metadata(metadata, &c_meta)
cdef shared_ptr[CTable] new_table
with nogil:
new_table = self.table.ReplaceSchemaMetadata(c_meta)
return pyarrow_wrap_table(new_table)
def equals(self, Table other):
"""
Check if contents of two tables are equal
Parameters
----------
other : pyarrow.Table
Returns
-------
are_equal : boolean
"""
cdef:
CTable* my_table = self.table
CTable* other_table = other.table
c_bool result
self._check_nullptr()
other._check_nullptr()
with nogil:
result = my_table.Equals(deref(other_table))
return result
@classmethod
def from_pandas(cls, df, Schema schema=None, bint preserve_index=True,
nthreads=None):
"""
Convert pandas.DataFrame to an Arrow Table
Parameters
----------
df : pandas.DataFrame
schema : pyarrow.Schema, optional
The expected schema of the Arrow Table. This can be used to
indicate the type of columns if we cannot infer it automatically.
preserve_index : bool, optional
Whether to store the index as an additional column in the resulting
``Table``.
nthreads : int, default None (may use up to system CPU count threads)
If greater than 1, convert columns to Arrow in parallel using
indicated number of threads
Returns
-------
pyarrow.Table
Examples
--------
>>> import pandas as pd
>>> import pyarrow as pa
>>> df = pd.DataFrame({
... 'int': [1, 2],
... 'str': ['a', 'b']
... })
>>> pa.Table.from_pandas(df)
<pyarrow.lib.Table object at 0x7f05d1fb1b40>
"""
names, arrays, metadata = pdcompat.dataframe_to_arrays(
df,
schema=schema,
preserve_index=preserve_index,
nthreads=nthreads
)
return cls.from_arrays(arrays, names=names, metadata=metadata)
@staticmethod
def from_arrays(arrays, names=None, schema=None, dict metadata=None):
"""
Construct a Table from Arrow arrays or columns
Parameters
----------
arrays: list of pyarrow.Array or pyarrow.Column
Equal-length arrays that should form the table.
names: list of str, optional
Names for the table columns. If Columns passed, will be
inferred. If Arrays passed, this argument is required
Returns
-------
pyarrow.Table
"""
cdef:
vector[shared_ptr[CColumn]] columns
Schema cy_schema
shared_ptr[CSchema] c_schema
shared_ptr[CTable] table
int i, K = <int> len(arrays)
if schema is None:
_schema_from_arrays(arrays, names, metadata, &c_schema)
elif schema is not None:
if names is not None:
raise ValueError('Cannot pass schema and arrays')
cy_schema = schema
if len(schema) != len(arrays):
raise ValueError('Schema and number of arrays unequal')
c_schema = cy_schema.sp_schema
columns.reserve(K)
for i in range(K):
if isinstance(arrays[i], Array):
columns.push_back(
make_shared[CColumn](
c_schema.get().field(i),
(<Array> arrays[i]).sp_array
)
)
elif isinstance(arrays[i], ChunkedArray):
columns.push_back(
make_shared[CColumn](
c_schema.get().field(i),
(<ChunkedArray> arrays[i]).sp_chunked_array
)
)
elif isinstance(arrays[i], Column):
# Make sure schema field and column are consistent
columns.push_back(
make_shared[CColumn](
c_schema.get().field(i),
(<Column> arrays[i]).sp_column.get().data()
)
)
else:
raise ValueError(type(arrays[i]))
table.reset(new CTable(c_schema, columns))
return pyarrow_wrap_table(table)
@staticmethod
def from_batches(batches):
"""
Construct a Table from a list of Arrow RecordBatches
Parameters
----------
batches: list of RecordBatch
RecordBatch list to be converted, schemas must be equal
"""
cdef:
vector[shared_ptr[CRecordBatch]] c_batches
shared_ptr[CTable] c_table
RecordBatch batch
for batch in batches:
c_batches.push_back(batch.sp_batch)
with nogil:
check_status(CTable.FromRecordBatches(c_batches, &c_table))
return pyarrow_wrap_table(c_table)
def to_pandas(self, nthreads=None, strings_to_categorical=False,
memory_pool=None, zero_copy_only=False):
"""
Convert the arrow::Table to a pandas DataFrame
Parameters
----------
nthreads : int, default max(1, multiprocessing.cpu_count() / 2)
For the default, we divide the CPU count by 2 because most modern
computers have hyperthreading turned on, so doubling the CPU count
beyond the number of physical cores does not help
strings_to_categorical : boolean, default False
Encode string (UTF8) and binary types to pandas.Categorical
memory_pool: MemoryPool, optional
Specific memory pool to use to allocate casted columns
zero_copy_only : boolean, default False
Raise an ArrowException if this function call would require copying
the underlying data
Returns
-------
pandas.DataFrame
"""
cdef:
PandasOptions options
options = PandasOptions(
strings_to_categorical=strings_to_categorical,
zero_copy_only=zero_copy_only)
self._check_nullptr()
if nthreads is None:
nthreads = cpu_count()
mgr = pdcompat.table_to_blockmanager(options, self, memory_pool,
nthreads)
return pd.DataFrame(mgr)
def to_pydict(self):
"""
Converted the arrow::Table to an OrderedDict
Returns
-------
OrderedDict
"""
cdef:
size_t i
size_t num_columns = self.table.num_columns()
list entries = []
Column column
self._check_nullptr()
for i in range(num_columns):
column = self.column(i)
entries.append((column.name, column.to_pylist()))
return OrderedDict(entries)
@property
def schema(self):
"""
Schema of the table and its columns
Returns
-------
pyarrow.Schema
"""
self._check_nullptr()
return pyarrow_wrap_schema(self.table.schema())
def column(self, int i):
"""
Select a column by its numeric index.
Parameters
----------
i : int
Returns
-------
pyarrow.Column
"""
cdef:
Column column = Column()
int num_columns = self.num_columns
int index
self._check_nullptr()
if not -num_columns <= i < num_columns:
raise IndexError(
'Table column index {:d} is out of range'.format(i)
)
index = i if i >= 0 else num_columns + i
assert index >= 0
column.init(self.table.column(index))
return column
def __getitem__(self, int64_t i):
return self.column(i)
def itercolumns(self):
"""
Iterator over all columns in their numerical order
"""
for i in range(self.num_columns):
yield self.column(i)
@property
def num_columns(self):
"""
Number of columns in this table
Returns
-------
int
"""
self._check_nullptr()
return self.table.num_columns()
@property
def num_rows(self):
"""
Number of rows in this table.
Due to the definition of a table, all columns have the same number of
rows.
Returns
-------
int
"""
self._check_nullptr()
return self.table.num_rows()
def __len__(self):
return self.num_rows