ARROW-439: [Python] Add option in "to_pandas" conversions to yield Categorical from...
[arrow.git] / python / pyarrow / table.pxi
1 # Licensed to the Apache Software Foundation (ASF) under one
2 # or more contributor license agreements.  See the NOTICE file
3 # distributed with this work for additional information
4 # regarding copyright ownership.  The ASF licenses this file
5 # to you under the Apache License, Version 2.0 (the
6 # "License"); you may not use this file except in compliance
7 # with the License.  You may obtain a copy of the License at
8 #
9 #   http://www.apache.org/licenses/LICENSE-2.0
10 #
11 # Unless required by applicable law or agreed to in writing,
12 # software distributed under the License is distributed on an
13 # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14 # KIND, either express or implied.  See the License for the
15 # specific language governing permissions and limitations
16 # under the License.
17
18 import json
19
20 from collections import OrderedDict
21
22 try:
23     import pandas as pd
24 except ImportError:
25     # The pure-Python based API works without a pandas installation
26     pass
27 else:
28     import pyarrow.pandas_compat as pdcompat
29
30
31 cdef class ChunkedArray:
32     """
33     Array backed via one or more memory chunks.
34
35     Warning
36     -------
37     Do not call this class's constructor directly.
38     """
39
40     def __cinit__(self):
41         self.chunked_array = NULL
42
43     cdef void init(self, const shared_ptr[CChunkedArray]& chunked_array):
44         self.sp_chunked_array = chunked_array
45         self.chunked_array = chunked_array.get()
46
47     property type:
48
49         def __get__(self):
50             return pyarrow_wrap_data_type(self.sp_chunked_array.get().type())
51
52     cdef int _check_nullptr(self) except -1:
53         if self.chunked_array == NULL:
54             raise ReferenceError(
55                 "{} object references a NULL pointer. Not initialized.".format(
56                     type(self).__name__
57                 )
58             )
59         return 0
60
61     def length(self):
62         self._check_nullptr()
63         return self.chunked_array.length()
64
65     def __len__(self):
66         return self.length()
67
68     @property
69     def null_count(self):
70         """
71         Number of null entires
72
73         Returns
74         -------
75         int
76         """
77         self._check_nullptr()
78         return self.chunked_array.null_count()
79
80     @property
81     def num_chunks(self):
82         """
83         Number of underlying chunks
84
85         Returns
86         -------
87         int
88         """
89         self._check_nullptr()
90         return self.chunked_array.num_chunks()
91
92     def chunk(self, i):
93         """
94         Select a chunk by its index
95
96         Parameters
97         ----------
98         i : int
99
100         Returns
101         -------
102         pyarrow.Array
103         """
104         self._check_nullptr()
105         return pyarrow_wrap_array(self.chunked_array.chunk(i))
106
107     def iterchunks(self):
108         for i in range(self.num_chunks):
109             yield self.chunk(i)
110
111     def to_pylist(self):
112         """
113         Convert to a list of native Python objects.
114         """
115         result = []
116         for i in range(self.num_chunks):
117             result += self.chunk(i).to_pylist()
118         return result
119
120
121 cdef class Column:
122     """
123     Named vector of elements of equal type.
124
125     Warning
126     -------
127     Do not call this class's constructor directly.
128     """
129
130     def __cinit__(self):
131         self.column = NULL
132
133     cdef void init(self, const shared_ptr[CColumn]& column):
134         self.sp_column = column
135         self.column = column.get()
136
137     def __repr__(self):
138         from pyarrow.compat import StringIO
139         result = StringIO()
140         result.write(object.__repr__(self))
141         data = self.data
142         for i in range(len(data)):
143             result.write('\nchunk {0}: {1}'.format(i, repr(data.chunk(0))))
144
145         return result.getvalue()
146
147     @staticmethod
148     def from_array(object field_or_name, Array arr):
149         cdef Field boxed_field
150
151         if isinstance(field_or_name, Field):
152             boxed_field = field_or_name
153         else:
154             boxed_field = field(field_or_name, arr.type)
155
156         cdef shared_ptr[CColumn] sp_column
157         sp_column.reset(new CColumn(boxed_field.sp_field, arr.sp_array))
158         return pyarrow_wrap_column(sp_column)
159
160     def to_pandas(self, strings_to_categorical=False):
161         """
162         Convert the arrow::Column to a pandas.Series
163
164         Returns
165         -------
166         pandas.Series
167         """
168         cdef:
169             PyObject* out
170             PandasOptions options
171
172         options = PandasOptions(strings_to_categorical=strings_to_categorical)
173
174         with nogil:
175             check_status(libarrow.ConvertColumnToPandas(options,
176                                                         self.sp_column,
177                                                         self, &out))
178
179         return pd.Series(wrap_array_output(out), name=self.name)
180
181     def equals(self, Column other):
182         """
183         Check if contents of two columns are equal
184
185         Parameters
186         ----------
187         other : pyarrow.Column
188
189         Returns
190         -------
191         are_equal : boolean
192         """
193         cdef:
194             CColumn* my_col = self.column
195             CColumn* other_col = other.column
196             c_bool result
197
198         self._check_nullptr()
199         other._check_nullptr()
200
201         with nogil:
202             result = my_col.Equals(deref(other_col))
203
204         return result
205
206     def to_pylist(self):
207         """
208         Convert to a list of native Python objects.
209         """
210         return self.data.to_pylist()
211
212     cdef int _check_nullptr(self) except -1:
213         if self.column == NULL:
214             raise ReferenceError(
215                 "{} object references a NULL pointer. Not initialized.".format(
216                     type(self).__name__
217                 )
218             )
219         return 0
220
221     def __len__(self):
222         return self.length()
223
224     def length(self):
225         self._check_nullptr()
226         return self.column.length()
227
228     @property
229     def shape(self):
230         """
231         Dimensions of this columns
232
233         Returns
234         -------
235         (int,)
236         """
237         self._check_nullptr()
238         return (self.length(),)
239
240     @property
241     def null_count(self):
242         """
243         Number of null entires
244
245         Returns
246         -------
247         int
248         """
249         self._check_nullptr()
250         return self.column.null_count()
251
252     @property
253     def name(self):
254         """
255         Label of the column
256
257         Returns
258         -------
259         str
260         """
261         return bytes(self.column.name()).decode('utf8')
262
263     @property
264     def type(self):
265         """
266         Type information for this column
267
268         Returns
269         -------
270         pyarrow.DataType
271         """
272         return pyarrow_wrap_data_type(self.column.type())
273
274     @property
275     def data(self):
276         """
277         The underlying data
278
279         Returns
280         -------
281         pyarrow.ChunkedArray
282         """
283         cdef ChunkedArray chunked_array = ChunkedArray()
284         chunked_array.init(self.column.data())
285         return chunked_array
286
287
288 cdef shared_ptr[const CKeyValueMetadata] unbox_metadata(dict metadata):
289     if metadata is None:
290         return <shared_ptr[const CKeyValueMetadata]> nullptr
291     cdef:
292         unordered_map[c_string, c_string] unordered_metadata = metadata
293     return (<shared_ptr[const CKeyValueMetadata]>
294             make_shared[CKeyValueMetadata](unordered_metadata))
295
296
297 cdef int _schema_from_arrays(
298         arrays, names, dict metadata, shared_ptr[CSchema]* schema) except -1:
299     cdef:
300         Column col
301         c_string c_name
302         vector[shared_ptr[CField]] fields
303         shared_ptr[CDataType] type_
304         Py_ssize_t K = len(arrays)
305
306     fields.resize(K)
307
308     if len(arrays) == 0:
309         raise ValueError('Must pass at least one array')
310
311     if isinstance(arrays[0], Column):
312         for i in range(K):
313             col = arrays[i]
314             type_ = col.sp_column.get().type()
315             c_name = tobytes(col.name)
316             fields[i].reset(new CField(c_name, type_, True))
317     else:
318         if names is None:
319             raise ValueError('Must pass names when constructing '
320                              'from Array objects')
321         for i in range(K):
322             val = arrays[i]
323             if isinstance(val, (Array, ChunkedArray)):
324                 type_ = (<DataType> val.type).sp_type
325             else:
326                 raise TypeError(type(val))
327
328             c_name = tobytes(names[i])
329             fields[i].reset(new CField(c_name, type_, True))
330
331     schema.reset(new CSchema(fields, unbox_metadata(metadata)))
332     return 0
333
334
335 cdef class RecordBatch:
336     """
337     Batch of rows of columns of equal length
338
339     Warning
340     -------
341     Do not call this class's constructor directly, use one of the ``from_*``
342     methods instead.
343     """
344
345     def __cinit__(self):
346         self.batch = NULL
347         self._schema = None
348
349     cdef void init(self, const shared_ptr[CRecordBatch]& batch):
350         self.sp_batch = batch
351         self.batch = batch.get()
352
353     cdef int _check_nullptr(self) except -1:
354         if self.batch == NULL:
355             raise ReferenceError(
356                 "{} object references a NULL pointer. Not initialized.".format(
357                     type(self).__name__
358                 )
359             )
360         return 0
361
362     def __len__(self):
363         self._check_nullptr()
364         return self.batch.num_rows()
365
366     def replace_schema_metadata(self, dict metadata=None):
367         """
368         EXPERIMENTAL: Create shallow copy of record batch by replacing schema
369         key-value metadata with the indicated new metadata (which may be None,
370         which deletes any existing metadata
371
372         Parameters
373         ----------
374         metadata : dict, default None
375
376         Returns
377         -------
378         shallow_copy : RecordBatch
379         """
380         cdef shared_ptr[CKeyValueMetadata] c_meta
381         if metadata is not None:
382             convert_metadata(metadata, &c_meta)
383
384         cdef shared_ptr[CRecordBatch] new_batch
385         with nogil:
386             new_batch = self.batch.ReplaceSchemaMetadata(c_meta)
387
388         return pyarrow_wrap_batch(new_batch)
389
390     @property
391     def num_columns(self):
392         """
393         Number of columns
394
395         Returns
396         -------
397         int
398         """
399         self._check_nullptr()
400         return self.batch.num_columns()
401
402     @property
403     def num_rows(self):
404         """
405         Number of rows
406
407         Due to the definition of a RecordBatch, all columns have the same
408         number of rows.
409
410         Returns
411         -------
412         int
413         """
414         return len(self)
415
416     @property
417     def schema(self):
418         """
419         Schema of the RecordBatch and its columns
420
421         Returns
422         -------
423         pyarrow.Schema
424         """
425         cdef Schema schema
426         self._check_nullptr()
427         if self._schema is None:
428             schema = Schema()
429             schema.init_schema(self.batch.schema())
430             self._schema = schema
431
432         return self._schema
433
434     def column(self, i):
435         """
436         Select single column from record batcha
437
438         Returns
439         -------
440         column : pyarrow.Array
441         """
442         if not -self.num_columns <= i < self.num_columns:
443             raise IndexError(
444                 'Record batch column index {:d} is out of range'.format(i)
445             )
446         return pyarrow_wrap_array(self.batch.column(i))
447
448     def __getitem__(self, key):
449         cdef:
450             Py_ssize_t start, stop
451         if isinstance(key, slice):
452             return _normalize_slice(self, key)
453         else:
454             return self.column(key)
455
456     def slice(self, offset=0, length=None):
457         """
458         Compute zero-copy slice of this RecordBatch
459
460         Parameters
461         ----------
462         offset : int, default 0
463             Offset from start of array to slice
464         length : int, default None
465             Length of slice (default is until end of batch starting from
466             offset)
467
468         Returns
469         -------
470         sliced : RecordBatch
471         """
472         cdef shared_ptr[CRecordBatch] result
473
474         if offset < 0:
475             raise IndexError('Offset must be non-negative')
476
477         if length is None:
478             result = self.batch.Slice(offset)
479         else:
480             result = self.batch.Slice(offset, length)
481
482         return pyarrow_wrap_batch(result)
483
484     def equals(self, RecordBatch other):
485         cdef:
486             CRecordBatch* my_batch = self.batch
487             CRecordBatch* other_batch = other.batch
488             c_bool result
489
490         self._check_nullptr()
491         other._check_nullptr()
492
493         with nogil:
494             result = my_batch.Equals(deref(other_batch))
495
496         return result
497
498     def to_pydict(self):
499         """
500         Converted the arrow::RecordBatch to an OrderedDict
501
502         Returns
503         -------
504         OrderedDict
505         """
506         entries = []
507         for i in range(self.batch.num_columns()):
508             name = bytes(self.batch.column_name(i)).decode('utf8')
509             column = self[i].to_pylist()
510             entries.append((name, column))
511         return OrderedDict(entries)
512
513     def to_pandas(self, nthreads=None):
514         """
515         Convert the arrow::RecordBatch to a pandas DataFrame
516
517         Returns
518         -------
519         pandas.DataFrame
520         """
521         return Table.from_batches([self]).to_pandas(nthreads=nthreads)
522
523     @classmethod
524     def from_pandas(cls, df, Schema schema=None, bint preserve_index=True):
525         """
526         Convert pandas.DataFrame to an Arrow RecordBatch
527
528         Parameters
529         ----------
530         df: pandas.DataFrame
531         schema: pyarrow.Schema, optional
532             The expected schema of the RecordBatch. This can be used to
533             indicate the type of columns if we cannot infer it automatically.
534         preserve_index : bool, optional
535             Whether to store the index as an additional column in the resulting
536             ``RecordBatch``.
537
538         Returns
539         -------
540         pyarrow.RecordBatch
541         """
542         names, arrays, metadata = pdcompat.dataframe_to_arrays(
543             df, False, schema, preserve_index
544         )
545         return cls.from_arrays(arrays, names, metadata)
546
547     @staticmethod
548     def from_arrays(list arrays, list names, dict metadata=None):
549         """
550         Construct a RecordBatch from multiple pyarrow.Arrays
551
552         Parameters
553         ----------
554         arrays: list of pyarrow.Array
555             column-wise data vectors
556         names: list of str
557             Labels for the columns
558
559         Returns
560         -------
561         pyarrow.RecordBatch
562         """
563         cdef:
564             Array arr
565             c_string c_name
566             shared_ptr[CSchema] schema
567             shared_ptr[CRecordBatch] batch
568             vector[shared_ptr[CArray]] c_arrays
569             int64_t num_rows
570             int64_t i
571             int64_t number_of_arrays = len(arrays)
572
573         if not number_of_arrays:
574             raise ValueError('Record batch cannot contain no arrays (for now)')
575
576         num_rows = len(arrays[0])
577         _schema_from_arrays(arrays, names, metadata, &schema)
578
579         c_arrays.reserve(len(arrays))
580         for arr in arrays:
581             c_arrays.push_back(arr.sp_array)
582
583         batch.reset(new CRecordBatch(schema, num_rows, c_arrays))
584         return pyarrow_wrap_batch(batch)
585
586
587 def table_to_blocks(PandasOptions options, Table table, int nthreads,
588                     MemoryPool memory_pool):
589     cdef:
590         PyObject* result_obj
591         shared_ptr[CTable] c_table = table.sp_table
592         CMemoryPool* pool
593
594     pool = maybe_unbox_memory_pool(memory_pool)
595     with nogil:
596         check_status(
597             libarrow.ConvertTableToPandas(
598                 options, c_table, nthreads, pool, &result_obj
599             )
600         )
601
602     return PyObject_to_object(result_obj)
603
604
605 cdef class Table:
606     """
607     A collection of top-level named, equal length Arrow arrays.
608
609     Warning
610     -------
611     Do not call this class's constructor directly, use one of the ``from_*``
612     methods instead.
613     """
614
615     def __cinit__(self):
616         self.table = NULL
617
618     def __repr__(self):
619         return 'pyarrow.{}\n{}'.format(type(self).__name__, str(self.schema))
620
621     cdef void init(self, const shared_ptr[CTable]& table):
622         self.sp_table = table
623         self.table = table.get()
624
625     cdef int _check_nullptr(self) except -1:
626         if self.table == nullptr:
627             raise ReferenceError(
628                 "Table object references a NULL pointer. Not initialized."
629             )
630         return 0
631
632     def replace_schema_metadata(self, dict metadata=None):
633         """
634         EXPERIMENTAL: Create shallow copy of table by replacing schema
635         key-value metadata with the indicated new metadata (which may be None,
636         which deletes any existing metadata
637
638         Parameters
639         ----------
640         metadata : dict, default None
641
642         Returns
643         -------
644         shallow_copy : Table
645         """
646         cdef shared_ptr[CKeyValueMetadata] c_meta
647         if metadata is not None:
648             convert_metadata(metadata, &c_meta)
649
650         cdef shared_ptr[CTable] new_table
651         with nogil:
652             new_table = self.table.ReplaceSchemaMetadata(c_meta)
653
654         return pyarrow_wrap_table(new_table)
655
656     def equals(self, Table other):
657         """
658         Check if contents of two tables are equal
659
660         Parameters
661         ----------
662         other : pyarrow.Table
663
664         Returns
665         -------
666         are_equal : boolean
667         """
668         cdef:
669             CTable* my_table = self.table
670             CTable* other_table = other.table
671             c_bool result
672
673         self._check_nullptr()
674         other._check_nullptr()
675
676         with nogil:
677             result = my_table.Equals(deref(other_table))
678
679         return result
680
681     @classmethod
682     def from_pandas(cls, df, bint timestamps_to_ms=False,
683                     Schema schema=None, bint preserve_index=True):
684         """
685         Convert pandas.DataFrame to an Arrow Table
686
687         Parameters
688         ----------
689         df : pandas.DataFrame
690         timestamps_to_ms : bool
691             Convert datetime columns to ms resolution. This is needed for
692             compability with other functionality like Parquet I/O which
693             only supports milliseconds.
694         schema : pyarrow.Schema, optional
695             The expected schema of the Arrow Table. This can be used to
696             indicate the type of columns if we cannot infer it automatically.
697         preserve_index : bool, optional
698             Whether to store the index as an additional column in the resulting
699             ``Table``.
700
701         Returns
702         -------
703         pyarrow.Table
704
705         Examples
706         --------
707
708         >>> import pandas as pd
709         >>> import pyarrow as pa
710         >>> df = pd.DataFrame({
711             ...     'int': [1, 2],
712             ...     'str': ['a', 'b']
713             ... })
714         >>> pa.Table.from_pandas(df)
715         <pyarrow.lib.Table object at 0x7f05d1fb1b40>
716         """
717         names, arrays, metadata = pdcompat.dataframe_to_arrays(
718             df,
719             timestamps_to_ms=timestamps_to_ms,
720             schema=schema,
721             preserve_index=preserve_index
722         )
723         return cls.from_arrays(arrays, names=names, metadata=metadata)
724
725     @staticmethod
726     def from_arrays(arrays, names=None, dict metadata=None):
727         """
728         Construct a Table from Arrow arrays or columns
729
730         Parameters
731         ----------
732         arrays: list of pyarrow.Array or pyarrow.Column
733             Equal-length arrays that should form the table.
734         names: list of str, optional
735             Names for the table columns. If Columns passed, will be
736             inferred. If Arrays passed, this argument is required
737
738         Returns
739         -------
740         pyarrow.Table
741
742         """
743         cdef:
744             vector[shared_ptr[CColumn]] columns
745             shared_ptr[CSchema] schema
746             shared_ptr[CTable] table
747             int i, K = <int> len(arrays)
748
749         _schema_from_arrays(arrays, names, metadata, &schema)
750
751         columns.reserve(K)
752
753         for i in range(K):
754             if isinstance(arrays[i], Array):
755                 columns.push_back(
756                     make_shared[CColumn](
757                         schema.get().field(i),
758                         (<Array> arrays[i]).sp_array
759                     )
760                 )
761             elif isinstance(arrays[i], ChunkedArray):
762                 columns.push_back(
763                     make_shared[CColumn](
764                         schema.get().field(i),
765                         (<ChunkedArray> arrays[i]).sp_chunked_array
766                     )
767                 )
768             elif isinstance(arrays[i], Column):
769                 columns.push_back((<Column> arrays[i]).sp_column)
770             else:
771                 raise ValueError(type(arrays[i]))
772
773         table.reset(new CTable(schema, columns))
774         return pyarrow_wrap_table(table)
775
776     @staticmethod
777     def from_batches(batches):
778         """
779         Construct a Table from a list of Arrow RecordBatches
780
781         Parameters
782         ----------
783
784         batches: list of RecordBatch
785             RecordBatch list to be converted, schemas must be equal
786         """
787         cdef:
788             vector[shared_ptr[CRecordBatch]] c_batches
789             shared_ptr[CTable] c_table
790             RecordBatch batch
791
792         for batch in batches:
793             c_batches.push_back(batch.sp_batch)
794
795         with nogil:
796             check_status(CTable.FromRecordBatches(c_batches, &c_table))
797
798         return pyarrow_wrap_table(c_table)
799
800     def to_pandas(self, nthreads=None, strings_to_categorical=False,
801                   memory_pool=None):
802         """
803         Convert the arrow::Table to a pandas DataFrame
804
805         Parameters
806         ----------
807         nthreads : int, default max(1, multiprocessing.cpu_count() / 2)
808             For the default, we divide the CPU count by 2 because most modern
809             computers have hyperthreading turned on, so doubling the CPU count
810             beyond the number of physical cores does not help
811         strings_to_categorical : boolean, default False
812             Encode string (UTF8) and binary types to pandas.Categorical
813         memory_pool: MemoryPool, optional
814             Specific memory pool to use to allocate casted columns
815
816         Returns
817         -------
818         pandas.DataFrame
819         """
820         cdef:
821             PandasOptions options
822         options = PandasOptions(strings_to_categorical=strings_to_categorical)
823         self._check_nullptr()
824         if nthreads is None:
825             nthreads = cpu_count()
826         mgr = pdcompat.table_to_blockmanager(options, self, memory_pool,
827                                              nthreads)
828         return pd.DataFrame(mgr)
829
830     def to_pydict(self):
831         """
832         Converted the arrow::Table to an OrderedDict
833
834         Returns
835         -------
836         OrderedDict
837         """
838         cdef:
839             size_t i
840             size_t num_columns = self.table.num_columns()
841             list entries = []
842             Column column
843
844         self._check_nullptr()
845         for i in range(num_columns):
846             column = self.column(i)
847             entries.append((column.name, column.to_pylist()))
848
849         return OrderedDict(entries)
850
851     @property
852     def schema(self):
853         """
854         Schema of the table and its columns
855
856         Returns
857         -------
858         pyarrow.Schema
859         """
860         self._check_nullptr()
861         return pyarrow_wrap_schema(self.table.schema())
862
863     def column(self, int i):
864         """
865         Select a column by its numeric index.
866
867         Parameters
868         ----------
869         i : int
870
871         Returns
872         -------
873         pyarrow.Column
874         """
875         cdef:
876             Column column = Column()
877             int num_columns = self.num_columns
878             int index
879
880         self._check_nullptr()
881         if not -num_columns <= i < num_columns:
882             raise IndexError(
883                 'Table column index {:d} is out of range'.format(i)
884             )
885
886         index = i if i >= 0 else num_columns + i
887         assert index >= 0
888
889         column.init(self.table.column(index))
890         return column
891
892     def __getitem__(self, int64_t i):
893         return self.column(i)
894
895     def itercolumns(self):
896         """
897         Iterator over all columns in their numerical order
898         """
899         for i in range(self.num_columns):
900             yield self.column(i)
901
902     @property
903     def num_columns(self):
904         """
905         Number of columns in this table
906
907         Returns
908         -------
909         int
910         """
911         self._check_nullptr()
912         return self.table.num_columns()
913
914     @property
915     def num_rows(self):
916         """
917         Number of rows in this table.
918
919         Due to the definition of a table, all columns have the same number of
920         rows.
921
922         Returns
923         -------
924         int
925         """
926         self._check_nullptr()
927         return self.table.num_rows()
928
929     def __len__(self):
930         return self.num_rows
931
932     @property
933     def shape(self):
934         """
935         Dimensions of the table: (#rows, #columns)
936
937         Returns
938         -------
939         (int, int)
940         """
941         return (self.num_rows, self.num_columns)
942
943     def add_column(self, int i, Column column):
944         """
945         Add column to Table at position. Returns new table
946         """
947         cdef shared_ptr[CTable] c_table
948         self._check_nullptr()
949
950         with nogil:
951             check_status(self.table.AddColumn(i, column.sp_column, &c_table))
952
953         return pyarrow_wrap_table(c_table)
954
955     def append_column(self, Column column):
956         """
957         Append column at end of columns. Returns new table
958         """
959         return self.add_column(self.num_columns, column)
960
961     def remove_column(self, int i):
962         """
963         Create new Table with the indicated column removed
964         """
965         cdef shared_ptr[CTable] c_table
966         self._check_nullptr()
967
968         with nogil:
969             check_status(self.table.RemoveColumn(i, &c_table))
970
971         return pyarrow_wrap_table(c_table)
972
973
974 def concat_tables(tables):
975     """
976     Perform zero-copy concatenation of pyarrow.Table objects. Raises exception
977     if all of the Table schemas are not the same
978
979     Parameters
980     ----------
981     tables : iterable of pyarrow.Table objects
982     output_name : string, default None
983       A name for the output table, if any
984     """
985     cdef:
986         vector[shared_ptr[CTable]] c_tables
987         shared_ptr[CTable] c_result
988         Table table
989
990     for table in tables:
991         c_tables.push_back(table.sp_table)
992
993     with nogil:
994         check_status(ConcatenateTables(c_tables, &c_result))
995
996     return pyarrow_wrap_table(c_result)