spark-instrumented-optimizer/python/pyspark/pandas/config.py
itholic caf04f9b77 [SPARK-34890][PYTHON] Port/integrate Koalas main codes into PySpark
### What changes were proposed in this pull request?

As a first step of [SPARK-34849](https://issues.apache.org/jira/browse/SPARK-34849), this PR proposes porting the Koalas main code into PySpark.

This PR contains minimal changes to the existing Koalas code as follows:
1. `databricks.koalas` -> `pyspark.pandas`
2. `from databricks import koalas as ks` -> `from pyspark import pandas as pp`
3. `ks.xxx -> pp.xxx`

Other than them:
1. Added a line to `python/mypy.ini` in order to ignore the mypy test. See related issue at [SPARK-34941](https://issues.apache.org/jira/browse/SPARK-34941).
2. Added a comment to several lines in several files to ignore the flake8 F401. See related issue at [SPARK-34943](https://issues.apache.org/jira/browse/SPARK-34943).

When this PR is merged, all the features that were previously used in [Koalas](https://github.com/databricks/koalas) will be available in PySpark as well.

Users can access to the pandas API in PySpark as below:

```python
>>> from pyspark import pandas as pp
>>> ppdf = pp.DataFrame({"A": [1, 2, 3], "B": [15, 20, 25]})
>>> ppdf
   A   B
0  1  15
1  2  20
2  3  25
```

The existing "options and settings" in Koalas are also available in the same way:

```python
>>> from pyspark.pandas.config import set_option, reset_option, get_option
>>> ppser1 = pp.Series([1, 2, 3])
>>> ppser2 = pp.Series([3, 4, 5])
>>> ppser1 + ppser2
Traceback (most recent call last):
...
ValueError: Cannot combine the series or dataframe because it comes from a different dataframe. In order to allow this operation, enable 'compute.ops_on_diff_frames' option.

>>> set_option("compute.ops_on_diff_frames", True)
>>> ppser1 + ppser2
0    4
1    6
2    8
dtype: int64
```

Please also refer to the [API Reference](https://koalas.readthedocs.io/en/latest/reference/index.html) and [Options and Settings](https://koalas.readthedocs.io/en/latest/user_guide/options.html) for more detail.

**NOTE** that this PR intentionally ports the main codes of Koalas first almost as are with minimal changes because:
- Koalas project is fairly large. Making some changes together for PySpark will make it difficult to review the individual change.
    Koalas dev includes multiple Spark committers who will review. By doing this, the committers will be able to more easily and effectively review and drive the development.
- Koalas tests and documentation require major changes to make it look great together with PySpark whereas main codes do not require.
- We lately froze the Koalas codebase, and plan to work together on the initial porting. By porting the main codes first as are, it unblocks the Koalas dev to work on other items in parallel.

I promise and will make sure on:
- Rename Koalas to PySpark pandas APIs and/or pandas-on-Spark accordingly in documentation, and the docstrings and comments in the main codes.
- Triage APIs to remove that don’t make sense when Koalas is in PySpark

The documentation changes will be tracked in [SPARK-34885](https://issues.apache.org/jira/browse/SPARK-34885), the test code changes will be tracked in [SPARK-34886](https://issues.apache.org/jira/browse/SPARK-34886).

### Why are the changes needed?

Please refer to:
- [[DISCUSS] Support pandas API layer on PySpark](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Support-pandas-API-layer-on-PySpark-td30945.html)
- [[VOTE] SPIP: Support pandas API layer on PySpark](http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-SPIP-Support-pandas-API-layer-on-PySpark-td30996.html)

### Does this PR introduce _any_ user-facing change?

Yes, now users can use the pandas APIs on Spark

### How was this patch tested?

Manually tested for exposed major APIs and options as described above.

### Koalas contributors

Koalas would not have been possible without the following contributors:

ueshin
HyukjinKwon
rxin
xinrong-databricks
RainFung
charlesdong1991
harupy
floscha
beobest2
thunterdb
garawalid
LucasG0
shril
deepyaman
gioa
fwani
90jam
thoo
AbdealiJK
abishekganesh72
gliptak
DumbMachine
dvgodoy
stbof
nitlev
hjoo
gatorsmile
tomspur
icexelloss
awdavidson
guyao
akhilputhiry
scook12
patryk-oleniuk
tracek
dennyglee
athena15
gstaubli
WeichenXu123
hsubbaraj
lfdversluis
ktksq
shengjh
margaret-databricks
LSturtew
sllynn
manuzhang
jijosg
sadikovi

Closes #32036 from itholic/SPARK-34890.

Authored-by: itholic <haejoon.lee@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-06 12:42:39 +09:00

443 lines
14 KiB
Python

#
# 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.
#
"""
Infrastructure of options for Koalas.
"""
from contextlib import contextmanager
import json
from typing import Union, Any, Tuple, Callable, List, Dict # noqa: F401 (SPARK-34943)
from pyspark._globals import _NoValue, _NoValueType
from pyspark.pandas.utils import default_session
__all__ = ["get_option", "set_option", "reset_option", "options", "option_context"]
class Option:
"""
Option class that defines an option with related properties.
This class holds all information relevant to the one option. Also,
Its instance can validate if the given value is acceptable or not.
It is currently for internal usage only.
Parameters
----------
key: str, keyword-only argument
the option name to use.
doc: str, keyword-only argument
the documentation for the current option.
default: Any, keyword-only argument
default value for this option.
types: Union[Tuple[type, ...], type], keyword-only argument
default is str. It defines the expected types for this option. It is
used with `isinstance` to validate the given value to this option.
check_func: Tuple[Callable[[Any], bool], str], keyword-only argument
default is a function that always returns `True` with a empty string.
It defines:
- a function to check the given value to this option
- the error message to show when this check is failed
When new value is set to this option, this function is called to check
if the given value is valid.
Examples
--------
>>> option = Option(
... key='option.name',
... doc="this is a test option",
... default="default",
... types=(float, int),
... check_func=(lambda v: v > 0, "should be a positive float"))
>>> option.validate('abc') # doctest: +NORMALIZE_WHITESPACE
Traceback (most recent call last):
...
ValueError: The value for option 'option.name' was <class 'str'>;
however, expected types are [(<class 'float'>, <class 'int'>)].
>>> option.validate(-1.1)
Traceback (most recent call last):
...
ValueError: should be a positive float
>>> option.validate(1.1)
"""
def __init__(
self,
*,
key: str,
doc: str,
default: Any,
types: Union[Tuple[type, ...], type] = str,
check_func: Tuple[Callable[[Any], bool], str] = (lambda v: True, "")
):
self.key = key
self.doc = doc
self.default = default
self.types = types
self.check_func = check_func
def validate(self, v: Any) -> None:
"""
Validate the given value and throw an exception with related information such as key.
"""
if not isinstance(v, self.types):
raise ValueError(
"The value for option '%s' was %s; however, expected types are "
"[%s]." % (self.key, type(v), str(self.types))
)
if not self.check_func[0](v):
raise ValueError(self.check_func[1])
# Available options.
#
# NOTE: if you are fixing or adding an option here, make sure you execute `show_options()` and
# copy & paste the results into show_options 'docs/source/user_guide/options.rst' as well.
# See the examples below:
# >>> from pyspark.pandas.config import show_options
# >>> show_options()
_options = [
Option(
key="display.max_rows",
doc=(
"This sets the maximum number of rows Koalas should output when printing out "
"various output. For example, this value determines the number of rows to be "
"shown at the repr() in a dataframe. Set `None` to unlimit the input length. "
"Default is 1000."
),
default=1000,
types=(int, type(None)),
check_func=(
lambda v: v is None or v >= 0,
"'display.max_rows' should be greater than or equal to 0.",
),
),
Option(
key="compute.max_rows",
doc=(
"'compute.max_rows' sets the limit of the current Koalas DataFrame. Set `None` to "
"unlimit the input length. When the limit is set, it is executed by the shortcut by "
"collecting the data into the driver, and then using the pandas API. If the limit is "
"unset, the operation is executed by PySpark. Default is 1000."
),
default=1000,
types=(int, type(None)),
check_func=(
lambda v: v is None or v >= 0,
"'compute.max_rows' should be greater than or equal to 0.",
),
),
Option(
key="compute.shortcut_limit",
doc=(
"'compute.shortcut_limit' sets the limit for a shortcut. "
"It computes specified number of rows and use its schema. When the dataframe "
"length is larger than this limit, Koalas uses PySpark to compute."
),
default=1000,
types=int,
check_func=(
lambda v: v >= 0,
"'compute.shortcut_limit' should be greater than or equal to 0.",
),
),
Option(
key="compute.ops_on_diff_frames",
doc=(
"This determines whether or not to operate between two different dataframes. "
"For example, 'combine_frames' function internally performs a join operation which "
"can be expensive in general. So, if `compute.ops_on_diff_frames` variable is not "
"True, that method throws an exception."
),
default=False,
types=bool,
),
Option(
key="compute.default_index_type",
doc=("This sets the default index type: sequence, distributed and distributed-sequence."),
default="sequence",
types=str,
check_func=(
lambda v: v in ("sequence", "distributed", "distributed-sequence"),
"Index type should be one of 'sequence', 'distributed', 'distributed-sequence'.",
),
),
Option(
key="compute.ordered_head",
doc=(
"'compute.ordered_head' sets whether or not to operate head with natural ordering. "
"Koalas does not guarantee the row ordering so `head` could return some rows from "
"distributed partitions. If 'compute.ordered_head' is set to True, Koalas performs "
"natural ordering beforehand, but it will cause a performance overhead."
),
default=False,
types=bool,
),
Option(
key="plotting.max_rows",
doc=(
"'plotting.max_rows' sets the visual limit on top-n-based plots such as `plot.bar` "
"and `plot.pie`. If it is set to 1000, the first 1000 data points will be used "
"for plotting. Default is 1000."
),
default=1000,
types=int,
check_func=(
lambda v: v is v >= 0,
"'plotting.max_rows' should be greater than or equal to 0.",
),
),
Option(
key="plotting.sample_ratio",
doc=(
"'plotting.sample_ratio' sets the proportion of data that will be plotted for sample-"
"based plots such as `plot.line` and `plot.area`. "
"This option defaults to 'plotting.max_rows' option."
),
default=None,
types=(float, type(None)),
check_func=(
lambda v: v is None or 1 >= v >= 0,
"'plotting.sample_ratio' should be 1.0 >= value >= 0.0.",
),
),
Option(
key="plotting.backend",
doc=(
"Backend to use for plotting. Default is plotly. "
"Supports any package that has a top-level `.plot` method. "
"Known options are: [matplotlib, plotly]."
),
default="plotly",
types=str,
),
] # type: List[Option]
_options_dict = dict(zip((option.key for option in _options), _options)) # type: Dict[str, Option]
_key_format = "koalas.{}".format
class OptionError(AttributeError, KeyError):
pass
def show_options():
"""
Make a pretty table that can be copied and pasted into public documentation.
This is currently for an internal purpose.
Examples
--------
>>> show_options() # doctest: +ELLIPSIS, +NORMALIZE_WHITESPACE
================... =======... =====================...
Option Default Description
================... =======... =====================...
display.max_rows 1000 This sets the maximum...
...
================... =======... =====================...
"""
import textwrap
header = ["Option", "Default", "Description"]
row_format = "{:<31} {:<14} {:<53}"
print(row_format.format("=" * 31, "=" * 14, "=" * 53))
print(row_format.format(*header))
print(row_format.format("=" * 31, "=" * 14, "=" * 53))
for option in _options:
doc = textwrap.fill(option.doc, 53)
formatted = "".join([line + "\n" + (" " * 47) for line in doc.split("\n")]).rstrip()
print(row_format.format(option.key, repr(option.default), formatted))
print(row_format.format("=" * 31, "=" * 14, "=" * 53))
def get_option(key: str, default: Union[Any, _NoValueType] = _NoValue) -> Any:
"""
Retrieves the value of the specified option.
Parameters
----------
key : str
The key which should match a single option.
default : object
The default value if the option is not set yet. The value should be JSON serializable.
Returns
-------
result : the value of the option
Raises
------
OptionError : if no such option exists and the default is not provided
"""
_check_option(key)
if default is _NoValue:
default = _options_dict[key].default
_options_dict[key].validate(default)
return json.loads(default_session().conf.get(_key_format(key), default=json.dumps(default)))
def set_option(key: str, value: Any) -> None:
"""
Sets the value of the specified option.
Parameters
----------
key : str
The key which should match a single option.
value : object
New value of option. The value should be JSON serializable.
Returns
-------
None
"""
_check_option(key)
_options_dict[key].validate(value)
default_session().conf.set(_key_format(key), json.dumps(value))
def reset_option(key: str) -> None:
"""
Reset one option to their default value.
Pass "all" as argument to reset all options.
Parameters
----------
key : str
If specified only option will be reset.
Returns
-------
None
"""
_check_option(key)
default_session().conf.unset(_key_format(key))
@contextmanager
def option_context(*args):
"""
Context manager to temporarily set options in the `with` statement context.
You need to invoke as ``option_context(pat, val, [(pat, val), ...])``.
Examples
--------
>>> with option_context('display.max_rows', 10, 'compute.max_rows', 5):
... print(get_option('display.max_rows'), get_option('compute.max_rows'))
10 5
>>> print(get_option('display.max_rows'), get_option('compute.max_rows'))
1000 1000
"""
if len(args) == 0 or len(args) % 2 != 0:
raise ValueError("Need to invoke as option_context(pat, val, [(pat, val), ...]).")
opts = dict(zip(args[::2], args[1::2]))
orig_opts = {key: get_option(key) for key in opts}
try:
for key, value in opts.items():
set_option(key, value)
yield
finally:
for key, value in orig_opts.items():
set_option(key, value)
def _check_option(key: str) -> None:
if key not in _options_dict:
raise OptionError(
"No such option: '{}'. Available options are [{}]".format(
key, ", ".join(list(_options_dict.keys()))
)
)
class DictWrapper:
""" provide attribute-style access to a nested dict"""
def __init__(self, d, prefix=""):
object.__setattr__(self, "d", d)
object.__setattr__(self, "prefix", prefix)
def __setattr__(self, key, val):
prefix = object.__getattribute__(self, "prefix")
d = object.__getattribute__(self, "d")
if prefix:
prefix += "."
canonical_key = prefix + key
candidates = [
k for k in d.keys() if all(x in k.split(".") for x in canonical_key.split("."))
]
if len(candidates) == 1 and candidates[0] == canonical_key:
return set_option(canonical_key, val)
else:
raise OptionError(
"No such option: '{}'. Available options are [{}]".format(
key, ", ".join(list(_options_dict.keys()))
)
)
def __getattr__(self, key):
prefix = object.__getattribute__(self, "prefix")
d = object.__getattribute__(self, "d")
if prefix:
prefix += "."
canonical_key = prefix + key
candidates = [
k for k in d.keys() if all(x in k.split(".") for x in canonical_key.split("."))
]
if len(candidates) == 1 and candidates[0] == canonical_key:
return get_option(canonical_key)
elif len(candidates) == 0:
raise OptionError(
"No such option: '{}'. Available options are [{}]".format(
key, ", ".join(list(_options_dict.keys()))
)
)
else:
return DictWrapper(d, canonical_key)
def __dir__(self):
prefix = object.__getattribute__(self, "prefix")
d = object.__getattribute__(self, "d")
if prefix == "":
candidates = d.keys()
offset = 0
else:
candidates = [k for k in d.keys() if all(x in k.split(".") for x in prefix.split("."))]
offset = len(prefix) + 1 # prefix (e.g. "compute.") to trim.
return [c[offset:] for c in candidates]
options = DictWrapper(_options_dict)