This is an automated email from the ASF dual-hosted git repository.
jorisvandenbossche pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git
The following commit(s) were added to refs/heads/master by this push:
new 3b92f0279d ARROW-16018: [Doc][Python] Run doctests on Python docstring
examples (--doctest-modules)
3b92f0279d is described below
commit 3b92f0279dd69b1120b1623cf1e98f8b559f7762
Author: Alenka Frim <[email protected]>
AuthorDate: Wed May 25 12:55:15 2022 +0200
ARROW-16018: [Doc][Python] Run doctests on Python docstring examples
(--doctest-modules)
A series of 3 PRs add `doctest` functionality to ensure that docstring
examples are actually correct (and keep being correct).
- [x] Add `--doctest-module`
- [x] Add `--doctest-cython` https://github.com/apache/arrow/pull/13204
- [x] Create a CI job https://github.com/apache/arrow/pull/13216
This PR can be tested with `pytest --doctest-modules python/pyarrow`.
Closes #13199 from AlenkaF/ARROW-16018
Lead-authored-by: Alenka Frim <[email protected]>
Co-authored-by: Alenka Frim <[email protected]>
Signed-off-by: Joris Van den Bossche <[email protected]>
---
python/pyarrow/_compute_docstrings.py | 8 +-
python/pyarrow/compute.py | 16 +--
python/pyarrow/conftest.py | 226 ++++++++++++++++++++++++++++++++++
python/pyarrow/dataset.py | 101 +++++++++++----
python/pyarrow/pandas_compat.py | 4 +-
python/pyarrow/parquet/__init__.py | 84 +++++++------
python/pyarrow/tests/conftest.py | 148 +---------------------
python/pyarrow/tests/test_compute.py | 4 +-
8 files changed, 365 insertions(+), 226 deletions(-)
diff --git a/python/pyarrow/_compute_docstrings.py
b/python/pyarrow/_compute_docstrings.py
index a88fbbecda..150dbdb117 100644
--- a/python/pyarrow/_compute_docstrings.py
+++ b/python/pyarrow/_compute_docstrings.py
@@ -28,13 +28,13 @@ function_doc_additions["filter"] = """
>>> 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>
+ <pyarrow.lib.StringArray object at ...>
[
"a",
"e"
]
>>> arr.filter(mask, null_selection_behavior='emit_null')
- <pyarrow.lib.StringArray object at 0x7fa826df9200>
+ <pyarrow.lib.StringArray object at ...>
[
"a",
null,
@@ -50,7 +50,7 @@ function_doc_additions["mode"] = """
>>> arr = pa.array([1, 1, 2, 2, 3, 2, 2, 2])
>>> modes = pc.mode(arr, 2)
>>> modes[0]
- <pyarrow.StructScalar: {'mode': 2, 'count': 5}>
+ <pyarrow.StructScalar: [('mode', 2), ('count', 5)]>
>>> modes[1]
- <pyarrow.StructScalar: {'mode': 1, 'count': 2}>
+ <pyarrow.StructScalar: [('mode', 1), ('count', 2)]>
"""
diff --git a/python/pyarrow/compute.py b/python/pyarrow/compute.py
index 99e0da677a..e10536dd10 100644
--- a/python/pyarrow/compute.py
+++ b/python/pyarrow/compute.py
@@ -345,7 +345,7 @@ def cast(arr, target_type, safe=True):
You can use ``pyarrow.DataType`` objects to specify the target type:
>>> cast(arr, pa.timestamp('ms'))
- <pyarrow.lib.TimestampArray object at 0x7fe93c0f6910>
+ <pyarrow.lib.TimestampArray object at ...>
[
2010-01-01 00:00:00.000,
2015-01-01 00:00:00.000
@@ -358,10 +358,10 @@ def cast(arr, target_type, safe=True):
types:
>>> arr.cast('timestamp[ms]')
- <pyarrow.lib.TimestampArray object at 0x10420eb88>
+ <pyarrow.lib.TimestampArray object at ...>
[
- 1262304000000,
- 1420070400000
+ 2010-01-01 00:00:00.000,
+ 2015-01-01 00:00:00.000
]
>>> arr.cast('timestamp[ms]').type
TimestampType(timestamp[ms])
@@ -448,7 +448,7 @@ def take(data, indices, *, boundscheck=True,
memory_pool=None):
>>> 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>
+ <pyarrow.lib.StringArray object at ...>
[
"a",
null,
@@ -486,7 +486,7 @@ def fill_null(values, fill_value):
>>> arr = pa.array([1, 2, None, 3], type=pa.int8())
>>> fill_value = pa.scalar(5, type=pa.int8())
>>> arr.fill_null(fill_value)
- pyarrow.lib.Int8Array object at 0x7f95437f01a0>
+ <pyarrow.lib.Int8Array object at ...>
[
1,
2,
@@ -531,7 +531,7 @@ def top_k_unstable(values, k, sort_keys=None, *,
memory_pool=None):
>>> import pyarrow.compute as pc
>>> arr = pa.array(["a", "b", "c", None, "e", "f"])
>>> pc.top_k_unstable(arr, k=3)
- <pyarrow.lib.UInt64Array object at 0x7fdcb19d7f30>
+ <pyarrow.lib.UInt64Array object at ...>
[
5,
4,
@@ -577,7 +577,7 @@ def bottom_k_unstable(values, k, sort_keys=None, *,
memory_pool=None):
>>> import pyarrow.compute as pc
>>> arr = pa.array(["a", "b", "c", None, "e", "f"])
>>> pc.bottom_k_unstable(arr, k=3)
- <pyarrow.lib.UInt64Array object at 0x7fdcb19d7fa0>
+ <pyarrow.lib.UInt64Array object at ...>
[
0,
1,
diff --git a/python/pyarrow/conftest.py b/python/pyarrow/conftest.py
new file mode 100644
index 0000000000..8423d41460
--- /dev/null
+++ b/python/pyarrow/conftest.py
@@ -0,0 +1,226 @@
+# 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 pytest
+from pyarrow import Codec
+
+groups = [
+ 'brotli',
+ 'bz2',
+ 'cython',
+ 'dataset',
+ 'hypothesis',
+ 'fastparquet',
+ 'gandiva',
+ 'gdb',
+ 'gzip',
+ 'hdfs',
+ 'large_memory',
+ 'lz4',
+ 'memory_leak',
+ 'nopandas',
+ 'orc',
+ 'pandas',
+ 'parquet',
+ 'parquet_encryption',
+ 'plasma',
+ 's3',
+ 'snappy',
+ 'substrait',
+ 'tensorflow',
+ 'flight',
+ 'slow',
+ 'requires_testing_data',
+ 'zstd',
+]
+
+defaults = {
+ 'brotli': Codec.is_available('brotli'),
+ 'bz2': Codec.is_available('bz2'),
+ 'cython': False,
+ 'dataset': False,
+ 'fastparquet': False,
+ 'flight': False,
+ 'gandiva': False,
+ 'gdb': True,
+ 'gzip': Codec.is_available('gzip'),
+ 'hdfs': False,
+ 'hypothesis': False,
+ 'large_memory': False,
+ 'lz4': Codec.is_available('lz4'),
+ 'memory_leak': False,
+ 'nopandas': False,
+ 'orc': False,
+ 'pandas': False,
+ 'parquet': False,
+ 'parquet_encryption': False,
+ 'plasma': False,
+ 'requires_testing_data': True,
+ 's3': False,
+ 'slow': False,
+ 'snappy': Codec.is_available('snappy'),
+ 'substrait': False,
+ 'tensorflow': False,
+ 'zstd': Codec.is_available('zstd'),
+}
+
+try:
+ import cython # noqa
+ defaults['cython'] = True
+except ImportError:
+ pass
+
+try:
+ import fastparquet # noqa
+ defaults['fastparquet'] = True
+except ImportError:
+ pass
+
+try:
+ import pyarrow.gandiva # noqa
+ defaults['gandiva'] = True
+except ImportError:
+ pass
+
+try:
+ import pyarrow.dataset # noqa
+ defaults['dataset'] = True
+except ImportError:
+ pass
+
+try:
+ import pyarrow.orc # noqa
+ defaults['orc'] = True
+except ImportError:
+ pass
+
+try:
+ import pandas # noqa
+ defaults['pandas'] = True
+except ImportError:
+ defaults['nopandas'] = True
+
+try:
+ import pyarrow.parquet # noqa
+ defaults['parquet'] = True
+except ImportError:
+ pass
+
+try:
+ import pyarrow.parquet.encryption # noqa
+ defaults['parquet_encryption'] = True
+except ImportError:
+ pass
+
+
+try:
+ import pyarrow.plasma # noqa
+ defaults['plasma'] = True
+except ImportError:
+ pass
+
+try:
+ import tensorflow # noqa
+ defaults['tensorflow'] = True
+except ImportError:
+ pass
+
+try:
+ import pyarrow.flight # noqa
+ defaults['flight'] = True
+except ImportError:
+ pass
+
+try:
+ from pyarrow.fs import S3FileSystem # noqa
+ defaults['s3'] = True
+except ImportError:
+ pass
+
+try:
+ from pyarrow.fs import HadoopFileSystem # noqa
+ defaults['hdfs'] = True
+except ImportError:
+ pass
+
+try:
+ import pyarrow.substrait # noqa
+ defaults['substrait'] = True
+except ImportError:
+ pass
+
+
+# Doctest should ignore files for the modules that are not built
+def pytest_ignore_collect(path, config):
+ if config.option.doctestmodules:
+ # don't try to run doctests on the /tests directory
+ if "/pyarrow/tests/" in str(path):
+ return True
+
+ doctest_groups = [
+ 'dataset',
+ 'orc',
+ 'parquet',
+ 'plasma',
+ 'flight',
+ 'substrait',
+ ]
+
+ # handle cuda, flight, etc
+ for group in doctest_groups:
+ if 'pyarrow/{}'.format(group) in str(path):
+ if not defaults[group]:
+ return True
+
+ if 'pyarrow/parquet/encryption' in str(path):
+ if not defaults['parquet_encryption']:
+ return True
+
+ if 'pyarrow/cuda' in str(path):
+ try:
+ import pyarrow.cuda # noqa
+ return False
+ except ImportError:
+ return True
+
+ if 'pyarrow/fs' in str(path):
+ try:
+ from pyarrow.fs import S3FileSystem # noqa
+ return False
+ except ImportError:
+ return True
+
+ return False
+
+
+# Save output files from doctest examples into temp dir
[email protected](autouse=True)
+def _docdir(request):
+
+ # Trigger ONLY for the doctests.
+ if request.config.option.doctestmodules:
+
+ # Get the fixture dynamically by its name.
+ tmpdir = request.getfixturevalue('tmpdir')
+
+ # Chdir only for the duration of the test.
+ with tmpdir.as_cwd():
+ yield
+
+ else:
+ # For normal tests, we have to yield, since this is a yield-fixture.
+ yield
diff --git a/python/pyarrow/dataset.py b/python/pyarrow/dataset.py
index b22a3d032f..2933b12ea3 100644
--- a/python/pyarrow/dataset.py
+++ b/python/pyarrow/dataset.py
@@ -155,18 +155,21 @@ def partitioning(schema=None, field_names=None,
flavor=None,
Specify the Schema for paths like "/2009/June":
- >>> partitioning(pa.schema([("year", pa.int16()), ("month", pa.string())]))
+ >>> import pyarrow as pa
+ >>> import pyarrow.dataset as ds
+ >>> part = ds.partitioning(pa.schema([("year", pa.int16()),
+ ... ("month", pa.string())]))
or let the types be inferred by only specifying the field names:
- >>> partitioning(field_names=["year", "month"])
+ >>> part = ds.partitioning(field_names=["year", "month"])
For paths like "/2009/June", the year will be inferred as int32 while month
will be inferred as string.
Specify a Schema with dictionary encoding, providing dictionary values:
- >>> partitioning(
+ >>> part = ds.partitioning(
... pa.schema([
... ("year", pa.int16()),
... ("month", pa.dictionary(pa.int8(), pa.string()))
@@ -178,7 +181,7 @@ def partitioning(schema=None, field_names=None, flavor=None,
Alternatively, specify a Schema with dictionary encoding, but have Arrow
infer the dictionary values:
- >>> partitioning(
+ >>> part = ds.partitioning(
... pa.schema([
... ("year", pa.int16()),
... ("month", pa.dictionary(pa.int8(), pa.string()))
@@ -187,15 +190,14 @@ def partitioning(schema=None, field_names=None,
flavor=None,
Create a Hive scheme for a path like "/year=2009/month=11":
- >>> partitioning(
+ >>> part = ds.partitioning(
... pa.schema([("year", pa.int16()), ("month", pa.int8())]),
... flavor="hive")
A Hive scheme can also be discovered from the directory structure (and
types will be inferred):
- >>> partitioning(flavor="hive")
-
+ >>> part = ds.partitioning(flavor="hive")
"""
if flavor is None:
# default flavor
@@ -622,26 +624,76 @@ RecordBatch or Table, iterable of RecordBatch,
RecordBatchReader, or URI
Examples
--------
+ Creating an example Table:
+
+ >>> import pyarrow as pa
+ >>> import pyarrow.parquet as pq
+ >>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021],
+ ... 'n_legs': [2, 2, 4, 4, 5, 100],
+ ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
+ ... "Brittle stars", "Centipede"]})
+ >>> pq.write_table(table, "file.parquet")
+
Opening a single file:
- >>> dataset("path/to/file.parquet", format="parquet")
+ >>> import pyarrow.dataset as ds
+ >>> dataset = ds.dataset("file.parquet", format="parquet")
+ >>> dataset.to_table()
+ pyarrow.Table
+ year: int64
+ n_legs: int64
+ animal: string
+ ----
+ year: [[2020,2022,2021,2022,2019,2021]]
+ n_legs: [[2,2,4,4,5,100]]
+ animal: [["Flamingo","Parrot","Dog","Horse","Brittle stars","Centipede"]]
Opening a single file with an explicit schema:
- >>> dataset("path/to/file.parquet", schema=myschema, format="parquet")
+ >>> myschema = pa.schema([
+ ... ('n_legs', pa.int64()),
+ ... ('animal', pa.string())])
+ >>> dataset = ds.dataset("file.parquet", schema=myschema, format="parquet")
+ >>> dataset.to_table()
+ pyarrow.Table
+ n_legs: int64
+ animal: string
+ ----
+ n_legs: [[2,2,4,4,5,100]]
+ animal: [["Flamingo","Parrot","Dog","Horse","Brittle stars","Centipede"]]
Opening a dataset for a single directory:
- >>> dataset("path/to/nyc-taxi/", format="parquet")
- >>> dataset("s3://mybucket/nyc-taxi/", format="parquet")
+ >>> ds.write_dataset(table, "partitioned_dataset", format="parquet",
+ ... partitioning=['year'])
+ >>> dataset = ds.dataset("partitioned_dataset", format="parquet")
+ >>> dataset.to_table()
+ pyarrow.Table
+ n_legs: int64
+ animal: string
+ ----
+ n_legs: [[5],[2],[4,100],[2,4]]
+ animal: [["Brittle stars"],["Flamingo"],...["Parrot","Horse"]]
+
+ For a single directory from a S3 bucket:
+
+ >>> ds.dataset("s3://mybucket/nyc-taxi/",
+ ... format="parquet") # doctest: +SKIP
Opening a dataset from a list of relatives local paths:
- >>> dataset([
- ... "part0/data.parquet",
- ... "part1/data.parquet",
- ... "part3/data.parquet",
+ >>> dataset = ds.dataset([
+ ... "partitioned_dataset/2019/part-0.parquet",
+ ... "partitioned_dataset/2020/part-0.parquet",
+ ... "partitioned_dataset/2021/part-0.parquet",
... ], format='parquet')
+ >>> dataset.to_table()
+ pyarrow.Table
+ n_legs: int64
+ animal: string
+ ----
+ n_legs: [[5],[2],[4,100]]
+ animal: [["Brittle stars"],["Flamingo"],["Dog","Centipede"]]
With filesystem provided:
@@ -650,12 +702,14 @@ RecordBatch or Table, iterable of RecordBatch,
RecordBatchReader, or URI
... 'part1/data.parquet',
... 'part3/data.parquet',
... ]
- >>> dataset(paths, filesystem='file:///directory/prefix, format='parquet')
+ >>> ds.dataset(paths, filesystem='file:///directory/prefix,
+ ... format='parquet') # doctest: +SKIP
Which is equivalent with:
- >>> fs = SubTreeFileSystem("/directory/prefix", LocalFileSystem())
- >>> dataset(paths, filesystem=fs, format='parquet')
+ >>> fs = SubTreeFileSystem("/directory/prefix",
+ ... LocalFileSystem()) # doctest: +SKIP
+ >>> ds.dataset(paths, filesystem=fs, format='parquet') # doctest: +SKIP
With a remote filesystem URI:
@@ -664,20 +718,21 @@ RecordBatch or Table, iterable of RecordBatch,
RecordBatchReader, or URI
... 'nested/directory/part1/data.parquet',
... 'nested/directory/part3/data.parquet',
... ]
- >>> dataset(paths, filesystem='s3://bucket/', format='parquet')
+ >>> ds.dataset(paths, filesystem='s3://bucket/',
+ ... format='parquet') # doctest: +SKIP
Similarly to the local example, the directory prefix may be included in the
filesystem URI:
- >>> dataset(paths, filesystem='s3://bucket/nested/directory',
- ... format='parquet')
+ >>> ds.dataset(paths, filesystem='s3://bucket/nested/directory',
+ ... format='parquet') # doctest: +SKIP
Construction of a nested dataset:
- >>> dataset([
+ >>> ds.dataset([
... dataset("s3://old-taxi-data", format="parquet"),
... dataset("local/path/to/data", format="ipc")
- ... ])
+ ... ]) # doctest: +SKIP
"""
# collect the keyword arguments for later reuse
kwargs = dict(
diff --git a/python/pyarrow/pandas_compat.py b/python/pyarrow/pandas_compat.py
index db227b906f..f2f67c1cb0 100644
--- a/python/pyarrow/pandas_compat.py
+++ b/python/pyarrow/pandas_compat.py
@@ -291,11 +291,11 @@ def _column_name_to_strings(name):
'foo'
>>> name = ('foo', 'bar')
>>> _column_name_to_strings(name)
- ('foo', 'bar')
+ "('foo', 'bar')"
>>> import pandas as pd
>>> name = (1, pd.Timestamp('2017-02-01 00:00:00'))
>>> _column_name_to_strings(name)
- ('1', '2017-02-01 00:00:00')
+ "('1', '2017-02-01 00:00:00')"
"""
if isinstance(name, str):
return name
diff --git a/python/pyarrow/parquet/__init__.py
b/python/pyarrow/parquet/__init__.py
index a9437aec05..049f955f80 100644
--- a/python/pyarrow/parquet/__init__.py
+++ b/python/pyarrow/parquet/__init__.py
@@ -1859,7 +1859,7 @@ Examples
pyarrow.Table
n_legs: int64
----
- n_legs: [[5],[2],...,[2],[4]]
+ n_legs: [[5],[2],[4,100],[2,4]]
"""
tables = []
for piece in self._pieces:
@@ -1911,10 +1911,8 @@ Examples
... "Brittle stars", "Centipede"]})
>>> table = pa.Table.from_pandas(df)
>>> import pyarrow.parquet as pq
- >>> pq.write_to_dataset(table, root_path='dataset_name_read_pandas',
- ... partition_cols=['year'],
- ... use_legacy_dataset=False)
- >>> dataset = pq.ParquetDataset('dataset_name_read_pandas/',
+ >>> pq.write_table(table, 'table.parquet')
+ >>> dataset = pq.ParquetDataset('table.parquet',
... use_legacy_dataset=False)
Read dataset including pandas metadata:
@@ -1923,7 +1921,7 @@ Examples
pyarrow.Table
n_legs: int64
----
- n_legs: [[5],[2],...,[2],[4]]
+ n_legs: [[2,2,4,4,5,100]]
Select pandas metadata:
@@ -2104,7 +2102,7 @@ Examples
>>> pq.write_to_dataset(table, root_path='dataset_name_fragments',
... partition_cols=['year'],
... use_legacy_dataset=False)
- >>> dataset = pq.ParquetDataset('dataset_name_files/',
+ >>> dataset = pq.ParquetDataset('dataset_name_fragments/',
... use_legacy_dataset=False)
List the fragments:
@@ -2142,7 +2140,7 @@ Examples
List the files:
>>> dataset.files
- ['dataset_name_files/year=2019/part-0.parquet', ...
+ ['dataset_name_files/year=2019/...-0.parquet', ...
"""
raise NotImplementedError(
"To use this property set 'use_legacy_dataset=False' while "
@@ -2234,9 +2232,9 @@ class _ParquetDatasetV2:
... partition_cols=['year'],
... use_legacy_dataset=False)
- create a _ParquetDatasetV2 object from the dataset source:
+ create a ParquetDataset object from the dataset source:
- >>> dataset = pq._ParquetDatasetV2('dataset_v2/')
+ >>> dataset = pq.ParquetDataset('dataset_v2/', use_legacy_dataset=False)
and read the data:
@@ -2249,10 +2247,11 @@ class _ParquetDatasetV2:
4 2 Parrot 2022
5 4 Horse 2022
- create a _ParquetDatasetV2 object with filter:
+ create a ParquetDataset object with filter:
- >>> dataset = pq._ParquetDatasetV2('dataset_v2/',
- ... filters=[('n_legs','=',4)])
+ >>> dataset = pq.ParquetDataset('dataset_v2/',
+ ... filters=[('n_legs','=',4)],
+ ... use_legacy_dataset=False)
>>> dataset.read().to_pandas()
n_legs animal year
0 4 Dog 2021
@@ -2371,7 +2370,8 @@ class _ParquetDatasetV2:
>>> pq.write_to_dataset(table, root_path='dataset_v2_schema',
... partition_cols=['year'],
... use_legacy_dataset=False)
- >>> dataset = pq._ParquetDatasetV2('dataset_v2_schema/')
+ >>> dataset = pq.ParquetDataset('dataset_v2_schema/',
+ ... use_legacy_dataset=False)
Read the schema:
@@ -2416,7 +2416,8 @@ class _ParquetDatasetV2:
>>> pq.write_to_dataset(table, root_path='dataset_v2_read',
... partition_cols=['year'],
... use_legacy_dataset=False)
- >>> dataset = pq._ParquetDatasetV2('dataset_v2_read/')
+ >>> dataset = pq.ParquetDataset('dataset_v2_read/',
+ ... use_legacy_dataset=False)
Read the dataset:
@@ -2424,7 +2425,7 @@ class _ParquetDatasetV2:
pyarrow.Table
n_legs: int64
----
- n_legs: [[5],[2],...,[2],[4]]
+ n_legs: [[5],[2],[4,100],[2,4]]
"""
# if use_pandas_metadata, we need to include index columns in the
# column selection, to be able to restore those in the pandas DataFrame
@@ -2462,18 +2463,19 @@ class _ParquetDatasetV2:
Examples
--------
- Generate an example dataset:
+ Generate an example parquet file:
>>> import pyarrow as pa
- >>> table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021],
- ... 'n_legs': [2, 2, 4, 4, 5, 100],
- ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
- ... "Brittle stars", "Centipede"]})
+ >>> import pandas as pd
+ >>> df = pd.DataFrame({'year': [2020, 2022, 2021, 2022, 2019, 2021],
+ ... 'n_legs': [2, 2, 4, 4, 5, 100],
+ ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
+ ... "Brittle stars", "Centipede"]})
+ >>> table = pa.Table.from_pandas(df)
>>> import pyarrow.parquet as pq
- >>> pq.write_to_dataset(table, root_path='dataset_v2_read_pandas',
- ... partition_cols=['year'],
- ... use_legacy_dataset=False)
- >>> dataset = pq._ParquetDatasetV2('dataset_v2_read_pandas/')
+ >>> pq.write_table(table, 'table_V2.parquet')
+ >>> dataset = pq.ParquetDataset('table_V2.parquet',
+ ... use_legacy_dataset=False)
Read the dataset with pandas metadata:
@@ -2481,7 +2483,7 @@ class _ParquetDatasetV2:
pyarrow.Table
n_legs: int64
----
- n_legs: [[5],[2],...,[2],[4]]
+ n_legs: [[2,2,4,4,5,100]]
>>> dataset.read_pandas(columns=["n_legs"]).schema.pandas_metadata
{'index_columns': [{'kind': 'range', ... 'pandas_version': '1.4.1'}
@@ -2515,7 +2517,8 @@ class _ParquetDatasetV2:
>>> pq.write_to_dataset(table, root_path='dataset_v2_fragments',
... partition_cols=['year'],
... use_legacy_dataset=False)
- >>> dataset = pq._ParquetDatasetV2('dataset_v2_fragments/')
+ >>> dataset = pq.ParquetDataset('dataset_v2_fragments/',
+ ... use_legacy_dataset=False)
List the fragments:
@@ -2542,12 +2545,13 @@ class _ParquetDatasetV2:
>>> pq.write_to_dataset(table, root_path='dataset_v2_files',
... partition_cols=['year'],
... use_legacy_dataset=False)
- >>> dataset = pq._ParquetDatasetV2('dataset_v2_files/')
+ >>> dataset = pq.ParquetDataset('dataset_v2_files/',
+ ... use_legacy_dataset=False)
List the files:
>>> dataset.files
- ['dataset_v2_files/year=2019/part-0.parquet', ...
+ ['dataset_v2_files/year=2019/...-0.parquet', ...
"""
return self._dataset.files
@@ -2674,8 +2678,8 @@ pyarrow.Table
n_legs: int64
animal: string
----
-n_legs: [[5],[2],...,[2],[4]]
-animal: [["Brittle stars"],["Flamingo"],...,["Parrot"],["Horse"]]
+n_legs: [[5],[2],[4,100],[2,4]]
+animal: [["Brittle stars"],["Flamingo"],["Dog","Centipede"],["Parrot","Horse"]]
Read a subset of columns and read one column as DictionaryArray:
@@ -2685,16 +2689,16 @@ pyarrow.Table
n_legs: int64
animal: dictionary<values=string, indices=int32, ordered=0>
----
-n_legs: [[5],[2],...,[2],[4]]
+n_legs: [[5],[2],[4,100],[2,4]]
animal: [ -- dictionary:
["Brittle stars"] -- indices:
[0], -- dictionary:
["Flamingo"] -- indices:
-[0],..., -- dictionary:
-["Parrot"] -- indices:
[0], -- dictionary:
-["Horse"] -- indices:
-[0]]
+["Dog","Centipede"] -- indices:
+[0,1], -- dictionary:
+["Parrot","Horse"] -- indices:
+[0,1]]
Read the table with filter:
@@ -2711,8 +2715,8 @@ Read data from a single Parquet file:
n_legs animal year
0 5 Brittle stars 2019
1 2 Flamingo 2020
-2 100 Centipede 2021
-3 4 Dog 2021
+2 4 Dog 2021
+3 100 Centipede 2021
4 2 Parrot 2022
5 4 Horse 2022
"""
@@ -3089,13 +3093,13 @@ def write_to_dataset(table, root_path,
partition_cols=None,
>>> pq.write_to_dataset(table, root_path='dataset_name_3',
... partition_cols=['year'])
>>> pq.ParquetDataset('dataset_name_3', use_legacy_dataset=False).files
- ['dataset_name_3/year=2019/part-0.parquet', ...
+ ['dataset_name_3/year=2019/...-0.parquet', ...
Write a single Parquet file into the root folder:
>>> pq.write_to_dataset(table, root_path='dataset_name_4')
>>> pq.ParquetDataset('dataset_name_4/', use_legacy_dataset=False).files
- ['dataset_name_4/part-0.parquet']
+ ['dataset_name_4/...-0.parquet']
"""
if use_legacy_dataset is None:
# if partition_filename_cb is specified ->
diff --git a/python/pyarrow/tests/conftest.py b/python/pyarrow/tests/conftest.py
index a5aae6f634..0b7f1618b0 100644
--- a/python/pyarrow/tests/conftest.py
+++ b/python/pyarrow/tests/conftest.py
@@ -22,9 +22,9 @@ from tempfile import TemporaryDirectory
import pytest
import hypothesis as h
+from ..conftest import groups, defaults
from pyarrow.util import find_free_port
-from pyarrow import Codec
# setup hypothesis profiles
@@ -44,152 +44,6 @@
h.settings.load_profile(os.environ.get('HYPOTHESIS_PROFILE', 'dev'))
os.environ['AWS_CONFIG_FILE'] = "/dev/null"
-groups = [
- 'brotli',
- 'bz2',
- 'cython',
- 'dataset',
- 'hypothesis',
- 'fastparquet',
- 'gandiva',
- 'gdb',
- 'gzip',
- 'hdfs',
- 'large_memory',
- 'lz4',
- 'memory_leak',
- 'nopandas',
- 'orc',
- 'pandas',
- 'parquet',
- 'parquet_encryption',
- 'plasma',
- 's3',
- 'snappy',
- 'substrait',
- 'tensorflow',
- 'flight',
- 'slow',
- 'requires_testing_data',
- 'zstd',
-]
-
-defaults = {
- 'brotli': Codec.is_available('brotli'),
- 'bz2': Codec.is_available('bz2'),
- 'cython': False,
- 'dataset': False,
- 'fastparquet': False,
- 'flight': False,
- 'gandiva': False,
- 'gdb': True,
- 'gzip': Codec.is_available('gzip'),
- 'hdfs': False,
- 'hypothesis': False,
- 'large_memory': False,
- 'lz4': Codec.is_available('lz4'),
- 'memory_leak': False,
- 'nopandas': False,
- 'orc': False,
- 'pandas': False,
- 'parquet': False,
- 'parquet_encryption': False,
- 'plasma': False,
- 'requires_testing_data': True,
- 's3': False,
- 'slow': False,
- 'snappy': Codec.is_available('snappy'),
- 'substrait': False,
- 'tensorflow': False,
- 'zstd': Codec.is_available('zstd'),
-}
-
-try:
- import cython # noqa
- defaults['cython'] = True
-except ImportError:
- pass
-
-try:
- import fastparquet # noqa
- defaults['fastparquet'] = True
-except ImportError:
- pass
-
-try:
- import pyarrow.gandiva # noqa
- defaults['gandiva'] = True
-except ImportError:
- pass
-
-try:
- import pyarrow.dataset # noqa
- defaults['dataset'] = True
-except ImportError:
- pass
-
-try:
- import pyarrow.orc # noqa
- defaults['orc'] = True
-except ImportError:
- pass
-
-try:
- import pandas # noqa
- defaults['pandas'] = True
-except ImportError:
- defaults['nopandas'] = True
-
-try:
- import pyarrow.parquet # noqa
- defaults['parquet'] = True
-except ImportError:
- pass
-
-try:
- import pyarrow.parquet.encryption # noqa
- defaults['parquet_encryption'] = True
-except ImportError:
- pass
-
-
-try:
- import pyarrow.plasma # noqa
- defaults['plasma'] = True
-except ImportError:
- pass
-
-try:
- import tensorflow # noqa
- defaults['tensorflow'] = True
-except ImportError:
- pass
-
-try:
- import pyarrow.flight # noqa
- defaults['flight'] = True
-except ImportError:
- pass
-
-try:
- from pyarrow.fs import S3FileSystem # noqa
- defaults['s3'] = True
-except ImportError:
- pass
-
-try:
- from pyarrow.fs import HadoopFileSystem # noqa
- defaults['hdfs'] = True
-except ImportError:
- pass
-
-try:
- import pyarrow.substrait # noqa
- defaults['substrait'] = True
-except ImportError:
- pass
-
-
def pytest_addoption(parser):
# Create options to selectively enable test groups
def bool_env(name, default=None):
diff --git a/python/pyarrow/tests/test_compute.py
b/python/pyarrow/tests/test_compute.py
index ec90af1e37..45282a2867 100644
--- a/python/pyarrow/tests/test_compute.py
+++ b/python/pyarrow/tests/test_compute.py
@@ -780,13 +780,13 @@ def test_generated_docstrings():
>>> 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>
+ <pyarrow.lib.StringArray object at ...>
[
"a",
"e"
]
>>> arr.filter(mask, null_selection_behavior='emit_null')
- <pyarrow.lib.StringArray object at 0x7fa826df9200>
+ <pyarrow.lib.StringArray object at ...>
[
"a",
null,