From caf04f9b77b3b215963936231fb11027dee57d6c Mon Sep 17 00:00:00 2001 From: itholic Date: Tue, 6 Apr 2021 12:42:39 +0900 Subject: [PATCH] [SPARK-34890][PYTHON] Port/integrate Koalas main codes into PySpark MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### 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 Signed-off-by: HyukjinKwon --- dev/lint-python | 1 + python/mypy.ini | 4 + python/pyspark/pandas/__init__.py | 209 + python/pyspark/pandas/accessors.py | 930 ++ python/pyspark/pandas/base.py | 1995 +++ python/pyspark/pandas/categorical.py | 164 + python/pyspark/pandas/config.py | 442 + python/pyspark/pandas/datetimes.py | 850 ++ python/pyspark/pandas/exceptions.py | 106 + python/pyspark/pandas/extensions.py | 342 + python/pyspark/pandas/frame.py | 11976 ++++++++++++++++ python/pyspark/pandas/generic.py | 3102 ++++ python/pyspark/pandas/groupby.py | 3186 ++++ python/pyspark/pandas/indexes/__init__.py | 20 + python/pyspark/pandas/indexes/base.py | 2471 ++++ python/pyspark/pandas/indexes/category.py | 188 + python/pyspark/pandas/indexes/datetimes.py | 742 + python/pyspark/pandas/indexes/multi.py | 1170 ++ python/pyspark/pandas/indexes/numeric.py | 147 + python/pyspark/pandas/indexing.py | 1708 +++ python/pyspark/pandas/internal.py | 1438 ++ python/pyspark/pandas/missing/__init__.py | 48 + python/pyspark/pandas/missing/common.py | 59 + python/pyspark/pandas/missing/frame.py | 98 + python/pyspark/pandas/missing/groupby.py | 103 + python/pyspark/pandas/missing/indexes.py | 218 + python/pyspark/pandas/missing/series.py | 125 + python/pyspark/pandas/missing/window.py | 126 + python/pyspark/pandas/ml.py | 91 + python/pyspark/pandas/mlflow.py | 192 + python/pyspark/pandas/namespace.py | 2877 ++++ python/pyspark/pandas/numpy_compat.py | 210 + python/pyspark/pandas/plot/__init__.py | 17 + python/pyspark/pandas/plot/core.py | 1134 ++ python/pyspark/pandas/plot/matplotlib.py | 897 ++ python/pyspark/pandas/plot/plotly.py | 212 + python/pyspark/pandas/series.py | 6235 ++++++++ python/pyspark/pandas/spark/__init__.py | 16 + python/pyspark/pandas/spark/accessors.py | 1249 ++ python/pyspark/pandas/spark/functions.py | 98 + python/pyspark/pandas/spark/utils.py | 124 + python/pyspark/pandas/sql.py | 302 + python/pyspark/pandas/strings.py | 2289 +++ python/pyspark/pandas/typedef/__init__.py | 18 + .../pandas/typedef/string_typehints.py | 37 + python/pyspark/pandas/typedef/typehints.py | 521 + .../pyspark/pandas/usage_logging/__init__.py | 269 + .../pandas/usage_logging/usage_logger.py | 132 + python/pyspark/pandas/utils.py | 878 ++ python/pyspark/pandas/version.py | 18 + python/pyspark/pandas/window.py | 1739 +++ 51 files changed, 51523 insertions(+) create mode 100644 python/pyspark/pandas/__init__.py create mode 100644 python/pyspark/pandas/accessors.py create mode 100644 python/pyspark/pandas/base.py create mode 100644 python/pyspark/pandas/categorical.py create mode 100644 python/pyspark/pandas/config.py create mode 100644 python/pyspark/pandas/datetimes.py create mode 100644 python/pyspark/pandas/exceptions.py create mode 100644 python/pyspark/pandas/extensions.py create mode 100644 python/pyspark/pandas/frame.py create mode 100644 python/pyspark/pandas/generic.py create mode 100644 python/pyspark/pandas/groupby.py create mode 100644 python/pyspark/pandas/indexes/__init__.py create mode 100644 python/pyspark/pandas/indexes/base.py create mode 100644 python/pyspark/pandas/indexes/category.py create mode 100644 python/pyspark/pandas/indexes/datetimes.py create mode 100644 python/pyspark/pandas/indexes/multi.py create mode 100644 python/pyspark/pandas/indexes/numeric.py create mode 100644 python/pyspark/pandas/indexing.py create mode 100644 python/pyspark/pandas/internal.py create mode 100644 python/pyspark/pandas/missing/__init__.py create mode 100644 python/pyspark/pandas/missing/common.py create mode 100644 python/pyspark/pandas/missing/frame.py create mode 100644 python/pyspark/pandas/missing/groupby.py create mode 100644 python/pyspark/pandas/missing/indexes.py create mode 100644 python/pyspark/pandas/missing/series.py create mode 100644 python/pyspark/pandas/missing/window.py create mode 100644 python/pyspark/pandas/ml.py create mode 100644 python/pyspark/pandas/mlflow.py create mode 100644 python/pyspark/pandas/namespace.py create mode 100644 python/pyspark/pandas/numpy_compat.py create mode 100644 python/pyspark/pandas/plot/__init__.py create mode 100644 python/pyspark/pandas/plot/core.py create mode 100644 python/pyspark/pandas/plot/matplotlib.py create mode 100644 python/pyspark/pandas/plot/plotly.py create mode 100644 python/pyspark/pandas/series.py create mode 100644 python/pyspark/pandas/spark/__init__.py create mode 100644 python/pyspark/pandas/spark/accessors.py create mode 100644 python/pyspark/pandas/spark/functions.py create mode 100644 python/pyspark/pandas/spark/utils.py create mode 100644 python/pyspark/pandas/sql.py create mode 100644 python/pyspark/pandas/strings.py create mode 100644 python/pyspark/pandas/typedef/__init__.py create mode 100644 python/pyspark/pandas/typedef/string_typehints.py create mode 100644 python/pyspark/pandas/typedef/typehints.py create mode 100644 python/pyspark/pandas/usage_logging/__init__.py create mode 100644 python/pyspark/pandas/usage_logging/usage_logger.py create mode 100644 python/pyspark/pandas/utils.py create mode 100644 python/pyspark/pandas/version.py create mode 100644 python/pyspark/pandas/window.py diff --git a/dev/lint-python b/dev/lint-python index 24923c1714..320e24fad3 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -17,6 +17,7 @@ # # define test binaries + versions FLAKE8_BUILD="flake8" +# TODO(SPARK-34943): minimum version should be 3.8+ MINIMUM_FLAKE8="3.5.0" MYPY_BUILD="mypy" PYCODESTYLE_BUILD="pycodestyle" diff --git a/python/mypy.ini b/python/mypy.ini index ad4fcf7f31..5bb4bb8981 100644 --- a/python/mypy.ini +++ b/python/mypy.ini @@ -126,3 +126,7 @@ ignore_missing_imports = True [mypy-psutil.*] ignore_missing_imports = True + +# TODO(SPARK-34941): Enable mypy for pandas-on-Spark +[mypy-pyspark.pandas.*] +ignore_errors = True diff --git a/python/pyspark/pandas/__init__.py b/python/pyspark/pandas/__init__.py new file mode 100644 index 0000000000..a605954624 --- /dev/null +++ b/python/pyspark/pandas/__init__.py @@ -0,0 +1,209 @@ +# +# 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 os +import sys +from distutils.version import LooseVersion + +from pyspark.pandas.version import __version__ # noqa: F401 + + +def assert_python_version(): + import warnings + + major = 3 + minor = 5 + deprecated_version = (major, minor) + min_supported_version = (major, minor + 1) + + if sys.version_info[:2] <= deprecated_version: + warnings.warn( + "Koalas support for Python {dep_ver} is deprecated and will be dropped in " + "the future release. At that point, existing Python {dep_ver} workflows " + "that use Koalas will continue to work without modification, but Python {dep_ver} " + "users will no longer get access to the latest Koalas features and bugfixes. " + "We recommend that you upgrade to Python {min_ver} or newer.".format( + dep_ver=".".join(map(str, deprecated_version)), + min_ver=".".join(map(str, min_supported_version)), + ), + FutureWarning, + ) + + +def assert_pyspark_version(): + import logging + + try: + import pyspark + except ImportError: + raise ImportError( + "Unable to import pyspark - consider doing a pip install with [spark] " + "extra to install pyspark with pip" + ) + else: + pyspark_ver = getattr(pyspark, "__version__") + if pyspark_ver is None or LooseVersion(pyspark_ver) < LooseVersion("2.4"): + logging.warning( + 'Found pyspark version "{}" installed. pyspark>=2.4.0 is recommended.'.format( + pyspark_ver if pyspark_ver is not None else "" + ) + ) + + +assert_python_version() +assert_pyspark_version() + +import pyspark +import pyarrow + +if LooseVersion(pyspark.__version__) < LooseVersion("3.0"): + if ( + LooseVersion(pyarrow.__version__) >= LooseVersion("0.15") + and "ARROW_PRE_0_15_IPC_FORMAT" not in os.environ + ): + import logging + + logging.warning( + "'ARROW_PRE_0_15_IPC_FORMAT' environment variable was not set. It is required to " + "set this environment variable to '1' in both driver and executor sides if you use " + "pyarrow>=0.15 and pyspark<3.0. " + "Koalas will set it for you but it does not work if there is a Spark context already " + "launched." + ) + # This is required to support PyArrow 0.15 in PySpark versions lower than 3.0. + # See SPARK-29367. + os.environ["ARROW_PRE_0_15_IPC_FORMAT"] = "1" +elif "ARROW_PRE_0_15_IPC_FORMAT" in os.environ: + raise RuntimeError( + "Please explicitly unset 'ARROW_PRE_0_15_IPC_FORMAT' environment variable in both " + "driver and executor sides. It is required to set this environment variable only " + "when you use pyarrow>=0.15 and pyspark<3.0." + ) + +if ( + LooseVersion(pyarrow.__version__) >= LooseVersion("2.0.0") + and "PYARROW_IGNORE_TIMEZONE" not in os.environ +): + import logging + + logging.warning( + "'PYARROW_IGNORE_TIMEZONE' environment variable was not set. It is required to " + "set this environment variable to '1' in both driver and executor sides if you use " + "pyarrow>=2.0.0. " + "Koalas will set it for you but it does not work if there is a Spark context already " + "launched." + ) + os.environ["PYARROW_IGNORE_TIMEZONE"] = "1" + +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.indexes.base import Index +from pyspark.pandas.indexes.category import CategoricalIndex +from pyspark.pandas.indexes.datetimes import DatetimeIndex +from pyspark.pandas.indexes.multi import MultiIndex +from pyspark.pandas.indexes.numeric import Float64Index, Int64Index +from pyspark.pandas.series import Series +from pyspark.pandas.groupby import NamedAgg + +__all__ = [ # noqa: F405 + "read_csv", + "read_parquet", + "to_datetime", + "date_range", + "from_pandas", + "get_dummies", + "DataFrame", + "Series", + "Index", + "MultiIndex", + "Int64Index", + "Float64Index", + "CategoricalIndex", + "DatetimeIndex", + "sql", + "range", + "concat", + "melt", + "get_option", + "set_option", + "reset_option", + "read_sql_table", + "read_sql_query", + "read_sql", + "options", + "option_context", + "NamedAgg", +] + + +def _auto_patch_spark(): + import os + import logging + + # Attach a usage logger. + logger_module = os.getenv("KOALAS_USAGE_LOGGER", "") + if logger_module != "": + try: + from pyspark.pandas import usage_logging + + usage_logging.attach(logger_module) + except Exception as e: + logger = logging.getLogger("pyspark.pandas.usage_logger") + logger.warning( + "Tried to attach usage logger `{}`, but an exception was raised: {}".format( + logger_module, str(e) + ) + ) + + # Autopatching is on by default. + x = os.getenv("SPARK_KOALAS_AUTOPATCH", "true") + if x.lower() in ("true", "1", "enabled"): + logger = logging.getLogger("spark") + logger.info( + "Patching spark automatically. You can disable it by setting " + "SPARK_KOALAS_AUTOPATCH=false in your environment" + ) + + from pyspark.sql import dataframe as df + + df.DataFrame.to_koalas = DataFrame.to_koalas + + +def _auto_patch_pandas(): + import pandas as pd + + # In order to use it in test cases. + global _frame_has_class_getitem + global _series_has_class_getitem + + _frame_has_class_getitem = hasattr(pd.DataFrame, "__class_getitem__") + _series_has_class_getitem = hasattr(pd.Series, "__class_getitem__") + + if sys.version_info >= (3, 7): + # Just in case pandas implements '__class_getitem__' later. + if not _frame_has_class_getitem: + pd.DataFrame.__class_getitem__ = lambda params: DataFrame.__class_getitem__(params) + + if not _series_has_class_getitem: + pd.Series.__class_getitem__ = lambda params: Series.__class_getitem__(params) + + +_auto_patch_spark() +_auto_patch_pandas() + +# Import after the usage logger is attached. +from pyspark.pandas.config import get_option, options, option_context, reset_option, set_option +from pyspark.pandas.namespace import * # F405 +from pyspark.pandas.sql import sql diff --git a/python/pyspark/pandas/accessors.py b/python/pyspark/pandas/accessors.py new file mode 100644 index 0000000000..39a647c0f3 --- /dev/null +++ b/python/pyspark/pandas/accessors.py @@ -0,0 +1,930 @@ +# +# 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. +# +""" +Koalas specific features. +""" +import inspect +from distutils.version import LooseVersion +from typing import Any, Optional, Tuple, Union, TYPE_CHECKING, cast +import types + +import numpy as np # noqa: F401 +import pandas as pd +import pyspark +from pyspark.sql import functions as F +from pyspark.sql.functions import pandas_udf, PandasUDFType +from pyspark.sql.types import StructField, StructType + +from pyspark.pandas.internal import ( + InternalFrame, + SPARK_INDEX_NAME_FORMAT, + SPARK_DEFAULT_SERIES_NAME, +) +from pyspark.pandas.typedef import infer_return_type, DataFrameType, ScalarType, SeriesType +from pyspark.pandas.spark.utils import as_nullable_spark_type, force_decimal_precision_scale +from pyspark.pandas.utils import ( + is_name_like_value, + is_name_like_tuple, + name_like_string, + scol_for, + verify_temp_column_name, +) + +if TYPE_CHECKING: + from pyspark.pandas.frame import DataFrame # noqa: F401 (SPARK-34943) + from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943) + + +class KoalasFrameMethods(object): + """ Koalas specific features for DataFrame. """ + + def __init__(self, frame: "DataFrame"): + self._kdf = frame + + def attach_id_column(self, id_type: str, column: Union[Any, Tuple]) -> "DataFrame": + """ + Attach a column to be used as identifier of rows similar to the default index. + + See also `Default Index type + `_. + + Parameters + ---------- + id_type : string + The id type. + + - 'sequence' : a sequence that increases one by one. + + .. note:: this uses Spark's Window without specifying partition specification. + This leads to move all data into single partition in single machine and + could cause serious performance degradation. + Avoid this method against very large dataset. + + - 'distributed-sequence' : a sequence that increases one by one, + by group-by and group-map approach in a distributed manner. + - 'distributed' : a monotonically increasing sequence simply by using PySpark’s + monotonically_increasing_id function in a fully distributed manner. + + column : string or tuple of string + The column name. + + Returns + ------- + DataFrame + The DataFrame attached the column. + + Examples + -------- + >>> df = pp.DataFrame({"x": ['a', 'b', 'c']}) + >>> df.koalas.attach_id_column(id_type="sequence", column="id") + x id + 0 a 0 + 1 b 1 + 2 c 2 + + >>> df.koalas.attach_id_column(id_type="distributed-sequence", column=0) + x 0 + 0 a 0 + 1 b 1 + 2 c 2 + + >>> df.koalas.attach_id_column(id_type="distributed", column=0.0) + ... # doctest: +ELLIPSIS +NORMALIZE_WHITESPACE + x 0.0 + 0 a ... + 1 b ... + 2 c ... + + For multi-index columns: + + >>> df = pp.DataFrame({("x", "y"): ['a', 'b', 'c']}) + >>> df.koalas.attach_id_column(id_type="sequence", column=("id-x", "id-y")) + x id-x + y id-y + 0 a 0 + 1 b 1 + 2 c 2 + + >>> df.koalas.attach_id_column(id_type="distributed-sequence", column=(0, 1.0)) + x 0 + y 1.0 + 0 a 0 + 1 b 1 + 2 c 2 + """ + from pyspark.pandas.frame import DataFrame + + if id_type == "sequence": + attach_func = InternalFrame.attach_sequence_column + elif id_type == "distributed-sequence": + attach_func = InternalFrame.attach_distributed_sequence_column + elif id_type == "distributed": + attach_func = InternalFrame.attach_distributed_column + else: + raise ValueError( + "id_type should be one of 'sequence', 'distributed-sequence' and 'distributed'" + ) + + assert is_name_like_value(column, allow_none=False), column + if not is_name_like_tuple(column): + column = (column,) + + internal = self._kdf._internal + + if len(column) != internal.column_labels_level: + raise ValueError( + "The given column `{}` must be the same length as the existing columns.".format( + column + ) + ) + elif column in internal.column_labels: + raise ValueError( + "The given column `{}` already exists.".format(name_like_string(column)) + ) + + # Make sure the underlying Spark column names are the form of + # `name_like_string(column_label)`. + sdf = internal.spark_frame.select( + [ + scol.alias(SPARK_INDEX_NAME_FORMAT(i)) + for i, scol in enumerate(internal.index_spark_columns) + ] + + [ + scol.alias(name_like_string(label)) + for scol, label in zip(internal.data_spark_columns, internal.column_labels) + ] + ) + sdf = attach_func(sdf, name_like_string(column)) + + return DataFrame( + InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, SPARK_INDEX_NAME_FORMAT(i)) for i in range(internal.index_level) + ], + index_names=internal.index_names, + index_dtypes=internal.index_dtypes, + column_labels=internal.column_labels + [column], + data_spark_columns=( + [scol_for(sdf, name_like_string(label)) for label in internal.column_labels] + + [scol_for(sdf, name_like_string(column))] + ), + data_dtypes=(internal.data_dtypes + [None]), + column_label_names=internal.column_label_names, + ).resolved_copy + ) + + def apply_batch(self, func, args=(), **kwds) -> "DataFrame": + """ + Apply a function that takes pandas DataFrame and outputs pandas DataFrame. The pandas + DataFrame given to the function is of a batch used internally. + + See also `Transform and apply a function + `_. + + .. note:: the `func` is unable to access to the whole input frame. Koalas internally + splits the input series into multiple batches and calls `func` with each batch multiple + times. Therefore, operations such as global aggregations are impossible. See the example + below. + + >>> # This case does not return the length of whole frame but of the batch internally + ... # used. + ... def length(pdf) -> pp.DataFrame[int]: + ... return pd.DataFrame([len(pdf)]) + ... + >>> df = pp.DataFrame({'A': range(1000)}) + >>> df.koalas.apply_batch(length) # doctest: +SKIP + c0 + 0 83 + 1 83 + 2 83 + ... + 10 83 + 11 83 + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def plus_one(x) -> pp.DataFrame[float, float]: + ... return x + 1 + + If the return type is specified, the output column names become + `c0, c1, c2 ... cn`. These names are positionally mapped to the returned + DataFrame in ``func``. + + To specify the column names, you can assign them in a pandas friendly style as below: + + >>> def plus_one(x) -> pp.DataFrame["a": float, "b": float]: + ... return x + 1 + + >>> pdf = pd.DataFrame({'a': [1, 2, 3], 'b': [3, 4, 5]}) + >>> def plus_one(x) -> pp.DataFrame[zip(pdf.dtypes, pdf.columns)]: + ... return x + 1 + + When the given function has the return type annotated, the original index of the + DataFrame will be lost and a default index will be attached to the result DataFrame. + Please be careful about configuring the default index. See also `Default Index Type + `_. + + + Parameters + ---------- + func : function + Function to apply to each pandas frame. + args : tuple + Positional arguments to pass to `func` in addition to the + array/series. + **kwds + Additional keyword arguments to pass as keywords arguments to + `func`. + + Returns + ------- + DataFrame + + See Also + -------- + DataFrame.apply: For row/columnwise operations. + DataFrame.applymap: For elementwise operations. + DataFrame.aggregate: Only perform aggregating type operations. + DataFrame.transform: Only perform transforming type operations. + Series.koalas.transform_batch: transform the search as each pandas chunpp. + + Examples + -------- + >>> df = pp.DataFrame([(1, 2), (3, 4), (5, 6)], columns=['A', 'B']) + >>> df + A B + 0 1 2 + 1 3 4 + 2 5 6 + + >>> def query_func(pdf) -> pp.DataFrame[int, int]: + ... return pdf.query('A == 1') + >>> df.koalas.apply_batch(query_func) + c0 c1 + 0 1 2 + + >>> def query_func(pdf) -> pp.DataFrame["A": int, "B": int]: + ... return pdf.query('A == 1') + >>> df.koalas.apply_batch(query_func) + A B + 0 1 2 + + You can also omit the type hints so Koalas infers the return schema as below: + + >>> df.koalas.apply_batch(lambda pdf: pdf.query('A == 1')) + A B + 0 1 2 + + You can also specify extra arguments. + + >>> def calculation(pdf, y, z) -> pp.DataFrame[int, int]: + ... return pdf ** y + z + >>> df.koalas.apply_batch(calculation, args=(10,), z=20) + c0 c1 + 0 21 1044 + 1 59069 1048596 + 2 9765645 60466196 + + You can also use ``np.ufunc`` and built-in functions as input. + + >>> df.koalas.apply_batch(np.add, args=(10,)) + A B + 0 11 12 + 1 13 14 + 2 15 16 + + >>> (df * -1).koalas.apply_batch(abs) + A B + 0 1 2 + 1 3 4 + 2 5 6 + + """ + # TODO: codes here partially duplicate `DataFrame.apply`. Can we deduplicate? + + from pyspark.pandas.groupby import GroupBy + from pyspark.pandas.frame import DataFrame + from pyspark import pandas as pp + + if not isinstance(func, types.FunctionType): + assert callable(func), "the first argument should be a callable function." + f = func + func = lambda *args, **kwargs: f(*args, **kwargs) + + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + should_infer_schema = return_sig is None + should_use_map_in_pandas = LooseVersion(pyspark.__version__) >= "3.0" + + original_func = func + func = lambda o: original_func(o, *args, **kwds) + + self_applied = DataFrame(self._kdf._internal.resolved_copy) # type: DataFrame + + if should_infer_schema: + # Here we execute with the first 1000 to get the return type. + # If the records were less than 1000, it uses pandas API directly for a shortcut. + limit = pp.get_option("compute.shortcut_limit") + pdf = self_applied.head(limit + 1)._to_internal_pandas() + applied = func(pdf) + if not isinstance(applied, pd.DataFrame): + raise ValueError( + "The given function should return a frame; however, " + "the return type was %s." % type(applied) + ) + kdf = pp.DataFrame(applied) # type: DataFrame + if len(pdf) <= limit: + return kdf + + return_schema = force_decimal_precision_scale( + as_nullable_spark_type(kdf._internal.to_internal_spark_frame.schema) + ) + if should_use_map_in_pandas: + output_func = GroupBy._make_pandas_df_builder_func( + self_applied, func, return_schema, retain_index=True + ) + sdf = self_applied._internal.to_internal_spark_frame.mapInPandas( + lambda iterator: map(output_func, iterator), schema=return_schema + ) + else: + sdf = GroupBy._spark_group_map_apply( + self_applied, func, (F.spark_partition_id(),), return_schema, retain_index=True + ) + + # If schema is inferred, we can restore indexes too. + internal = kdf._internal.with_new_sdf(sdf) + else: + return_type = infer_return_type(original_func) + is_return_dataframe = isinstance(return_type, DataFrameType) + if not is_return_dataframe: + raise TypeError( + "The given function should specify a frame as its type " + "hints; however, the return type was %s." % return_sig + ) + return_schema = cast(DataFrameType, return_type).spark_type + + if should_use_map_in_pandas: + output_func = GroupBy._make_pandas_df_builder_func( + self_applied, func, return_schema, retain_index=False + ) + sdf = self_applied._internal.to_internal_spark_frame.mapInPandas( + lambda iterator: map(output_func, iterator), schema=return_schema + ) + else: + sdf = GroupBy._spark_group_map_apply( + self_applied, func, (F.spark_partition_id(),), return_schema, retain_index=False + ) + + # Otherwise, it loses index. + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=None, + data_dtypes=cast(DataFrameType, return_type).dtypes, + ) + + return DataFrame(internal) + + def transform_batch(self, func, *args, **kwargs) -> Union["DataFrame", "Series"]: + """ + Transform chunks with a function that takes pandas DataFrame and outputs pandas DataFrame. + The pandas DataFrame given to the function is of a batch used internally. The length of + each input and output should be the same. + + See also `Transform and apply a function + `_. + + .. note:: the `func` is unable to access to the whole input frame. Koalas internally + splits the input series into multiple batches and calls `func` with each batch multiple + times. Therefore, operations such as global aggregations are impossible. See the example + below. + + >>> # This case does not return the length of whole frame but of the batch internally + ... # used. + ... def length(pdf) -> pp.DataFrame[int]: + ... return pd.DataFrame([len(pdf)] * len(pdf)) + ... + >>> df = pp.DataFrame({'A': range(1000)}) + >>> df.koalas.transform_batch(length) # doctest: +SKIP + c0 + 0 83 + 1 83 + 2 83 + ... + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def plus_one(x) -> pp.DataFrame[float, float]: + ... return x + 1 + + If the return type is specified, the output column names become + `c0, c1, c2 ... cn`. These names are positionally mapped to the returned + DataFrame in ``func``. + + To specify the column names, you can assign them in a pandas friendly style as below: + + >>> def plus_one(x) -> pp.DataFrame['a': float, 'b': float]: + ... return x + 1 + + >>> pdf = pd.DataFrame({'a': [1, 2, 3], 'b': [3, 4, 5]}) + >>> def plus_one(x) -> pp.DataFrame[zip(pdf.dtypes, pdf.columns)]: + ... return x + 1 + + When the given function returns DataFrame and has the return type annotated, the + original index of the DataFrame will be lost and then a default index will be attached + to the result. Please be careful about configuring the default index. See also + `Default Index Type + `_. + + Parameters + ---------- + func : function + Function to transform each pandas frame. + *args + Positional arguments to pass to func. + **kwargs + Keyword arguments to pass to func. + + Returns + ------- + DataFrame or Series + + See Also + -------- + DataFrame.koalas.apply_batch: For row/columnwise operations. + Series.koalas.transform_batch: transform the search as each pandas chunpp. + + Examples + -------- + >>> df = pp.DataFrame([(1, 2), (3, 4), (5, 6)], columns=['A', 'B']) + >>> df + A B + 0 1 2 + 1 3 4 + 2 5 6 + + >>> def plus_one_func(pdf) -> pp.DataFrame[int, int]: + ... return pdf + 1 + >>> df.koalas.transform_batch(plus_one_func) + c0 c1 + 0 2 3 + 1 4 5 + 2 6 7 + + >>> def plus_one_func(pdf) -> pp.DataFrame['A': int, 'B': int]: + ... return pdf + 1 + >>> df.koalas.transform_batch(plus_one_func) + A B + 0 2 3 + 1 4 5 + 2 6 7 + + >>> def plus_one_func(pdf) -> pp.Series[int]: + ... return pdf.B + 1 + >>> df.koalas.transform_batch(plus_one_func) + 0 3 + 1 5 + 2 7 + dtype: int64 + + You can also omit the type hints so Koalas infers the return schema as below: + + >>> df.koalas.transform_batch(lambda pdf: pdf + 1) + A B + 0 2 3 + 1 4 5 + 2 6 7 + + >>> (df * -1).koalas.transform_batch(abs) + A B + 0 1 2 + 1 3 4 + 2 5 6 + + Note that you should not transform the index. The index information will not change. + + >>> df.koalas.transform_batch(lambda pdf: pdf.B + 1) + 0 3 + 1 5 + 2 7 + Name: B, dtype: int64 + + You can also specify extra arguments as below. + + >>> df.koalas.transform_batch(lambda pdf, a, b, c: pdf.B + a + b + c, 1, 2, c=3) + 0 8 + 1 10 + 2 12 + Name: B, dtype: int64 + """ + from pyspark.pandas.groupby import GroupBy + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.series import first_series + from pyspark import pandas as pp + + assert callable(func), "the first argument should be a callable function." + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + should_infer_schema = return_sig is None + original_func = func + func = lambda o: original_func(o, *args, **kwargs) + + names = self._kdf._internal.to_internal_spark_frame.schema.names + should_by_pass = LooseVersion(pyspark.__version__) >= "3.0" + + def pandas_concat(series): + # The input can only be a DataFrame for struct from Spark 3.0. + # This works around to make the input as a frame. See SPARK-27240 + pdf = pd.concat(series, axis=1) + pdf.columns = names + return pdf + + def apply_func(pdf): + return func(pdf).to_frame() + + def pandas_extract(pdf, name): + # This is for output to work around a DataFrame for struct + # from Spark 3.0. See SPARK-23836 + return pdf[name] + + def pandas_series_func(f, by_pass): + ff = f + if by_pass: + return lambda *series: first_series(ff(*series)) + else: + return lambda *series: first_series(ff(pandas_concat(series))) + + def pandas_frame_func(f, field_name): + ff = f + return lambda *series: pandas_extract(ff(pandas_concat(series)), field_name) + + if should_infer_schema: + # Here we execute with the first 1000 to get the return type. + # If the records were less than 1000, it uses pandas API directly for a shortcut. + limit = pp.get_option("compute.shortcut_limit") + pdf = self._kdf.head(limit + 1)._to_internal_pandas() + transformed = func(pdf) + if not isinstance(transformed, (pd.DataFrame, pd.Series)): + raise ValueError( + "The given function should return a frame; however, " + "the return type was %s." % type(transformed) + ) + if len(transformed) != len(pdf): + raise ValueError("transform_batch cannot produce aggregated results") + kdf_or_kser = pp.from_pandas(transformed) + + if isinstance(kdf_or_kser, pp.Series): + kser = cast(pp.Series, kdf_or_kser) + + spark_return_type = force_decimal_precision_scale( + as_nullable_spark_type(kser.spark.data_type) + ) + return_schema = StructType( + [StructField(SPARK_DEFAULT_SERIES_NAME, spark_return_type)] + ) + output_func = GroupBy._make_pandas_df_builder_func( + self._kdf, apply_func, return_schema, retain_index=False + ) + + pudf = pandas_udf( + pandas_series_func(output_func, should_by_pass), + returnType=spark_return_type, + functionType=PandasUDFType.SCALAR, + ) + columns = self._kdf._internal.spark_columns + # TODO: Index will be lost in this case. + internal = self._kdf._internal.copy( + column_labels=kser._internal.column_labels, + data_spark_columns=[ + (pudf(F.struct(*columns)) if should_by_pass else pudf(*columns)).alias( + kser._internal.data_spark_column_names[0] + ) + ], + data_dtypes=kser._internal.data_dtypes, + column_label_names=kser._internal.column_label_names, + ) + return first_series(DataFrame(internal)) + else: + kdf = cast(DataFrame, kdf_or_kser) + if len(pdf) <= limit: + # only do the short cut when it returns a frame to avoid + # operations on different dataframes in case of series. + return kdf + + # Force nullability. + return_schema = force_decimal_precision_scale( + as_nullable_spark_type(kdf._internal.to_internal_spark_frame.schema) + ) + + self_applied = DataFrame(self._kdf._internal.resolved_copy) # type: DataFrame + + output_func = GroupBy._make_pandas_df_builder_func( + self_applied, func, return_schema, retain_index=True + ) + columns = self_applied._internal.spark_columns + if should_by_pass: + pudf = pandas_udf( + output_func, returnType=return_schema, functionType=PandasUDFType.SCALAR + ) + temp_struct_column = verify_temp_column_name( + self_applied._internal.spark_frame, "__temp_struct__" + ) + applied = pudf(F.struct(*columns)).alias(temp_struct_column) + sdf = self_applied._internal.spark_frame.select(applied) + sdf = sdf.selectExpr("%s.*" % temp_struct_column) + else: + applied = [] + for field in return_schema.fields: + applied.append( + pandas_udf( + pandas_frame_func(output_func, field.name), + returnType=field.dataType, + functionType=PandasUDFType.SCALAR, + )(*columns).alias(field.name) + ) + sdf = self_applied._internal.spark_frame.select(*applied) + return DataFrame(kdf._internal.with_new_sdf(sdf)) + else: + return_type = infer_return_type(original_func) + is_return_series = isinstance(return_type, SeriesType) + is_return_dataframe = isinstance(return_type, DataFrameType) + if not is_return_dataframe and not is_return_series: + raise TypeError( + "The given function should specify a frame or series as its type " + "hints; however, the return type was %s." % return_sig + ) + if is_return_series: + spark_return_type = force_decimal_precision_scale( + as_nullable_spark_type(cast(SeriesType, return_type).spark_type) + ) + return_schema = StructType( + [StructField(SPARK_DEFAULT_SERIES_NAME, spark_return_type)] + ) + output_func = GroupBy._make_pandas_df_builder_func( + self._kdf, apply_func, return_schema, retain_index=False + ) + + pudf = pandas_udf( + pandas_series_func(output_func, should_by_pass), + returnType=spark_return_type, + functionType=PandasUDFType.SCALAR, + ) + columns = self._kdf._internal.spark_columns + internal = self._kdf._internal.copy( + column_labels=[None], + data_spark_columns=[ + (pudf(F.struct(*columns)) if should_by_pass else pudf(*columns)).alias( + SPARK_DEFAULT_SERIES_NAME + ) + ], + data_dtypes=[cast(SeriesType, return_type).dtype], + column_label_names=None, + ) + return first_series(DataFrame(internal)) + else: + return_schema = cast(DataFrameType, return_type).spark_type + + self_applied = DataFrame(self._kdf._internal.resolved_copy) + + output_func = GroupBy._make_pandas_df_builder_func( + self_applied, func, return_schema, retain_index=False + ) + columns = self_applied._internal.spark_columns + + if should_by_pass: + pudf = pandas_udf( + output_func, returnType=return_schema, functionType=PandasUDFType.SCALAR + ) + temp_struct_column = verify_temp_column_name( + self_applied._internal.spark_frame, "__temp_struct__" + ) + applied = pudf(F.struct(*columns)).alias(temp_struct_column) + sdf = self_applied._internal.spark_frame.select(applied) + sdf = sdf.selectExpr("%s.*" % temp_struct_column) + else: + applied = [] + for field in return_schema.fields: + applied.append( + pandas_udf( + pandas_frame_func(output_func, field.name), + returnType=field.dataType, + functionType=PandasUDFType.SCALAR, + )(*columns).alias(field.name) + ) + sdf = self_applied._internal.spark_frame.select(*applied) + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=None, + data_dtypes=cast(DataFrameType, return_type).dtypes, + ) + return DataFrame(internal) + + +class KoalasSeriesMethods(object): + """ Koalas specific features for Series. """ + + def __init__(self, series: "Series"): + self._kser = series + + def transform_batch(self, func, *args, **kwargs) -> "Series": + """ + Transform the data with the function that takes pandas Series and outputs pandas Series. + The pandas Series given to the function is of a batch used internally. + + See also `Transform and apply a function + `_. + + .. note:: the `func` is unable to access to the whole input series. Koalas internally + splits the input series into multiple batches and calls `func` with each batch multiple + times. Therefore, operations such as global aggregations are impossible. See the example + below. + + >>> # This case does not return the length of whole frame but of the batch internally + ... # used. + ... def length(pser) -> pp.Series[int]: + ... return pd.Series([len(pser)] * len(pser)) + ... + >>> df = pp.DataFrame({'A': range(1000)}) + >>> df.A.koalas.transform_batch(length) # doctest: +SKIP + c0 + 0 83 + 1 83 + 2 83 + ... + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def plus_one(x) -> pp.Series[int]: + ... return x + 1 + + Parameters + ---------- + func : function + Function to apply to each pandas frame. + *args + Positional arguments to pass to func. + **kwargs + Keyword arguments to pass to func. + + Returns + ------- + DataFrame + + See Also + -------- + DataFrame.koalas.apply_batch : Similar but it takes pandas DataFrame as its internal batch. + + Examples + -------- + >>> df = pp.DataFrame([(1, 2), (3, 4), (5, 6)], columns=['A', 'B']) + >>> df + A B + 0 1 2 + 1 3 4 + 2 5 6 + + >>> def plus_one_func(pser) -> pp.Series[np.int64]: + ... return pser + 1 + >>> df.A.koalas.transform_batch(plus_one_func) + 0 2 + 1 4 + 2 6 + Name: A, dtype: int64 + + You can also omit the type hints so Koalas infers the return schema as below: + + >>> df.A.koalas.transform_batch(lambda pser: pser + 1) + 0 2 + 1 4 + 2 6 + Name: A, dtype: int64 + + You can also specify extra arguments. + + >>> def plus_one_func(pser, a, b, c=3) -> pp.Series[np.int64]: + ... return pser + a + b + c + >>> df.A.koalas.transform_batch(plus_one_func, 1, b=2) + 0 7 + 1 9 + 2 11 + Name: A, dtype: int64 + + You can also use ``np.ufunc`` and built-in functions as input. + + >>> df.A.koalas.transform_batch(np.add, 10) + 0 11 + 1 13 + 2 15 + Name: A, dtype: int64 + + >>> (df * -1).A.koalas.transform_batch(abs) + 0 1 + 1 3 + 2 5 + Name: A, dtype: int64 + """ + assert callable(func), "the first argument should be a callable function." + + return_sig = None + try: + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + except TypeError: + # Falls back to schema inference if it fails to get signature. + pass + + return_type = None + if return_sig is not None: + # Extract the signature arguments from this function. + sig_return = infer_return_type(func) + if not isinstance(sig_return, SeriesType): + raise ValueError( + "Expected the return type of this function to be of type column," + " but found type {}".format(sig_return) + ) + return_type = cast(SeriesType, sig_return) + + return self._transform_batch(lambda c: func(c, *args, **kwargs), return_type) + + def _transform_batch(self, func, return_type: Optional[Union[SeriesType, ScalarType]]): + from pyspark.pandas.groupby import GroupBy + from pyspark.pandas.series import Series, first_series + from pyspark import pandas as pp + + if not isinstance(func, types.FunctionType): + f = func + func = lambda *args, **kwargs: f(*args, **kwargs) + + if return_type is None: + # TODO: In this case, it avoids the shortcut for now (but only infers schema) + # because it returns a series from a different DataFrame and it has a different + # anchor. We should fix this to allow the shortcut or only allow to infer + # schema. + limit = pp.get_option("compute.shortcut_limit") + pser = self._kser.head(limit + 1)._to_internal_pandas() + transformed = pser.transform(func) + kser = Series(transformed) # type: Series + spark_return_type = force_decimal_precision_scale( + as_nullable_spark_type(kser.spark.data_type) + ) + dtype = kser.dtype + else: + spark_return_type = return_type.spark_type + dtype = return_type.dtype + + kdf = self._kser.to_frame() + columns = kdf._internal.spark_column_names + + def pandas_concat(series): + # The input can only be a DataFrame for struct from Spark 3.0. + # This works around to make the input as a frame. See SPARK-27240 + pdf = pd.concat(series, axis=1) + pdf.columns = columns + return pdf + + def apply_func(pdf): + return func(first_series(pdf)).to_frame() + + return_schema = StructType([StructField(SPARK_DEFAULT_SERIES_NAME, spark_return_type)]) + output_func = GroupBy._make_pandas_df_builder_func( + kdf, apply_func, return_schema, retain_index=False + ) + + pudf = pandas_udf( + lambda *series: first_series(output_func(pandas_concat(series))), + returnType=spark_return_type, + functionType=PandasUDFType.SCALAR, + ) + + return self._kser._with_new_scol( + scol=pudf(*kdf._internal.spark_columns).alias( + self._kser._internal.spark_column_names[0] + ), + dtype=dtype, + ) diff --git a/python/pyspark/pandas/base.py b/python/pyspark/pandas/base.py new file mode 100644 index 0000000000..6ae27a8b0d --- /dev/null +++ b/python/pyspark/pandas/base.py @@ -0,0 +1,1995 @@ +# +# 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. +# + +""" +Base and utility classes for Koalas objects. +""" +from abc import ABCMeta, abstractmethod +import datetime +from functools import wraps, partial +from itertools import chain +from typing import Any, Callable, Optional, Tuple, Union, cast, TYPE_CHECKING +import warnings + +import numpy as np +import pandas as pd # noqa: F401 +from pandas.api.types import is_list_like, CategoricalDtype +from pyspark import sql as spark +from pyspark.sql import functions as F, Window, Column +from pyspark.sql.types import ( + BooleanType, + DateType, + DoubleType, + FloatType, + IntegralType, + LongType, + NumericType, + StringType, + TimestampType, +) + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas import numpy_compat +from pyspark.pandas.config import get_option, option_context +from pyspark.pandas.internal import ( + InternalFrame, + NATURAL_ORDER_COLUMN_NAME, + SPARK_DEFAULT_INDEX_NAME, +) +from pyspark.pandas.spark import functions as SF +from pyspark.pandas.spark.accessors import SparkIndexOpsMethods +from pyspark.pandas.typedef import ( + Dtype, + as_spark_type, + extension_dtypes, + koalas_dtype, + spark_type_to_pandas_dtype, +) +from pyspark.pandas.utils import ( + combine_frames, + same_anchor, + scol_for, + validate_axis, + ERROR_MESSAGE_CANNOT_COMBINE, +) +from pyspark.pandas.frame import DataFrame + +if TYPE_CHECKING: + from pyspark.pandas.indexes import Index # noqa: F401 (SPARK-34943) + from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943) + + +def should_alignment_for_column_op(self: "IndexOpsMixin", other: "IndexOpsMixin") -> bool: + from pyspark.pandas.series import Series + + if isinstance(self, Series) and isinstance(other, Series): + return not same_anchor(self, other) + else: + return self._internal.spark_frame is not other._internal.spark_frame + + +def align_diff_index_ops(func, this_index_ops: "IndexOpsMixin", *args) -> "IndexOpsMixin": + """ + Align the `IndexOpsMixin` objects and apply the function. + + Parameters + ---------- + func : The function to apply + this_index_ops : IndexOpsMixin + A base `IndexOpsMixin` object + args : list of other arguments including other `IndexOpsMixin` objects + + Returns + ------- + `Index` if all `this_index_ops` and arguments are `Index`; otherwise `Series` + """ + from pyspark.pandas.indexes import Index + from pyspark.pandas.series import Series, first_series + + cols = [arg for arg in args if isinstance(arg, IndexOpsMixin)] + + if isinstance(this_index_ops, Series) and all(isinstance(col, Series) for col in cols): + combined = combine_frames( + this_index_ops.to_frame(), + *[cast(Series, col).rename(i) for i, col in enumerate(cols)], + how="full" + ) + + return column_op(func)( + combined["this"]._kser_for(combined["this"]._internal.column_labels[0]), + *[ + combined["that"]._kser_for(label) + for label in combined["that"]._internal.column_labels + ] + ).rename(this_index_ops.name) + else: + # This could cause as many counts, reset_index calls, joins for combining + # as the number of `Index`s in `args`. So far it's fine since we can assume the ops + # only work between at most two `Index`s. We might need to fix it in the future. + + self_len = len(this_index_ops) + if any(len(col) != self_len for col in args if isinstance(col, IndexOpsMixin)): + raise ValueError("operands could not be broadcast together with shapes") + + with option_context("compute.default_index_type", "distributed-sequence"): + if isinstance(this_index_ops, Index) and all(isinstance(col, Index) for col in cols): + return Index( + column_op(func)( + this_index_ops.to_series().reset_index(drop=True), + *[ + arg.to_series().reset_index(drop=True) + if isinstance(arg, Index) + else arg + for arg in args + ] + ).sort_index(), + name=this_index_ops.name, + ) + elif isinstance(this_index_ops, Series): + this = this_index_ops.reset_index() + that = [ + cast(Series, col.to_series() if isinstance(col, Index) else col) + .rename(i) + .reset_index(drop=True) + for i, col in enumerate(cols) + ] + + combined = combine_frames(this, *that, how="full").sort_index() + combined = combined.set_index( + combined._internal.column_labels[: this_index_ops._internal.index_level] + ) + combined.index.names = this_index_ops._internal.index_names + + return column_op(func)( + first_series(combined["this"]), + *[ + combined["that"]._kser_for(label) + for label in combined["that"]._internal.column_labels + ] + ).rename(this_index_ops.name) + else: + this = cast(Index, this_index_ops).to_frame().reset_index(drop=True) + + that_series = next(col for col in cols if isinstance(col, Series)) + that_frame = that_series._kdf[ + [ + cast(Series, col.to_series() if isinstance(col, Index) else col).rename(i) + for i, col in enumerate(cols) + ] + ] + + combined = combine_frames(this, that_frame.reset_index()).sort_index() + + self_index = ( + combined["this"].set_index(combined["this"]._internal.column_labels).index + ) + + other = combined["that"].set_index( + combined["that"]._internal.column_labels[: that_series._internal.index_level] + ) + other.index.names = that_series._internal.index_names + + return column_op(func)( + self_index, + *[ + other._kser_for(label) + for label, col in zip(other._internal.column_labels, cols) + ] + ).rename(that_series.name) + + +def booleanize_null(scol, f) -> Column: + """ + Booleanize Null in Spark Column + """ + comp_ops = [ + getattr(Column, "__{}__".format(comp_op)) + for comp_op in ["eq", "ne", "lt", "le", "ge", "gt"] + ] + + if f in comp_ops: + # if `f` is "!=", fill null with True otherwise False + filler = f == Column.__ne__ + scol = F.when(scol.isNull(), filler).otherwise(scol) + + return scol + + +def column_op(f): + """ + A decorator that wraps APIs taking/returning Spark Column so that Koalas Series can be + supported too. If this decorator is used for the `f` function that takes Spark Column and + returns Spark Column, decorated `f` takes Koalas Series as well and returns Koalas + Series. + + :param f: a function that takes Spark Column and returns Spark Column. + :param self: Koalas Series + :param args: arguments that the function `f` takes. + """ + + @wraps(f) + def wrapper(self, *args): + from pyspark.pandas.series import Series + + # It is possible for the function `f` takes other arguments than Spark Column. + # To cover this case, explicitly check if the argument is Koalas Series and + # extract Spark Column. For other arguments, they are used as are. + cols = [arg for arg in args if isinstance(arg, IndexOpsMixin)] + + if all(not should_alignment_for_column_op(self, col) for col in cols): + # Same DataFrame anchors + args = [arg.spark.column if isinstance(arg, IndexOpsMixin) else arg for arg in args] + scol = f(self.spark.column, *args) + + spark_type = self._internal.spark_frame.select(scol).schema[0].dataType + use_extension_dtypes = any( + isinstance(col.dtype, extension_dtypes) for col in [self] + cols + ) + dtype = spark_type_to_pandas_dtype( + spark_type, use_extension_dtypes=use_extension_dtypes + ) + + if not isinstance(dtype, extension_dtypes): + scol = booleanize_null(scol, f) + + if isinstance(self, Series) or not any(isinstance(col, Series) for col in cols): + index_ops = self._with_new_scol(scol, dtype=dtype) + else: + kser = next(col for col in cols if isinstance(col, Series)) + index_ops = kser._with_new_scol(scol, dtype=dtype) + elif get_option("compute.ops_on_diff_frames"): + index_ops = align_diff_index_ops(f, self, *args) + else: + raise ValueError(ERROR_MESSAGE_CANNOT_COMBINE) + + if not all(self.name == col.name for col in cols): + index_ops = index_ops.rename(None) + + return index_ops + + return wrapper + + +def numpy_column_op(f): + @wraps(f) + def wrapper(self, *args): + # PySpark does not support NumPy type out of the box. For now, we convert NumPy types + # into some primitive types understandable in PySpark. + new_args = [] + for arg in args: + # TODO: This is a quick hack to support NumPy type. We should revisit this. + if isinstance(self.spark.data_type, LongType) and isinstance(arg, np.timedelta64): + new_args.append(float(arg / np.timedelta64(1, "s"))) + else: + new_args.append(arg) + return column_op(f)(self, *new_args) + + return wrapper + + +class IndexOpsMixin(object, metaclass=ABCMeta): + """common ops mixin to support a unified interface / docs for Series / Index + + Assuming there are following attributes or properties and function. + """ + + @property + @abstractmethod + def _internal(self) -> InternalFrame: + pass + + @property + @abstractmethod + def _kdf(self) -> DataFrame: + pass + + @abstractmethod + def _with_new_scol(self, scol: spark.Column, *, dtype=None): + pass + + @property + @abstractmethod + def _column_label(self) -> Tuple: + pass + + @property + @abstractmethod + def spark(self) -> SparkIndexOpsMethods: + pass + + @property + def spark_column(self) -> Column: + warnings.warn( + "Series.spark_column is deprecated as of Series.spark.column. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.column + + spark_column.__doc__ = SparkIndexOpsMethods.column.__doc__ + + # arithmetic operators + __neg__ = column_op(Column.__neg__) + + def __add__(self, other) -> Union["Series", "Index"]: + if not isinstance(self.spark.data_type, StringType) and ( + (isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, StringType)) + or isinstance(other, str) + ): + raise TypeError("string addition can only be applied to string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("addition can not be applied to date times.") + + if isinstance(self.spark.data_type, StringType): + # Concatenate string columns + if isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, StringType): + return column_op(F.concat)(self, other) + # Handle df['col'] + 'literal' + elif isinstance(other, str): + return column_op(F.concat)(self, F.lit(other)) + else: + raise TypeError("string addition can only be applied to string series or literals.") + else: + return column_op(Column.__add__)(self, other) + + def __sub__(self, other) -> Union["Series", "Index"]: + if ( + isinstance(self.spark.data_type, StringType) + or (isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, StringType)) + or isinstance(other, str) + ): + raise TypeError("substraction can not be applied to string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + # Note that timestamp subtraction casts arguments to integer. This is to mimic pandas's + # behaviors. pandas returns 'timedelta64[ns]' from 'datetime64[ns]'s subtraction. + msg = ( + "Note that there is a behavior difference of timestamp subtraction. " + "The timestamp subtraction returns an integer in seconds, " + "whereas pandas returns 'timedelta64[ns]'." + ) + if isinstance(other, IndexOpsMixin) and isinstance( + other.spark.data_type, TimestampType + ): + warnings.warn(msg, UserWarning) + return self.astype("long") - other.astype("long") + elif isinstance(other, datetime.datetime): + warnings.warn(msg, UserWarning) + return self.astype("long") - F.lit(other).cast(as_spark_type("long")) + else: + raise TypeError("datetime subtraction can only be applied to datetime series.") + elif isinstance(self.spark.data_type, DateType): + # Note that date subtraction casts arguments to integer. This is to mimic pandas's + # behaviors. pandas returns 'timedelta64[ns]' in days from date's subtraction. + msg = ( + "Note that there is a behavior difference of date subtraction. " + "The date subtraction returns an integer in days, " + "whereas pandas returns 'timedelta64[ns]'." + ) + if isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, DateType): + warnings.warn(msg, UserWarning) + return column_op(F.datediff)(self, other).astype("long") + elif isinstance(other, datetime.date) and not isinstance(other, datetime.datetime): + warnings.warn(msg, UserWarning) + return column_op(F.datediff)(self, F.lit(other)).astype("long") + else: + raise TypeError("date subtraction can only be applied to date series.") + return column_op(Column.__sub__)(self, other) + + def __mul__(self, other) -> Union["Series", "Index"]: + if isinstance(other, str): + raise TypeError("multiplication can not be applied to a string literal.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("multiplication can not be applied to date times.") + + if ( + isinstance(self.spark.data_type, IntegralType) + and isinstance(other, IndexOpsMixin) + and isinstance(other.spark.data_type, StringType) + ): + return column_op(SF.repeat)(other, self) + + if isinstance(self.spark.data_type, StringType): + if ( + isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, IntegralType) + ) or isinstance(other, int): + return column_op(SF.repeat)(self, other) + else: + raise TypeError( + "a string series can only be multiplied to an int series or literal" + ) + + return column_op(Column.__mul__)(self, other) + + def __truediv__(self, other) -> Union["Series", "Index"]: + """ + __truediv__ has different behaviour between pandas and PySpark for several cases. + 1. When divide np.inf by zero, PySpark returns null whereas pandas returns np.inf + 2. When divide positive number by zero, PySpark returns null whereas pandas returns np.inf + 3. When divide -np.inf by zero, PySpark returns null whereas pandas returns -np.inf + 4. When divide negative number by zero, PySpark returns null whereas pandas returns -np.inf + + +-------------------------------------------+ + | dividend (divisor: 0) | PySpark | pandas | + |-----------------------|---------|---------| + | np.inf | null | np.inf | + | -np.inf | null | -np.inf | + | 10 | null | np.inf | + | -10 | null | -np.inf | + +-----------------------|---------|---------+ + """ + + if ( + isinstance(self.spark.data_type, StringType) + or (isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, StringType)) + or isinstance(other, str) + ): + raise TypeError("division can not be applied on string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("division can not be applied to date times.") + + def truediv(left, right): + return F.when(F.lit(right != 0) | F.lit(right).isNull(), left.__div__(right)).otherwise( + F.when(F.lit(left == np.inf) | F.lit(left == -np.inf), left).otherwise( + F.lit(np.inf).__div__(left) + ) + ) + + return numpy_column_op(truediv)(self, other) + + def __mod__(self, other) -> Union["Series", "Index"]: + if ( + isinstance(self.spark.data_type, StringType) + or (isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, StringType)) + or isinstance(other, str) + ): + raise TypeError("modulo can not be applied on string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("modulo can not be applied to date times.") + + def mod(left, right): + return ((left % right) + right) % right + + return column_op(mod)(self, other) + + def __radd__(self, other) -> Union["Series", "Index"]: + # Handle 'literal' + df['col'] + if not isinstance(self.spark.data_type, StringType) and isinstance(other, str): + raise TypeError("string addition can only be applied to string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("addition can not be applied to date times.") + + if isinstance(self.spark.data_type, StringType): + if isinstance(other, str): + return self._with_new_scol( + F.concat(F.lit(other), self.spark.column) + ) # TODO: dtype? + else: + raise TypeError("string addition can only be applied to string series or literals.") + else: + return column_op(Column.__radd__)(self, other) + + def __rsub__(self, other) -> Union["Series", "Index"]: + if isinstance(self.spark.data_type, StringType) or isinstance(other, str): + raise TypeError("substraction can not be applied to string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + # Note that timestamp subtraction casts arguments to integer. This is to mimic pandas's + # behaviors. pandas returns 'timedelta64[ns]' from 'datetime64[ns]'s subtraction. + msg = ( + "Note that there is a behavior difference of timestamp subtraction. " + "The timestamp subtraction returns an integer in seconds, " + "whereas pandas returns 'timedelta64[ns]'." + ) + if isinstance(other, datetime.datetime): + warnings.warn(msg, UserWarning) + return -(self.astype("long") - F.lit(other).cast(as_spark_type("long"))) + else: + raise TypeError("datetime subtraction can only be applied to datetime series.") + elif isinstance(self.spark.data_type, DateType): + # Note that date subtraction casts arguments to integer. This is to mimic pandas's + # behaviors. pandas returns 'timedelta64[ns]' in days from date's subtraction. + msg = ( + "Note that there is a behavior difference of date subtraction. " + "The date subtraction returns an integer in days, " + "whereas pandas returns 'timedelta64[ns]'." + ) + if isinstance(other, datetime.date) and not isinstance(other, datetime.datetime): + warnings.warn(msg, UserWarning) + return -column_op(F.datediff)(self, F.lit(other)).astype("long") + else: + raise TypeError("date subtraction can only be applied to date series.") + return column_op(Column.__rsub__)(self, other) + + def __rmul__(self, other) -> Union["Series", "Index"]: + if isinstance(other, str): + raise TypeError("multiplication can not be applied to a string literal.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("multiplication can not be applied to date times.") + + if isinstance(self.spark.data_type, StringType): + if isinstance(other, int): + return column_op(SF.repeat)(self, other) + else: + raise TypeError( + "a string series can only be multiplied to an int series or literal" + ) + + return column_op(Column.__rmul__)(self, other) + + def __rtruediv__(self, other) -> Union["Series", "Index"]: + if isinstance(self.spark.data_type, StringType) or isinstance(other, str): + raise TypeError("division can not be applied on string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("division can not be applied to date times.") + + def rtruediv(left, right): + return F.when(left == 0, F.lit(np.inf).__div__(right)).otherwise( + F.lit(right).__truediv__(left) + ) + + return numpy_column_op(rtruediv)(self, other) + + def __floordiv__(self, other) -> Union["Series", "Index"]: + """ + __floordiv__ has different behaviour between pandas and PySpark for several cases. + 1. When divide np.inf by zero, PySpark returns null whereas pandas returns np.inf + 2. When divide positive number by zero, PySpark returns null whereas pandas returns np.inf + 3. When divide -np.inf by zero, PySpark returns null whereas pandas returns -np.inf + 4. When divide negative number by zero, PySpark returns null whereas pandas returns -np.inf + + +-------------------------------------------+ + | dividend (divisor: 0) | PySpark | pandas | + |-----------------------|---------|---------| + | np.inf | null | np.inf | + | -np.inf | null | -np.inf | + | 10 | null | np.inf | + | -10 | null | -np.inf | + +-----------------------|---------|---------+ + """ + if ( + isinstance(self.spark.data_type, StringType) + or (isinstance(other, IndexOpsMixin) and isinstance(other.spark.data_type, StringType)) + or isinstance(other, str) + ): + raise TypeError("division can not be applied on string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("division can not be applied to date times.") + + def floordiv(left, right): + return F.when(F.lit(right is np.nan), np.nan).otherwise( + F.when( + F.lit(right != 0) | F.lit(right).isNull(), F.floor(left.__div__(right)) + ).otherwise( + F.when(F.lit(left == np.inf) | F.lit(left == -np.inf), left).otherwise( + F.lit(np.inf).__div__(left) + ) + ) + ) + + return numpy_column_op(floordiv)(self, other) + + def __rfloordiv__(self, other) -> Union["Series", "Index"]: + if isinstance(self.spark.data_type, StringType) or isinstance(other, str): + raise TypeError("division can not be applied on string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("division can not be applied to date times.") + + def rfloordiv(left, right): + return F.when(F.lit(left == 0), F.lit(np.inf).__div__(right)).otherwise( + F.when(F.lit(left) == np.nan, np.nan).otherwise(F.floor(F.lit(right).__div__(left))) + ) + + return numpy_column_op(rfloordiv)(self, other) + + def __rmod__(self, other) -> Union["Series", "Index"]: + if isinstance(self.spark.data_type, StringType) or isinstance(other, str): + raise TypeError("modulo can not be applied on string series or literals.") + + if isinstance(self.spark.data_type, TimestampType): + raise TypeError("modulo can not be applied to date times.") + + def rmod(left, right): + return ((right % left) + left) % left + + return column_op(rmod)(self, other) + + def __pow__(self, other) -> Union["Series", "Index"]: + def pow_func(left, right): + return F.when(left == 1, left).otherwise(Column.__pow__(left, right)) + + return column_op(pow_func)(self, other) + + def __rpow__(self, other) -> Union["Series", "Index"]: + def rpow_func(left, right): + return F.when(F.lit(right == 1), right).otherwise(Column.__rpow__(left, right)) + + return column_op(rpow_func)(self, other) + + __abs__ = column_op(F.abs) + + # comparison operators + __eq__ = column_op(Column.__eq__) + __ne__ = column_op(Column.__ne__) + __lt__ = column_op(Column.__lt__) + __le__ = column_op(Column.__le__) + __ge__ = column_op(Column.__ge__) + __gt__ = column_op(Column.__gt__) + + # `and`, `or`, `not` cannot be overloaded in Python, + # so use bitwise operators as boolean operators + def __and__(self, other) -> Union["Series", "Index"]: + if isinstance(self.dtype, extension_dtypes) or ( + isinstance(other, IndexOpsMixin) and isinstance(other.dtype, extension_dtypes) + ): + + def and_func(left, right): + if not isinstance(right, spark.Column): + if pd.isna(right): + right = F.lit(None) + else: + right = F.lit(right) + return left & right + + else: + + def and_func(left, right): + if not isinstance(right, spark.Column): + if pd.isna(right): + right = F.lit(None) + else: + right = F.lit(right) + scol = left & right + return F.when(scol.isNull(), False).otherwise(scol) + + return column_op(and_func)(self, other) + + def __or__(self, other) -> Union["Series", "Index"]: + if isinstance(self.dtype, extension_dtypes) or ( + isinstance(other, IndexOpsMixin) and isinstance(other.dtype, extension_dtypes) + ): + + def or_func(left, right): + if not isinstance(right, spark.Column): + if pd.isna(right): + right = F.lit(None) + else: + right = F.lit(right) + return left | right + + else: + + def or_func(left, right): + if not isinstance(right, spark.Column) and pd.isna(right): + return F.lit(False) + else: + scol = left | F.lit(right) + return F.when(left.isNull() | scol.isNull(), False).otherwise(scol) + + return column_op(or_func)(self, other) + + __invert__ = column_op(Column.__invert__) + + def __rand__(self, other) -> Union["Series", "Index"]: + return self.__and__(other) + + def __ror__(self, other) -> Union["Series", "Index"]: + return self.__or__(other) + + def __len__(self): + return len(self._kdf) + + # NDArray Compat + def __array_ufunc__(self, ufunc: Callable, method: str, *inputs: Any, **kwargs: Any): + # Try dunder methods first. + result = numpy_compat.maybe_dispatch_ufunc_to_dunder_op( + self, ufunc, method, *inputs, **kwargs + ) + + # After that, we try with PySpark APIs. + if result is NotImplemented: + result = numpy_compat.maybe_dispatch_ufunc_to_spark_func( + self, ufunc, method, *inputs, **kwargs + ) + + if result is not NotImplemented: + return result + else: + # TODO: support more APIs? + raise NotImplementedError("Koalas objects currently do not support %s." % ufunc) + + @property + def dtype(self) -> Dtype: + """Return the dtype object of the underlying data. + + Examples + -------- + >>> s = pp.Series([1, 2, 3]) + >>> s.dtype + dtype('int64') + + >>> s = pp.Series(list('abc')) + >>> s.dtype + dtype('O') + + >>> s = pp.Series(pd.date_range('20130101', periods=3)) + >>> s.dtype + dtype('>> s.rename("a").to_frame().set_index("a").index.dtype + dtype(' bool: + """ + Returns true if the current object is empty. Otherwise, returns false. + + >>> pp.range(10).id.empty + False + + >>> pp.range(0).id.empty + True + + >>> pp.DataFrame({}, index=list('abc')).index.empty + False + """ + return self._internal.resolved_copy.spark_frame.rdd.isEmpty() + + @property + def hasnans(self) -> bool: + """ + Return True if it has any missing values. Otherwise, it returns False. + + >>> pp.DataFrame({}, index=list('abc')).index.hasnans + False + + >>> pp.Series(['a', None]).hasnans + True + + >>> pp.Series([1.0, 2.0, np.nan]).hasnans + True + + >>> pp.Series([1, 2, 3]).hasnans + False + + >>> (pp.Series([1.0, 2.0, np.nan]) + 1).hasnans + True + + >>> pp.Series([1, 2, 3]).rename("a").to_frame().set_index("a").index.hasnans + False + """ + sdf = self._internal.spark_frame + scol = self.spark.column + + if isinstance(self.spark.data_type, (DoubleType, FloatType)): + return sdf.select(F.max(scol.isNull() | F.isnan(scol))).collect()[0][0] + else: + return sdf.select(F.max(scol.isNull())).collect()[0][0] + + @property + def is_monotonic(self) -> bool: + """ + Return boolean if values in the object are monotonically increasing. + + .. note:: the current implementation of is_monotonic requires to shuffle + and aggregate multiple times to check the order locally and globally, + which is potentially expensive. In case of multi-index, all data are + transferred to single node which can easily cause out-of-memory error currently. + + .. note:: Disable the Spark config `spark.sql.optimizer.nestedSchemaPruning.enabled` + for multi-index if you're using Koalas < 1.7.0 with PySpark 3.1.1. + + Returns + ------- + is_monotonic : bool + + Examples + -------- + >>> ser = pp.Series(['1/1/2018', '3/1/2018', '4/1/2018']) + >>> ser.is_monotonic + True + + >>> df = pp.DataFrame({'dates': [None, '1/1/2018', '2/1/2018', '3/1/2018']}) + >>> df.dates.is_monotonic + False + + >>> df.index.is_monotonic + True + + >>> ser = pp.Series([1]) + >>> ser.is_monotonic + True + + >>> ser = pp.Series([]) + >>> ser.is_monotonic + True + + >>> ser.rename("a").to_frame().set_index("a").index.is_monotonic + True + + >>> ser = pp.Series([5, 4, 3, 2, 1], index=[1, 2, 3, 4, 5]) + >>> ser.is_monotonic + False + + >>> ser.index.is_monotonic + True + + Support for MultiIndex + + >>> midx = pp.MultiIndex.from_tuples( + ... [('x', 'a'), ('x', 'b'), ('y', 'c'), ('y', 'd'), ('z', 'e')]) + >>> midx # doctest: +SKIP + MultiIndex([('x', 'a'), + ('x', 'b'), + ('y', 'c'), + ('y', 'd'), + ('z', 'e')], + ) + >>> midx.is_monotonic + True + + >>> midx = pp.MultiIndex.from_tuples( + ... [('z', 'a'), ('z', 'b'), ('y', 'c'), ('y', 'd'), ('x', 'e')]) + >>> midx # doctest: +SKIP + MultiIndex([('z', 'a'), + ('z', 'b'), + ('y', 'c'), + ('y', 'd'), + ('x', 'e')], + ) + >>> midx.is_monotonic + False + """ + return self._is_monotonic("increasing") + + is_monotonic_increasing = is_monotonic + + @property + def is_monotonic_decreasing(self) -> bool: + """ + Return boolean if values in the object are monotonically decreasing. + + .. note:: the current implementation of is_monotonic_decreasing requires to shuffle + and aggregate multiple times to check the order locally and globally, + which is potentially expensive. In case of multi-index, all data are transferred + to single node which can easily cause out-of-memory error currently. + + .. note:: Disable the Spark config `spark.sql.optimizer.nestedSchemaPruning.enabled` + for multi-index if you're using Koalas < 1.7.0 with PySpark 3.1.1. + + Returns + ------- + is_monotonic : bool + + Examples + -------- + >>> ser = pp.Series(['4/1/2018', '3/1/2018', '1/1/2018']) + >>> ser.is_monotonic_decreasing + True + + >>> df = pp.DataFrame({'dates': [None, '3/1/2018', '2/1/2018', '1/1/2018']}) + >>> df.dates.is_monotonic_decreasing + False + + >>> df.index.is_monotonic_decreasing + False + + >>> ser = pp.Series([1]) + >>> ser.is_monotonic_decreasing + True + + >>> ser = pp.Series([]) + >>> ser.is_monotonic_decreasing + True + + >>> ser.rename("a").to_frame().set_index("a").index.is_monotonic_decreasing + True + + >>> ser = pp.Series([5, 4, 3, 2, 1], index=[1, 2, 3, 4, 5]) + >>> ser.is_monotonic_decreasing + True + + >>> ser.index.is_monotonic_decreasing + False + + Support for MultiIndex + + >>> midx = pp.MultiIndex.from_tuples( + ... [('x', 'a'), ('x', 'b'), ('y', 'c'), ('y', 'd'), ('z', 'e')]) + >>> midx # doctest: +SKIP + MultiIndex([('x', 'a'), + ('x', 'b'), + ('y', 'c'), + ('y', 'd'), + ('z', 'e')], + ) + >>> midx.is_monotonic_decreasing + False + + >>> midx = pp.MultiIndex.from_tuples( + ... [('z', 'e'), ('z', 'd'), ('y', 'c'), ('y', 'b'), ('x', 'a')]) + >>> midx # doctest: +SKIP + MultiIndex([('z', 'a'), + ('z', 'b'), + ('y', 'c'), + ('y', 'd'), + ('x', 'e')], + ) + >>> midx.is_monotonic_decreasing + True + """ + return self._is_monotonic("decreasing") + + def _is_locally_monotonic_spark_column(self, order): + window = ( + Window.partitionBy(F.col("__partition_id")) + .orderBy(NATURAL_ORDER_COLUMN_NAME) + .rowsBetween(-1, -1) + ) + + if order == "increasing": + return (F.col("__origin") >= F.lag(F.col("__origin"), 1).over(window)) & F.col( + "__origin" + ).isNotNull() + else: + return (F.col("__origin") <= F.lag(F.col("__origin"), 1).over(window)) & F.col( + "__origin" + ).isNotNull() + + def _is_monotonic(self, order): + assert order in ("increasing", "decreasing") + + sdf = self._internal.spark_frame + + sdf = ( + sdf.select( + F.spark_partition_id().alias( + "__partition_id" + ), # Make sure we use the same partition id in the whole job. + F.col(NATURAL_ORDER_COLUMN_NAME), + self.spark.column.alias("__origin"), + ) + .select( + F.col("__partition_id"), + F.col("__origin"), + self._is_locally_monotonic_spark_column(order).alias( + "__comparison_within_partition" + ), + ) + .groupby(F.col("__partition_id")) + .agg( + F.min(F.col("__origin")).alias("__partition_min"), + F.max(F.col("__origin")).alias("__partition_max"), + F.min(F.coalesce(F.col("__comparison_within_partition"), F.lit(True))).alias( + "__comparison_within_partition" + ), + ) + ) + + # Now we're windowing the aggregation results without partition specification. + # The number of rows here will be as the same of partitions, which is expected + # to be small. + window = Window.orderBy(F.col("__partition_id")).rowsBetween(-1, -1) + if order == "increasing": + comparison_col = F.col("__partition_min") >= F.lag(F.col("__partition_max"), 1).over( + window + ) + else: + comparison_col = F.col("__partition_min") <= F.lag(F.col("__partition_max"), 1).over( + window + ) + + sdf = sdf.select( + comparison_col.alias("__comparison_between_partitions"), + F.col("__comparison_within_partition"), + ) + + ret = sdf.select( + F.min(F.coalesce(F.col("__comparison_between_partitions"), F.lit(True))) + & F.min(F.coalesce(F.col("__comparison_within_partition"), F.lit(True))) + ).collect()[0][0] + if ret is None: + return True + else: + return ret + + @property + def ndim(self) -> int: + """ + Return an int representing the number of array dimensions. + + Return 1 for Series / Index / MultiIndex. + + Examples + -------- + + For Series + + >>> s = pp.Series([None, 1, 2, 3, 4], index=[4, 5, 2, 1, 8]) + >>> s.ndim + 1 + + For Index + + >>> s.index.ndim + 1 + + For MultiIndex + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [1, 1, 1, 1, 1, 2, 1, 2, 2]]) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], index=midx) + >>> s.index.ndim + 1 + """ + return 1 + + def astype(self, dtype: Union[str, type, Dtype]) -> Union["Index", "Series"]: + """ + Cast a Koalas object to a specified dtype ``dtype``. + + Parameters + ---------- + dtype : data type + Use a numpy.dtype or Python type to cast entire pandas object to + the same type. + + Returns + ------- + casted : same type as caller + + See Also + -------- + to_datetime : Convert argument to datetime. + + Examples + -------- + >>> ser = pp.Series([1, 2], dtype='int32') + >>> ser + 0 1 + 1 2 + dtype: int32 + + >>> ser.astype('int64') + 0 1 + 1 2 + dtype: int64 + + >>> ser.rename("a").to_frame().set_index("a").index.astype('int64') + Int64Index([1, 2], dtype='int64', name='a') + """ + dtype, spark_type = koalas_dtype(dtype) + if not spark_type: + raise ValueError("Type {} not understood".format(dtype)) + + if isinstance(self.dtype, CategoricalDtype): + if isinstance(dtype, CategoricalDtype) and dtype.categories is None: + return cast(Union[pp.Index, pp.Series], self).copy() + + categories = self.dtype.categories + if len(categories) == 0: + scol = F.lit(None) + else: + kvs = list( + chain( + *[ + (F.lit(code), F.lit(category)) + for code, category in enumerate(categories) + ] + ) + ) + map_scol = F.create_map(kvs) + scol = map_scol.getItem(self.spark.column) + return self._with_new_scol( + scol.alias(self._internal.data_spark_column_names[0]) + ).astype(dtype) + elif isinstance(dtype, CategoricalDtype): + if dtype.categories is None: + codes, uniques = self.factorize() + return codes._with_new_scol( + codes.spark.column, dtype=CategoricalDtype(categories=uniques) + ) + else: + categories = dtype.categories + if len(categories) == 0: + scol = F.lit(-1) + else: + kvs = list( + chain( + *[ + (F.lit(category), F.lit(code)) + for code, category in enumerate(categories) + ] + ) + ) + map_scol = F.create_map(kvs) + + scol = F.coalesce(map_scol.getItem(self.spark.column), F.lit(-1)) + return self._with_new_scol( + scol.alias(self._internal.data_spark_column_names[0]), dtype=dtype + ) + + if isinstance(spark_type, BooleanType): + if isinstance(dtype, extension_dtypes): + scol = self.spark.column.cast(spark_type) + else: + if isinstance(self.spark.data_type, StringType): + scol = F.when(self.spark.column.isNull(), F.lit(False)).otherwise( + F.length(self.spark.column) > 0 + ) + elif isinstance(self.spark.data_type, (FloatType, DoubleType)): + scol = F.when( + self.spark.column.isNull() | F.isnan(self.spark.column), F.lit(True) + ).otherwise(self.spark.column.cast(spark_type)) + else: + scol = F.when(self.spark.column.isNull(), F.lit(False)).otherwise( + self.spark.column.cast(spark_type) + ) + elif isinstance(spark_type, StringType): + if isinstance(dtype, extension_dtypes): + if isinstance(self.spark.data_type, BooleanType): + scol = F.when( + self.spark.column.isNotNull(), + F.when(self.spark.column, "True").otherwise("False"), + ) + elif isinstance(self.spark.data_type, TimestampType): + # seems like a pandas' bug? + scol = F.when(self.spark.column.isNull(), str(pd.NaT)).otherwise( + self.spark.column.cast(spark_type) + ) + else: + scol = self.spark.column.cast(spark_type) + else: + if isinstance(self.spark.data_type, NumericType): + null_str = str(np.nan) + elif isinstance(self.spark.data_type, (DateType, TimestampType)): + null_str = str(pd.NaT) + else: + null_str = str(None) + if isinstance(self.spark.data_type, BooleanType): + casted = F.when(self.spark.column, "True").otherwise("False") + else: + casted = self.spark.column.cast(spark_type) + scol = F.when(self.spark.column.isNull(), null_str).otherwise(casted) + else: + scol = self.spark.column.cast(spark_type) + return self._with_new_scol( + scol.alias(self._internal.data_spark_column_names[0]), dtype=dtype + ) + + def isin(self, values) -> Union["Series", "Index"]: + """ + Check whether `values` are contained in Series or Index. + + Return a boolean Series or Index showing whether each element in the Series + matches an element in the passed sequence of `values` exactly. + + Parameters + ---------- + values : set or list-like + The sequence of values to test. + + Returns + ------- + isin : Series (bool dtype) or Index (bool dtype) + + Examples + -------- + >>> s = pp.Series(['lama', 'cow', 'lama', 'beetle', 'lama', + ... 'hippo'], name='animal') + >>> s.isin(['cow', 'lama']) + 0 True + 1 True + 2 True + 3 False + 4 True + 5 False + Name: animal, dtype: bool + + Passing a single string as ``s.isin('lama')`` will raise an error. Use + a list of one element instead: + + >>> s.isin(['lama']) + 0 True + 1 False + 2 True + 3 False + 4 True + 5 False + Name: animal, dtype: bool + + >>> s.rename("a").to_frame().set_index("a").index.isin(['lama']) + Index([True, False, True, False, True, False], dtype='object', name='a') + """ + if not is_list_like(values): + raise TypeError( + "only list-like objects are allowed to be passed" + " to isin(), you passed a [{values_type}]".format(values_type=type(values).__name__) + ) + + values = values.tolist() if isinstance(values, np.ndarray) else list(values) + return self._with_new_scol(self.spark.column.isin(values)) + + def isnull(self) -> Union["Series", "Index"]: + """ + Detect existing (non-missing) values. + + Return a boolean same-sized object indicating if the values are NA. + NA values, such as None or numpy.NaN, gets mapped to True values. + Everything else gets mapped to False values. Characters such as empty strings '' or + numpy.inf are not considered NA values + (unless you set pandas.options.mode.use_inf_as_na = True). + + Returns + ------- + Series or Index : Mask of bool values for each element in Series + that indicates whether an element is not an NA value. + + Examples + -------- + >>> ser = pp.Series([5, 6, np.NaN]) + >>> ser.isna() # doctest: +NORMALIZE_WHITESPACE + 0 False + 1 False + 2 True + dtype: bool + + >>> ser.rename("a").to_frame().set_index("a").index.isna() + Index([False, False, True], dtype='object', name='a') + """ + from pyspark.pandas.indexes import MultiIndex + + if isinstance(self, MultiIndex): + raise NotImplementedError("isna is not defined for MultiIndex") + if isinstance(self.spark.data_type, (FloatType, DoubleType)): + return self._with_new_scol(self.spark.column.isNull() | F.isnan(self.spark.column)) + else: + return self._with_new_scol(self.spark.column.isNull()) + + isna = isnull + + def notnull(self) -> Union["Series", "Index"]: + """ + Detect existing (non-missing) values. + Return a boolean same-sized object indicating if the values are not NA. + Non-missing values get mapped to True. + Characters such as empty strings '' or numpy.inf are not considered NA values + (unless you set pandas.options.mode.use_inf_as_na = True). + NA values, such as None or numpy.NaN, get mapped to False values. + + Returns + ------- + Series or Index : Mask of bool values for each element in Series + that indicates whether an element is not an NA value. + + Examples + -------- + Show which entries in a Series are not NA. + + >>> ser = pp.Series([5, 6, np.NaN]) + >>> ser + 0 5.0 + 1 6.0 + 2 NaN + dtype: float64 + + >>> ser.notna() + 0 True + 1 True + 2 False + dtype: bool + + >>> ser.rename("a").to_frame().set_index("a").index.notna() + Index([True, True, False], dtype='object', name='a') + """ + from pyspark.pandas.indexes import MultiIndex + + if isinstance(self, MultiIndex): + raise NotImplementedError("notna is not defined for MultiIndex") + return (~self.isnull()).rename( + self.name # type: ignore + ) + + notna = notnull + + # TODO: axis, skipna, and many arguments should be implemented. + def all(self, axis: Union[int, str] = 0) -> bool: + """ + Return whether all elements are True. + + Returns True unless there at least one element within a series that is + False or equivalent (e.g. zero or empty) + + Parameters + ---------- + axis : {0 or 'index'}, default 0 + Indicate which axis or axes should be reduced. + + * 0 / 'index' : reduce the index, return a Series whose index is the + original column labels. + + Examples + -------- + >>> pp.Series([True, True]).all() + True + + >>> pp.Series([True, False]).all() + False + + >>> pp.Series([0, 1]).all() + False + + >>> pp.Series([1, 2, 3]).all() + True + + >>> pp.Series([True, True, None]).all() + True + + >>> pp.Series([True, False, None]).all() + False + + >>> pp.Series([]).all() + True + + >>> pp.Series([np.nan]).all() + True + + >>> df = pp.Series([True, False, None]).rename("a").to_frame() + >>> df.set_index("a").index.all() + False + """ + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + sdf = self._internal.spark_frame.select(self.spark.column) + col = scol_for(sdf, sdf.columns[0]) + + # Note that we're ignoring `None`s here for now. + # any and every was added as of Spark 3.0 + # ret = sdf.select(F.expr("every(CAST(`%s` AS BOOLEAN))" % sdf.columns[0])).collect()[0][0] + # Here we use min as its alternative: + ret = sdf.select(F.min(F.coalesce(col.cast("boolean"), F.lit(True)))).collect()[0][0] + if ret is None: + return True + else: + return ret + + # TODO: axis, skipna, and many arguments should be implemented. + def any(self, axis: Union[int, str] = 0) -> bool: + """ + Return whether any element is True. + + Returns False unless there at least one element within a series that is + True or equivalent (e.g. non-zero or non-empty). + + Parameters + ---------- + axis : {0 or 'index'}, default 0 + Indicate which axis or axes should be reduced. + + * 0 / 'index' : reduce the index, return a Series whose index is the + original column labels. + + Examples + -------- + >>> pp.Series([False, False]).any() + False + + >>> pp.Series([True, False]).any() + True + + >>> pp.Series([0, 0]).any() + False + + >>> pp.Series([0, 1, 2]).any() + True + + >>> pp.Series([False, False, None]).any() + False + + >>> pp.Series([True, False, None]).any() + True + + >>> pp.Series([]).any() + False + + >>> pp.Series([np.nan]).any() + False + + >>> df = pp.Series([True, False, None]).rename("a").to_frame() + >>> df.set_index("a").index.any() + True + """ + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + sdf = self._internal.spark_frame.select(self.spark.column) + col = scol_for(sdf, sdf.columns[0]) + + # Note that we're ignoring `None`s here for now. + # any and every was added as of Spark 3.0 + # ret = sdf.select(F.expr("any(CAST(`%s` AS BOOLEAN))" % sdf.columns[0])).collect()[0][0] + # Here we use max as its alternative: + ret = sdf.select(F.max(F.coalesce(col.cast("boolean"), F.lit(False)))).collect()[0][0] + if ret is None: + return False + else: + return ret + + # TODO: add frep and axis parameter + def shift(self, periods=1, fill_value=None) -> Union["Series", "Index"]: + """ + Shift Series/Index by desired number of periods. + + .. note:: the current implementation of shift uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + periods : int + Number of periods to shift. Can be positive or negative. + fill_value : object, optional + The scalar value to use for newly introduced missing values. + The default depends on the dtype of self. For numeric data, np.nan is used. + + Returns + ------- + Copy of input Series/Index, shifted. + + Examples + -------- + >>> df = pp.DataFrame({'Col1': [10, 20, 15, 30, 45], + ... 'Col2': [13, 23, 18, 33, 48], + ... 'Col3': [17, 27, 22, 37, 52]}, + ... columns=['Col1', 'Col2', 'Col3']) + + >>> df.Col1.shift(periods=3) + 0 NaN + 1 NaN + 2 NaN + 3 10.0 + 4 20.0 + Name: Col1, dtype: float64 + + >>> df.Col2.shift(periods=3, fill_value=0) + 0 0 + 1 0 + 2 0 + 3 13 + 4 23 + Name: Col2, dtype: int64 + + >>> df.index.shift(periods=3, fill_value=0) + Int64Index([0, 0, 0, 0, 1], dtype='int64') + """ + return self._shift(periods, fill_value).spark.analyzed + + def _shift(self, periods, fill_value, *, part_cols=()): + if not isinstance(periods, int): + raise ValueError("periods should be an int; however, got [%s]" % type(periods).__name__) + + col = self.spark.column + window = ( + Window.partitionBy(*part_cols) + .orderBy(NATURAL_ORDER_COLUMN_NAME) + .rowsBetween(-periods, -periods) + ) + lag_col = F.lag(col, periods).over(window) + col = F.when(lag_col.isNull() | F.isnan(lag_col), fill_value).otherwise(lag_col) + return self._with_new_scol(col, dtype=self.dtype) + + # TODO: Update Documentation for Bins Parameter when its supported + def value_counts( + self, normalize=False, sort=True, ascending=False, bins=None, dropna=True + ) -> "Series": + """ + Return a Series containing counts of unique values. + The resulting object will be in descending order so that the + first element is the most frequently-occurring element. + Excludes NA values by default. + + Parameters + ---------- + normalize : boolean, default False + If True then the object returned will contain the relative + frequencies of the unique values. + sort : boolean, default True + Sort by values. + ascending : boolean, default False + Sort in ascending order. + bins : Not Yet Supported + dropna : boolean, default True + Don't include counts of NaN. + + Returns + ------- + counts : Series + + See Also + -------- + Series.count: Number of non-NA elements in a Series. + + Examples + -------- + For Series + + >>> df = pp.DataFrame({'x':[0, 0, 1, 1, 1, np.nan]}) + >>> df.x.value_counts() # doctest: +NORMALIZE_WHITESPACE + 1.0 3 + 0.0 2 + Name: x, dtype: int64 + + With `normalize` set to `True`, returns the relative frequency by + dividing all values by the sum of values. + + >>> df.x.value_counts(normalize=True) # doctest: +NORMALIZE_WHITESPACE + 1.0 0.6 + 0.0 0.4 + Name: x, dtype: float64 + + **dropna** + With `dropna` set to `False` we can also see NaN index values. + + >>> df.x.value_counts(dropna=False) # doctest: +NORMALIZE_WHITESPACE + 1.0 3 + 0.0 2 + NaN 1 + Name: x, dtype: int64 + + For Index + + >>> idx = pp.Index([3, 1, 2, 3, 4, np.nan]) + >>> idx + Float64Index([3.0, 1.0, 2.0, 3.0, 4.0, nan], dtype='float64') + + >>> idx.value_counts().sort_index() + 1.0 1 + 2.0 1 + 3.0 2 + 4.0 1 + dtype: int64 + + **sort** + + With `sort` set to `False`, the result wouldn't be sorted by number of count. + + >>> idx.value_counts(sort=True).sort_index() + 1.0 1 + 2.0 1 + 3.0 2 + 4.0 1 + dtype: int64 + + **normalize** + + With `normalize` set to `True`, returns the relative frequency by + dividing all values by the sum of values. + + >>> idx.value_counts(normalize=True).sort_index() + 1.0 0.2 + 2.0 0.2 + 3.0 0.4 + 4.0 0.2 + dtype: float64 + + **dropna** + + With `dropna` set to `False` we can also see NaN index values. + + >>> idx.value_counts(dropna=False).sort_index() # doctest: +SKIP + 1.0 1 + 2.0 1 + 3.0 2 + 4.0 1 + NaN 1 + dtype: int64 + + For MultiIndex. + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [1, 1, 1, 1, 1, 2, 1, 2, 2]]) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], index=midx) + >>> s.index # doctest: +SKIP + MultiIndex([( 'lama', 'weight'), + ( 'lama', 'weight'), + ( 'lama', 'weight'), + ( 'cow', 'weight'), + ( 'cow', 'weight'), + ( 'cow', 'length'), + ('falcon', 'weight'), + ('falcon', 'length'), + ('falcon', 'length')], + ) + + >>> s.index.value_counts().sort_index() + (cow, length) 1 + (cow, weight) 2 + (falcon, length) 2 + (falcon, weight) 1 + (lama, weight) 3 + dtype: int64 + + >>> s.index.value_counts(normalize=True).sort_index() + (cow, length) 0.111111 + (cow, weight) 0.222222 + (falcon, length) 0.222222 + (falcon, weight) 0.111111 + (lama, weight) 0.333333 + dtype: float64 + + If Index has name, keep the name up. + + >>> idx = pp.Index([0, 0, 0, 1, 1, 2, 3], name='koalas') + >>> idx.value_counts().sort_index() + 0 3 + 1 2 + 2 1 + 3 1 + Name: koalas, dtype: int64 + """ + from pyspark.pandas.series import first_series + + if bins is not None: + raise NotImplementedError("value_counts currently does not support bins") + + if dropna: + sdf_dropna = self._internal.spark_frame.select(self.spark.column).dropna() + else: + sdf_dropna = self._internal.spark_frame.select(self.spark.column) + index_name = SPARK_DEFAULT_INDEX_NAME + column_name = self._internal.data_spark_column_names[0] + sdf = sdf_dropna.groupby(scol_for(sdf_dropna, column_name).alias(index_name)).count() + if sort: + if ascending: + sdf = sdf.orderBy(F.col("count")) + else: + sdf = sdf.orderBy(F.col("count").desc()) + + if normalize: + sum = sdf_dropna.count() + sdf = sdf.withColumn("count", F.col("count") / F.lit(sum)) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, index_name)], + column_labels=self._internal.column_labels, + data_spark_columns=[scol_for(sdf, "count")], + column_label_names=self._internal.column_label_names, + ) + + return first_series(DataFrame(internal)) + + def nunique(self, dropna: bool = True, approx: bool = False, rsd: float = 0.05) -> int: + """ + Return number of unique elements in the object. + Excludes NA values by default. + + Parameters + ---------- + dropna : bool, default True + Don’t include NaN in the count. + approx: bool, default False + If False, will use the exact algorithm and return the exact number of unique. + If True, it uses the HyperLogLog approximate algorithm, which is significantly faster + for large amount of data. + Note: This parameter is specific to Koalas and is not found in pandas. + rsd: float, default 0.05 + Maximum estimation error allowed in the HyperLogLog algorithm. + Note: Just like ``approx`` this parameter is specific to Koalas. + + Returns + ------- + int + + See Also + -------- + DataFrame.nunique: Method nunique for DataFrame. + Series.count: Count non-NA/null observations in the Series. + + Examples + -------- + >>> pp.Series([1, 2, 3, np.nan]).nunique() + 3 + + >>> pp.Series([1, 2, 3, np.nan]).nunique(dropna=False) + 4 + + On big data, we recommend using the approximate algorithm to speed up this function. + The result will be very close to the exact unique count. + + >>> pp.Series([1, 2, 3, np.nan]).nunique(approx=True) + 3 + + >>> idx = pp.Index([1, 1, 2, None]) + >>> idx + Float64Index([1.0, 1.0, 2.0, nan], dtype='float64') + + >>> idx.nunique() + 2 + + >>> idx.nunique(dropna=False) + 3 + """ + res = self._internal.spark_frame.select([self._nunique(dropna, approx, rsd)]) + return res.collect()[0][0] + + def _nunique(self, dropna=True, approx=False, rsd=0.05): + colname = self._internal.data_spark_column_names[0] + count_fn = partial(F.approx_count_distinct, rsd=rsd) if approx else F.countDistinct + if dropna: + return count_fn(self.spark.column).alias(colname) + else: + return ( + count_fn(self.spark.column) + + F.when( + F.count(F.when(self.spark.column.isNull(), 1).otherwise(None)) >= 1, 1 + ).otherwise(0) + ).alias(colname) + + def take(self, indices) -> Union["Series", "Index"]: + """ + Return the elements in the given *positional* indices along an axis. + + This means that we are not indexing according to actual values in + the index attribute of the object. We are indexing according to the + actual position of the element in the object. + + Parameters + ---------- + indices : array-like + An array of ints indicating which positions to take. + + Returns + ------- + taken : same type as caller + An array-like containing the elements taken from the object. + + See Also + -------- + DataFrame.loc : Select a subset of a DataFrame by labels. + DataFrame.iloc : Select a subset of a DataFrame by positions. + numpy.take : Take elements from an array along an axis. + + Examples + -------- + + Series + + >>> kser = pp.Series([100, 200, 300, 400, 500]) + >>> kser + 0 100 + 1 200 + 2 300 + 3 400 + 4 500 + dtype: int64 + + >>> kser.take([0, 2, 4]).sort_index() + 0 100 + 2 300 + 4 500 + dtype: int64 + + Index + + >>> kidx = pp.Index([100, 200, 300, 400, 500]) + >>> kidx + Int64Index([100, 200, 300, 400, 500], dtype='int64') + + >>> kidx.take([0, 2, 4]).sort_values() + Int64Index([100, 300, 500], dtype='int64') + + MultiIndex + + >>> kmidx = pp.MultiIndex.from_tuples([("x", "a"), ("x", "b"), ("x", "c")]) + >>> kmidx # doctest: +SKIP + MultiIndex([('x', 'a'), + ('x', 'b'), + ('x', 'c')], + ) + + >>> kmidx.take([0, 2]) # doctest: +SKIP + MultiIndex([('x', 'a'), + ('x', 'c')], + ) + """ + if not is_list_like(indices) or isinstance(indices, (dict, set)): + raise ValueError("`indices` must be a list-like except dict or set") + if isinstance(self, pp.Series): + return cast(pp.Series, self.iloc[indices]) + else: + return self._kdf.iloc[indices].index + + def factorize( + self, sort: bool = True, na_sentinel: Optional[int] = -1 + ) -> Tuple[Union["Series", "Index"], pd.Index]: + """ + Encode the object as an enumerated type or categorical variable. + + This method is useful for obtaining a numeric representation of an + array when all that matters is identifying distinct values. + + Parameters + ---------- + sort : bool, default True + na_sentinel : int or None, default -1 + Value to mark "not found". If None, will not drop the NaN + from the uniques of the values. + + Returns + ------- + codes : Series or Index + A Series or Index that's an indexer into `uniques`. + ``uniques.take(codes)`` will have the same values as `values`. + uniques : pd.Index + The unique valid values. + + .. note :: + + Even if there's a missing value in `values`, `uniques` will + *not* contain an entry for it. + + Examples + -------- + >>> kser = pp.Series(['b', None, 'a', 'c', 'b']) + >>> codes, uniques = kser.factorize() + >>> codes + 0 1 + 1 -1 + 2 0 + 3 2 + 4 1 + dtype: int32 + >>> uniques + Index(['a', 'b', 'c'], dtype='object') + + >>> codes, uniques = kser.factorize(na_sentinel=None) + >>> codes + 0 1 + 1 3 + 2 0 + 3 2 + 4 1 + dtype: int32 + >>> uniques + Index(['a', 'b', 'c', None], dtype='object') + + >>> codes, uniques = kser.factorize(na_sentinel=-2) + >>> codes + 0 1 + 1 -2 + 2 0 + 3 2 + 4 1 + dtype: int32 + >>> uniques + Index(['a', 'b', 'c'], dtype='object') + + For Index: + + >>> kidx = pp.Index(['b', None, 'a', 'c', 'b']) + >>> codes, uniques = kidx.factorize() + >>> codes + Int64Index([1, -1, 0, 2, 1], dtype='int64') + >>> uniques + Index(['a', 'b', 'c'], dtype='object') + """ + from pyspark.pandas.series import first_series + + assert (na_sentinel is None) or isinstance(na_sentinel, int) + assert sort is True + + if isinstance(self.dtype, CategoricalDtype): + categories = self.dtype.categories + if len(categories) == 0: + scol = F.lit(None) + else: + kvs = list( + chain( + *[ + (F.lit(code), F.lit(category)) + for code, category in enumerate(categories) + ] + ) + ) + map_scol = F.create_map(kvs) + scol = map_scol.getItem(self.spark.column) + codes, uniques = self._with_new_scol( + scol.alias(self._internal.data_spark_column_names[0]) + ).factorize(na_sentinel=na_sentinel) + return codes, uniques.astype(self.dtype) + + uniq_sdf = self._internal.spark_frame.select(self.spark.column).distinct() + + # Check number of uniques and constructs sorted `uniques_list` + max_compute_count = get_option("compute.max_rows") + if max_compute_count is not None: + uniq_pdf = uniq_sdf.limit(max_compute_count + 1).toPandas() + if len(uniq_pdf) > max_compute_count: + raise ValueError( + "Current Series has more then {0} unique values. " + "Please set 'compute.max_rows' by using 'pyspark.pandas.config.set_option' " + "to more than {0} rows. Note that, before changing the " + "'compute.max_rows', this operation is considerably expensive.".format( + max_compute_count + ) + ) + else: + uniq_pdf = uniq_sdf.toPandas() + # pandas takes both NaN and null in Spark to np.nan, so de-duplication is required + uniq_series = first_series(uniq_pdf).drop_duplicates() + uniques_list = uniq_series.tolist() + uniques_list = sorted(uniques_list, key=lambda x: (pd.isna(x), x)) + + # Constructs `unique_to_code` mapping non-na unique to code + unique_to_code = {} + if na_sentinel is not None: + na_sentinel_code = na_sentinel + code = 0 + for unique in uniques_list: + if pd.isna(unique): + if na_sentinel is None: + na_sentinel_code = code + else: + unique_to_code[unique] = code + code += 1 + + kvs = list( + chain(*([(F.lit(unique), F.lit(code)) for unique, code in unique_to_code.items()])) + ) + + if len(kvs) == 0: # uniques are all missing values + new_scol = F.lit(na_sentinel_code) + else: + scol = self.spark.column + if isinstance(self.spark.data_type, (FloatType, DoubleType)): + cond = scol.isNull() | F.isnan(scol) + else: + cond = scol.isNull() + map_scol = F.create_map(kvs) + + null_scol = F.when(cond, F.lit(na_sentinel_code)) + new_scol = null_scol.otherwise(map_scol.getItem(scol)) + + codes = self._with_new_scol(new_scol.alias(self._internal.data_spark_column_names[0])) + + if na_sentinel is not None: + # Drops the NaN from the uniques of the values + uniques_list = [x for x in uniques_list if not pd.isna(x)] + + uniques = pd.Index(uniques_list) + + return codes, uniques diff --git a/python/pyspark/pandas/categorical.py b/python/pyspark/pandas/categorical.py new file mode 100644 index 0000000000..87d56d0deb --- /dev/null +++ b/python/pyspark/pandas/categorical.py @@ -0,0 +1,164 @@ +# +# 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. +# +from typing import TYPE_CHECKING + +import pandas as pd +from pandas.api.types import CategoricalDtype + +if TYPE_CHECKING: + import pyspark.pandas as pp # noqa: F401 (SPARK-34943) + + +class CategoricalAccessor(object): + """ + Accessor object for categorical properties of the Series values. + + Examples + -------- + >>> s = pp.Series(list("abbccc"), dtype="category") + >>> s # doctest: +SKIP + 0 a + 1 b + 2 b + 3 c + 4 c + 5 c + dtype: category + Categories (3, object): ['a', 'b', 'c'] + + >>> s.cat.categories + Index(['a', 'b', 'c'], dtype='object') + + >>> s.cat.codes + 0 0 + 1 1 + 2 1 + 3 2 + 4 2 + 5 2 + dtype: int8 + """ + + def __init__(self, series: "pp.Series"): + if not isinstance(series.dtype, CategoricalDtype): + raise ValueError("Cannot call CategoricalAccessor on type {}".format(series.dtype)) + self._data = series + + @property + def categories(self) -> pd.Index: + """ + The categories of this categorical. + + Examples + -------- + >>> s = pp.Series(list("abbccc"), dtype="category") + >>> s # doctest: +SKIP + 0 a + 1 b + 2 b + 3 c + 4 c + 5 c + dtype: category + Categories (3, object): ['a', 'b', 'c'] + + >>> s.cat.categories + Index(['a', 'b', 'c'], dtype='object') + """ + return self._data.dtype.categories + + @categories.setter + def categories(self, categories) -> None: + raise NotImplementedError() + + @property + def ordered(self) -> bool: + """ + Whether the categories have an ordered relationship. + + Examples + -------- + >>> s = pp.Series(list("abbccc"), dtype="category") + >>> s # doctest: +SKIP + 0 a + 1 b + 2 b + 3 c + 4 c + 5 c + dtype: category + Categories (3, object): ['a', 'b', 'c'] + + >>> s.cat.ordered + False + """ + return self._data.dtype.ordered + + @property + def codes(self) -> "pp.Series": + """ + Return Series of codes as well as the index. + + Examples + -------- + >>> s = pp.Series(list("abbccc"), dtype="category") + >>> s # doctest: +SKIP + 0 a + 1 b + 2 b + 3 c + 4 c + 5 c + dtype: category + Categories (3, object): ['a', 'b', 'c'] + + >>> s.cat.codes + 0 0 + 1 1 + 2 1 + 3 2 + 4 2 + 5 2 + dtype: int8 + """ + return self._data._with_new_scol(self._data.spark.column).rename() + + def add_categories(self, new_categories, inplace: bool = False): + raise NotImplementedError() + + def as_ordered(self, inplace: bool = False): + raise NotImplementedError() + + def as_unordered(self, inplace: bool = False): + raise NotImplementedError() + + def remove_categories(self, removals, inplace: bool = False): + raise NotImplementedError() + + def remove_unused_categories(self): + raise NotImplementedError() + + def rename_categories(self, new_categories, inplace: bool = False): + raise NotImplementedError() + + def reorder_categories(self, new_categories, ordered: bool = None, inplace: bool = False): + raise NotImplementedError() + + def set_categories( + self, new_categories, ordered: bool = None, rename: bool = False, inplace: bool = False + ): + raise NotImplementedError() diff --git a/python/pyspark/pandas/config.py b/python/pyspark/pandas/config.py new file mode 100644 index 0000000000..7f011a283c --- /dev/null +++ b/python/pyspark/pandas/config.py @@ -0,0 +1,442 @@ +# +# 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 ; + however, expected types are [(, )]. + + >>> 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) diff --git a/python/pyspark/pandas/datetimes.py b/python/pyspark/pandas/datetimes.py new file mode 100644 index 0000000000..a5d3f3823b --- /dev/null +++ b/python/pyspark/pandas/datetimes.py @@ -0,0 +1,850 @@ +# +# 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. +# + +""" +Date/Time related functions on Koalas Series +""" +from typing import TYPE_CHECKING + +import numpy as np # noqa: F401 (SPARK-34943) +import pandas as pd # noqa: F401 +import pyspark.sql.functions as F +from pyspark.sql.types import DateType, TimestampType, LongType + +if TYPE_CHECKING: + import pyspark.pandas as pp # noqa: F401 (SPARK-34943) + + +class DatetimeMethods(object): + """Date/Time methods for Koalas Series""" + + def __init__(self, series: "pp.Series"): + if not isinstance(series.spark.data_type, (DateType, TimestampType)): + raise ValueError( + "Cannot call DatetimeMethods on type {}".format(series.spark.data_type) + ) + self._data = series + + # Properties + @property + def date(self) -> "pp.Series": + """ + Returns a Series of python datetime.date objects (namely, the date + part of Timestamps without timezone information). + """ + # TODO: Hit a weird exception + # syntax error in attribute name: `to_date(`start_date`)` with alias + return self._data.spark.transform(F.to_date) + + @property + def time(self) -> "pp.Series": + raise NotImplementedError() + + @property + def timetz(self) -> "pp.Series": + raise NotImplementedError() + + @property + def year(self) -> "pp.Series": + """ + The year of the datetime. + """ + return self._data.spark.transform(lambda c: F.year(c).cast(LongType())) + + @property + def month(self) -> "pp.Series": + """ + The month of the timestamp as January = 1 December = 12. + """ + return self._data.spark.transform(lambda c: F.month(c).cast(LongType())) + + @property + def day(self) -> "pp.Series": + """ + The days of the datetime. + """ + return self._data.spark.transform(lambda c: F.dayofmonth(c).cast(LongType())) + + @property + def hour(self) -> "pp.Series": + """ + The hours of the datetime. + """ + return self._data.spark.transform(lambda c: F.hour(c).cast(LongType())) + + @property + def minute(self) -> "pp.Series": + """ + The minutes of the datetime. + """ + return self._data.spark.transform(lambda c: F.minute(c).cast(LongType())) + + @property + def second(self) -> "pp.Series": + """ + The seconds of the datetime. + """ + return self._data.spark.transform(lambda c: F.second(c).cast(LongType())) + + @property + def microsecond(self) -> "pp.Series": + """ + The microseconds of the datetime. + """ + + def pandas_microsecond(s) -> "pp.Series[np.int64]": + return s.dt.microsecond + + return self._data.koalas.transform_batch(pandas_microsecond) + + @property + def nanosecond(self) -> "pp.Series": + raise NotImplementedError() + + @property + def week(self) -> "pp.Series": + """ + The week ordinal of the year. + """ + return self._data.spark.transform(lambda c: F.weekofyear(c).cast(LongType())) + + @property + def weekofyear(self) -> "pp.Series": + return self.week + + weekofyear.__doc__ = week.__doc__ + + @property + def dayofweek(self) -> "pp.Series": + """ + The day of the week with Monday=0, Sunday=6. + + Return the day of the week. It is assumed the week starts on + Monday, which is denoted by 0 and ends on Sunday which is denoted + by 6. This method is available on both Series with datetime + values (using the `dt` accessor). + + Returns + ------- + Series + Containing integers indicating the day number. + + See Also + -------- + Series.dt.dayofweek : Alias. + Series.dt.weekday : Alias. + Series.dt.day_name : Returns the name of the day of the week. + + Examples + -------- + >>> s = pp.from_pandas(pd.date_range('2016-12-31', '2017-01-08', freq='D').to_series()) + >>> s.dt.dayofweek + 2016-12-31 5 + 2017-01-01 6 + 2017-01-02 0 + 2017-01-03 1 + 2017-01-04 2 + 2017-01-05 3 + 2017-01-06 4 + 2017-01-07 5 + 2017-01-08 6 + dtype: int64 + """ + + def pandas_dayofweek(s) -> "pp.Series[np.int64]": + return s.dt.dayofweek + + return self._data.koalas.transform_batch(pandas_dayofweek) + + @property + def weekday(self) -> "pp.Series": + return self.dayofweek + + weekday.__doc__ = dayofweek.__doc__ + + @property + def dayofyear(self) -> "pp.Series": + """ + The ordinal day of the year. + """ + + def pandas_dayofyear(s) -> "pp.Series[np.int64]": + return s.dt.dayofyear + + return self._data.koalas.transform_batch(pandas_dayofyear) + + @property + def quarter(self) -> "pp.Series": + """ + The quarter of the date. + """ + + def pandas_quarter(s) -> "pp.Series[np.int64]": + return s.dt.quarter + + return self._data.koalas.transform_batch(pandas_quarter) + + @property + def is_month_start(self) -> "pp.Series": + """ + Indicates whether the date is the first day of the month. + + Returns + ------- + Series + For Series, returns a Series with boolean values. + + See Also + -------- + is_month_end : Return a boolean indicating whether the date + is the last day of the month. + + Examples + -------- + This method is available on Series with datetime values under + the ``.dt`` accessor. + + >>> s = pp.Series(pd.date_range("2018-02-27", periods=3)) + >>> s + 0 2018-02-27 + 1 2018-02-28 + 2 2018-03-01 + dtype: datetime64[ns] + + >>> s.dt.is_month_start + 0 False + 1 False + 2 True + dtype: bool + """ + + def pandas_is_month_start(s) -> "pp.Series[bool]": + return s.dt.is_month_start + + return self._data.koalas.transform_batch(pandas_is_month_start) + + @property + def is_month_end(self) -> "pp.Series": + """ + Indicates whether the date is the last day of the month. + + Returns + ------- + Series + For Series, returns a Series with boolean values. + + See Also + -------- + is_month_start : Return a boolean indicating whether the date + is the first day of the month. + + Examples + -------- + This method is available on Series with datetime values under + the ``.dt`` accessor. + + >>> s = pp.Series(pd.date_range("2018-02-27", periods=3)) + >>> s + 0 2018-02-27 + 1 2018-02-28 + 2 2018-03-01 + dtype: datetime64[ns] + + >>> s.dt.is_month_end + 0 False + 1 True + 2 False + dtype: bool + """ + + def pandas_is_month_end(s) -> "pp.Series[bool]": + return s.dt.is_month_end + + return self._data.koalas.transform_batch(pandas_is_month_end) + + @property + def is_quarter_start(self) -> "pp.Series": + """ + Indicator for whether the date is the first day of a quarter. + + Returns + ------- + is_quarter_start : Series + The same type as the original data with boolean values. Series will + have the same name and index. + + See Also + -------- + quarter : Return the quarter of the date. + is_quarter_end : Similar property for indicating the quarter start. + + Examples + -------- + This method is available on Series with datetime values under + the ``.dt`` accessor. + + >>> df = pp.DataFrame({'dates': pd.date_range("2017-03-30", + ... periods=4)}) + >>> df + dates + 0 2017-03-30 + 1 2017-03-31 + 2 2017-04-01 + 3 2017-04-02 + + >>> df.dates.dt.quarter + 0 1 + 1 1 + 2 2 + 3 2 + Name: dates, dtype: int64 + + >>> df.dates.dt.is_quarter_start + 0 False + 1 False + 2 True + 3 False + Name: dates, dtype: bool + """ + + def pandas_is_quarter_start(s) -> "pp.Series[bool]": + return s.dt.is_quarter_start + + return self._data.koalas.transform_batch(pandas_is_quarter_start) + + @property + def is_quarter_end(self) -> "pp.Series": + """ + Indicator for whether the date is the last day of a quarter. + + Returns + ------- + is_quarter_end : Series + The same type as the original data with boolean values. Series will + have the same name and index. + + See Also + -------- + quarter : Return the quarter of the date. + is_quarter_start : Similar property indicating the quarter start. + + Examples + -------- + This method is available on Series with datetime values under + the ``.dt`` accessor. + + >>> df = pp.DataFrame({'dates': pd.date_range("2017-03-30", + ... periods=4)}) + >>> df + dates + 0 2017-03-30 + 1 2017-03-31 + 2 2017-04-01 + 3 2017-04-02 + + >>> df.dates.dt.quarter + 0 1 + 1 1 + 2 2 + 3 2 + Name: dates, dtype: int64 + + >>> df.dates.dt.is_quarter_start + 0 False + 1 False + 2 True + 3 False + Name: dates, dtype: bool + """ + + def pandas_is_quarter_end(s) -> "pp.Series[bool]": + return s.dt.is_quarter_end + + return self._data.koalas.transform_batch(pandas_is_quarter_end) + + @property + def is_year_start(self) -> "pp.Series": + """ + Indicate whether the date is the first day of a year. + + Returns + ------- + Series + The same type as the original data with boolean values. Series will + have the same name and index. + + See Also + -------- + is_year_end : Similar property indicating the last day of the year. + + Examples + -------- + This method is available on Series with datetime values under + the ``.dt`` accessor. + + >>> dates = pp.Series(pd.date_range("2017-12-30", periods=3)) + >>> dates + 0 2017-12-30 + 1 2017-12-31 + 2 2018-01-01 + dtype: datetime64[ns] + + >>> dates.dt.is_year_start + 0 False + 1 False + 2 True + dtype: bool + """ + + def pandas_is_year_start(s) -> "pp.Series[bool]": + return s.dt.is_year_start + + return self._data.koalas.transform_batch(pandas_is_year_start) + + @property + def is_year_end(self) -> "pp.Series": + """ + Indicate whether the date is the last day of the year. + + Returns + ------- + Series + The same type as the original data with boolean values. Series will + have the same name and index. + + See Also + -------- + is_year_start : Similar property indicating the start of the year. + + Examples + -------- + This method is available on Series with datetime values under + the ``.dt`` accessor. + + >>> dates = pp.Series(pd.date_range("2017-12-30", periods=3)) + >>> dates + 0 2017-12-30 + 1 2017-12-31 + 2 2018-01-01 + dtype: datetime64[ns] + + >>> dates.dt.is_year_end + 0 False + 1 True + 2 False + dtype: bool + """ + + def pandas_is_year_end(s) -> "pp.Series[bool]": + return s.dt.is_year_end + + return self._data.koalas.transform_batch(pandas_is_year_end) + + @property + def is_leap_year(self) -> "pp.Series": + """ + Boolean indicator if the date belongs to a leap year. + + A leap year is a year, which has 366 days (instead of 365) including + 29th of February as an intercalary day. + Leap years are years which are multiples of four with the exception + of years divisible by 100 but not by 400. + + Returns + ------- + Series + Booleans indicating if dates belong to a leap year. + + Examples + -------- + This method is available on Series with datetime values under + the ``.dt`` accessor. + + >>> dates_series = pp.Series(pd.date_range("2012-01-01", "2015-01-01", freq="Y")) + >>> dates_series + 0 2012-12-31 + 1 2013-12-31 + 2 2014-12-31 + dtype: datetime64[ns] + + >>> dates_series.dt.is_leap_year + 0 True + 1 False + 2 False + dtype: bool + """ + + def pandas_is_leap_year(s) -> "pp.Series[bool]": + return s.dt.is_leap_year + + return self._data.koalas.transform_batch(pandas_is_leap_year) + + @property + def daysinmonth(self) -> "pp.Series": + """ + The number of days in the month. + """ + + def pandas_daysinmonth(s) -> "pp.Series[np.int64]": + return s.dt.daysinmonth + + return self._data.koalas.transform_batch(pandas_daysinmonth) + + @property + def days_in_month(self) -> "pp.Series": + return self.daysinmonth + + days_in_month.__doc__ = daysinmonth.__doc__ + + # Methods + + def tz_localize(self, tz) -> "pp.Series": + """ + Localize tz-naive Datetime column to tz-aware Datetime column. + """ + # Neither tz-naive or tz-aware datetime exists in Spark + raise NotImplementedError() + + def tz_convert(self, tz) -> "pp.Series": + """ + Convert tz-aware Datetime column from one time zone to another. + """ + # tz-aware datetime doesn't exist in Spark + raise NotImplementedError() + + def normalize(self) -> "pp.Series": + """ + Convert times to midnight. + + The time component of the date-time is converted to midnight i.e. + 00:00:00. This is useful in cases, when the time does not matter. + Length is unaltered. The timezones are unaffected. + + This method is available on Series with datetime values under + the ``.dt`` accessor, and directly on Datetime Array. + + Returns + ------- + Series + The same type as the original data. Series will have the same + name and index. + + See Also + -------- + floor : Floor the series to the specified freq. + ceil : Ceil the series to the specified freq. + round : Round the series to the specified freq. + + Examples + -------- + >>> series = pp.Series(pd.Series(pd.date_range('2012-1-1 12:45:31', periods=3, freq='M'))) + >>> series.dt.normalize() + 0 2012-01-31 + 1 2012-02-29 + 2 2012-03-31 + dtype: datetime64[ns] + """ + + def pandas_normalize(s) -> "pp.Series[np.datetime64]": + return s.dt.normalize() + + return self._data.koalas.transform_batch(pandas_normalize) + + def strftime(self, date_format) -> "pp.Series": + """ + Convert to a string Series using specified date_format. + + Return an series of formatted strings specified by date_format, which + supports the same string format as the python standard library. Details + of the string format can be found in python string format + doc. + + Parameters + ---------- + date_format : str + Date format string (e.g. "%%Y-%%m-%%d"). + + Returns + ------- + Series + Series of formatted strings. + + See Also + -------- + to_datetime : Convert the given argument to datetime. + normalize : Return series with times to midnight. + round : Round the series to the specified freq. + floor : Floor the series to the specified freq. + + Examples + -------- + >>> series = pp.Series(pd.date_range(pd.Timestamp("2018-03-10 09:00"), + ... periods=3, freq='s')) + >>> series + 0 2018-03-10 09:00:00 + 1 2018-03-10 09:00:01 + 2 2018-03-10 09:00:02 + dtype: datetime64[ns] + + >>> series.dt.strftime('%B %d, %Y, %r') + 0 March 10, 2018, 09:00:00 AM + 1 March 10, 2018, 09:00:01 AM + 2 March 10, 2018, 09:00:02 AM + dtype: object + """ + + def pandas_strftime(s) -> "pp.Series[str]": + return s.dt.strftime(date_format) + + return self._data.koalas.transform_batch(pandas_strftime) + + def round(self, freq, *args, **kwargs) -> "pp.Series": + """ + Perform round operation on the data to the specified freq. + + Parameters + ---------- + freq : str or Offset + The frequency level to round the index to. Must be a fixed + frequency like 'S' (second) not 'ME' (month end). + + nonexistent : 'shift_forward', 'shift_backward, 'NaT', timedelta, default 'raise' + A nonexistent time does not exist in a particular timezone + where clocks moved forward due to DST. + + - 'shift_forward' will shift the nonexistent time forward to the + closest existing time + - 'shift_backward' will shift the nonexistent time backward to the + closest existing time + - 'NaT' will return NaT where there are nonexistent times + - timedelta objects will shift nonexistent times by the timedelta + - 'raise' will raise an NonExistentTimeError if there are + nonexistent times + + .. note:: this option only works with pandas 0.24.0+ + + Returns + ------- + Series + a Series with the same index for a Series. + + Raises + ------ + ValueError if the `freq` cannot be converted. + + Examples + -------- + >>> series = pp.Series(pd.date_range('1/1/2018 11:59:00', periods=3, freq='min')) + >>> series + 0 2018-01-01 11:59:00 + 1 2018-01-01 12:00:00 + 2 2018-01-01 12:01:00 + dtype: datetime64[ns] + + >>> series.dt.round("H") + 0 2018-01-01 12:00:00 + 1 2018-01-01 12:00:00 + 2 2018-01-01 12:00:00 + dtype: datetime64[ns] + """ + + def pandas_round(s) -> "pp.Series[np.datetime64]": + return s.dt.round(freq, *args, **kwargs) + + return self._data.koalas.transform_batch(pandas_round) + + def floor(self, freq, *args, **kwargs) -> "pp.Series": + """ + Perform floor operation on the data to the specified freq. + + Parameters + ---------- + freq : str or Offset + The frequency level to floor the index to. Must be a fixed + frequency like 'S' (second) not 'ME' (month end). + + nonexistent : 'shift_forward', 'shift_backward, 'NaT', timedelta, default 'raise' + A nonexistent time does not exist in a particular timezone + where clocks moved forward due to DST. + + - 'shift_forward' will shift the nonexistent time forward to the + closest existing time + - 'shift_backward' will shift the nonexistent time backward to the + closest existing time + - 'NaT' will return NaT where there are nonexistent times + - timedelta objects will shift nonexistent times by the timedelta + - 'raise' will raise an NonExistentTimeError if there are + nonexistent times + + .. note:: this option only works with pandas 0.24.0+ + + Returns + ------- + Series + a Series with the same index for a Series. + + Raises + ------ + ValueError if the `freq` cannot be converted. + + Examples + -------- + >>> series = pp.Series(pd.date_range('1/1/2018 11:59:00', periods=3, freq='min')) + >>> series + 0 2018-01-01 11:59:00 + 1 2018-01-01 12:00:00 + 2 2018-01-01 12:01:00 + dtype: datetime64[ns] + + >>> series.dt.floor("H") + 0 2018-01-01 11:00:00 + 1 2018-01-01 12:00:00 + 2 2018-01-01 12:00:00 + dtype: datetime64[ns] + """ + + def pandas_floor(s) -> "pp.Series[np.datetime64]": + return s.dt.floor(freq, *args, **kwargs) + + return self._data.koalas.transform_batch(pandas_floor) + + def ceil(self, freq, *args, **kwargs) -> "pp.Series": + """ + Perform ceil operation on the data to the specified freq. + + Parameters + ---------- + freq : str or Offset + The frequency level to round the index to. Must be a fixed + frequency like 'S' (second) not 'ME' (month end). + + nonexistent : 'shift_forward', 'shift_backward, 'NaT', timedelta, default 'raise' + A nonexistent time does not exist in a particular timezone + where clocks moved forward due to DST. + + - 'shift_forward' will shift the nonexistent time forward to the + closest existing time + - 'shift_backward' will shift the nonexistent time backward to the + closest existing time + - 'NaT' will return NaT where there are nonexistent times + - timedelta objects will shift nonexistent times by the timedelta + - 'raise' will raise an NonExistentTimeError if there are + nonexistent times + + .. note:: this option only works with pandas 0.24.0+ + + Returns + ------- + Series + a Series with the same index for a Series. + + Raises + ------ + ValueError if the `freq` cannot be converted. + + Examples + -------- + >>> series = pp.Series(pd.date_range('1/1/2018 11:59:00', periods=3, freq='min')) + >>> series + 0 2018-01-01 11:59:00 + 1 2018-01-01 12:00:00 + 2 2018-01-01 12:01:00 + dtype: datetime64[ns] + + >>> series.dt.ceil("H") + 0 2018-01-01 12:00:00 + 1 2018-01-01 12:00:00 + 2 2018-01-01 13:00:00 + dtype: datetime64[ns] + """ + + def pandas_ceil(s) -> "pp.Series[np.datetime64]": + return s.dt.ceil(freq, *args, **kwargs) + + return self._data.koalas.transform_batch(pandas_ceil) + + def month_name(self, locale=None) -> "pp.Series": + """ + Return the month names of the series with specified locale. + + Parameters + ---------- + locale : str, optional + Locale determining the language in which to return the month name. + Default is English locale. + + Returns + ------- + Series + Series of month names. + + Examples + -------- + >>> series = pp.Series(pd.date_range(start='2018-01', freq='M', periods=3)) + >>> series + 0 2018-01-31 + 1 2018-02-28 + 2 2018-03-31 + dtype: datetime64[ns] + + >>> series.dt.month_name() + 0 January + 1 February + 2 March + dtype: object + """ + + def pandas_month_name(s) -> "pp.Series[str]": + return s.dt.month_name(locale=locale) + + return self._data.koalas.transform_batch(pandas_month_name) + + def day_name(self, locale=None) -> "pp.Series": + """ + Return the day names of the series with specified locale. + + Parameters + ---------- + locale : str, optional + Locale determining the language in which to return the day name. + Default is English locale. + + Returns + ------- + Series + Series of day names. + + Examples + -------- + >>> series = pp.Series(pd.date_range(start='2018-01-01', freq='D', periods=3)) + >>> series + 0 2018-01-01 + 1 2018-01-02 + 2 2018-01-03 + dtype: datetime64[ns] + + >>> series.dt.day_name() + 0 Monday + 1 Tuesday + 2 Wednesday + dtype: object + """ + + def pandas_day_name(s) -> "pp.Series[str]": + return s.dt.day_name(locale=locale) + + return self._data.koalas.transform_batch(pandas_day_name) diff --git a/python/pyspark/pandas/exceptions.py b/python/pyspark/pandas/exceptions.py new file mode 100644 index 0000000000..7ec874b101 --- /dev/null +++ b/python/pyspark/pandas/exceptions.py @@ -0,0 +1,106 @@ +# +# 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. +# + +""" +Exceptions/Errors used in Koalas. +""" + + +class DataError(Exception): + pass + + +class SparkPandasIndexingError(Exception): + pass + + +def code_change_hint(pandas_function, spark_target_function): + if pandas_function is not None and spark_target_function is not None: + return "You are trying to use pandas function {}, use spark function {}".format( + pandas_function, spark_target_function + ) + elif pandas_function is not None and spark_target_function is None: + return ( + "You are trying to use pandas function {}, checkout the spark " + "user guide to find a relevant function" + ).format(pandas_function) + elif pandas_function is None and spark_target_function is not None: + return "Use spark function {}".format(spark_target_function) + else: # both none + return "Checkout the spark user guide to find a relevant function" + + +class SparkPandasNotImplementedError(NotImplementedError): + def __init__(self, pandas_function=None, spark_target_function=None, description=""): + self.pandas_source = pandas_function + self.spark_target = spark_target_function + hint = code_change_hint(pandas_function, spark_target_function) + if len(description) > 0: + description += " " + hint + else: + description = hint + super().__init__(description) + + +class PandasNotImplementedError(NotImplementedError): + def __init__( + self, + class_name, + method_name=None, + arg_name=None, + property_name=None, + deprecated=False, + reason="", + ): + assert (method_name is None) != (property_name is None) + self.class_name = class_name + self.method_name = method_name + self.arg_name = arg_name + if method_name is not None: + if arg_name is not None: + msg = "The method `{0}.{1}()` does not support `{2}` parameter. {3}".format( + class_name, method_name, arg_name, reason + ) + else: + if deprecated: + msg = ( + "The method `{0}.{1}()` is deprecated in pandas and will therefore " + + "not be supported in Koalas. {2}" + ).format(class_name, method_name, reason) + else: + if reason == "": + reason = " yet." + else: + reason = ". " + reason + msg = "The method `{0}.{1}()` is not implemented{2}".format( + class_name, method_name, reason + ) + else: + if deprecated: + msg = ( + "The property `{0}.{1}()` is deprecated in pandas and will therefore " + + "not be supported in Koalas. {2}" + ).format(class_name, property_name, reason) + else: + if reason == "": + reason = " yet." + else: + reason = ". " + reason + msg = "The property `{0}.{1}()` is not implemented{2}".format( + class_name, property_name, reason + ) + super().__init__(msg) diff --git a/python/pyspark/pandas/extensions.py b/python/pyspark/pandas/extensions.py new file mode 100644 index 0000000000..9b67f2a169 --- /dev/null +++ b/python/pyspark/pandas/extensions.py @@ -0,0 +1,342 @@ +# +# 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 warnings + + +class CachedAccessor: + """ + Custom property-like object. + + A descriptor for caching accessors: + + Parameters + ---------- + name : str + Namespace that accessor's methods, properties, etc will be accessed under, e.g. "foo" for a + dataframe accessor yields the accessor ``df.foo`` + accessor: cls + Class with the extension methods. + + Notes + ----- + For accessor, the class's __init__ method assumes that you are registering an accessor for one + of ``Series``, ``DataFrame``, or ``Index``. + + This object is not meant to be instantiated directly. Instead, use register_dataframe_accessor, + register_series_accessor, or register_index_accessor. + + The Koalas accessor is modified based on pandas.core.accessor. + """ + + def __init__(self, name, accessor): + self._name = name + self._accessor = accessor + + def __get__(self, obj, cls): + if obj is None: + return self._accessor + accessor_obj = self._accessor(obj) + object.__setattr__(obj, self._name, accessor_obj) + return accessor_obj + + +def _register_accessor(name, cls): + """ + Register a custom accessor on {klass} objects. + + Parameters + ---------- + name : str + Name under which the accessor should be registered. A warning is issued if this name + conflicts with a preexisting attribute. + + Returns + ------- + callable + A class decorator. + + See Also + -------- + register_dataframe_accessor: Register a custom accessor on DataFrame objects + register_series_accessor: Register a custom accessor on Series objects + register_index_accessor: Register a custom accessor on Index objects + + Notes + ----- + When accessed, your accessor will be initialiazed with the Koalas object the user is interacting + with. The code signature must be: + + .. code-block:: python + + def __init__(self, koalas_obj): + # constructor logic + ... + + In the pandas API, if data passed to your accessor has an incorrect dtype, it's recommended to + raise an ``AttributeError`` for consistency purposes. In Koalas, ``ValueError`` is more + frequently used to annotate when a value's datatype is unexpected for a given method/function. + + Ultimately, you can structure this however you like, but Koalas would likely do something like + this: + + >>> pp.Series(['a', 'b']).dt + ... + Traceback (most recent call last): + ... + ValueError: Cannot call DatetimeMethods on type StringType + + Note: This function is not meant to be used directly - instead, use register_dataframe_accessor, + register_series_accessor, or register_index_accessor. + """ + + def decorator(accessor): + if hasattr(cls, name): + msg = ( + "registration of accessor {0} under name '{1}' for type {2} is overriding " + "a preexisting attribute with the same name.".format(accessor, name, cls.__name__) + ) + + warnings.warn( + msg, UserWarning, stacklevel=2, + ) + setattr(cls, name, CachedAccessor(name, accessor)) + return accessor + + return decorator + + +def register_dataframe_accessor(name): + """ + Register a custom accessor with a DataFrame + + Parameters + ---------- + name : str + name used when calling the accessor after its registered + + Returns + ------- + callable + A class decorator. + + See Also + -------- + register_series_accessor: Register a custom accessor on Series objects + register_index_accessor: Register a custom accessor on Index objects + + Notes + ----- + When accessed, your accessor will be initialiazed with the Koalas object the user is interacting + with. The accessor's init method should always ingest the object being accessed. See the + examples for the init signature. + + In the pandas API, if data passed to your accessor has an incorrect dtype, it's recommended to + raise an ``AttributeError`` for consistency purposes. In Koalas, ``ValueError`` is more + frequently used to annotate when a value's datatype is unexpected for a given method/function. + + Ultimately, you can structure this however you like, but Koalas would likely do something like + this: + + >>> pp.Series(['a', 'b']).dt + ... + Traceback (most recent call last): + ... + ValueError: Cannot call DatetimeMethods on type StringType + + Examples + -------- + In your library code:: + + from pyspark.pandas.extensions import register_dataframe_accessor + + @register_dataframe_accessor("geo") + class GeoAccessor: + + def __init__(self, koalas_obj): + self._obj = koalas_obj + # other constructor logic + + @property + def center(self): + # return the geographic center point of this DataFrame + lat = self._obj.latitude + lon = self._obj.longitude + return (float(lon.mean()), float(lat.mean())) + + def plot(self): + # plot this array's data on a map + pass + + Then, in an ipython session:: + + >>> ## Import if the accessor is in the other file. + >>> # from my_ext_lib import GeoAccessor + >>> kdf = pp.DataFrame({"longitude": np.linspace(0,10), + ... "latitude": np.linspace(0, 20)}) + >>> kdf.geo.center # doctest: +SKIP + (5.0, 10.0) + + >>> kdf.geo.plot() # doctest: +SKIP + """ + from pyspark.pandas import DataFrame + + return _register_accessor(name, DataFrame) + + +def register_series_accessor(name): + """ + Register a custom accessor with a Series object + + Parameters + ---------- + name : str + name used when calling the accessor after its registered + + Returns + ------- + callable + A class decorator. + + See Also + -------- + register_dataframe_accessor: Register a custom accessor on DataFrame objects + register_index_accessor: Register a custom accessor on Index objects + + Notes + ----- + When accessed, your accessor will be initialiazed with the Koalas object the user is interacting + with. The code signature must be:: + + def __init__(self, koalas_obj): + # constructor logic + ... + + In the pandas API, if data passed to your accessor has an incorrect dtype, it's recommended to + raise an ``AttributeError`` for consistency purposes. In Koalas, ``ValueError`` is more + frequently used to annotate when a value's datatype is unexpected for a given method/function. + + Ultimately, you can structure this however you like, but Koalas would likely do something like + this: + + >>> pp.Series(['a', 'b']).dt + ... + Traceback (most recent call last): + ... + ValueError: Cannot call DatetimeMethods on type StringType + + Examples + -------- + In your library code:: + + from pyspark.pandas.extensions import register_series_accessor + + @register_series_accessor("geo") + class GeoAccessor: + + def __init__(self, koalas_obj): + self._obj = koalas_obj + + @property + def is_valid(self): + # boolean check to see if series contains valid geometry + return True + + Then, in an ipython session:: + + >>> ## Import if the accessor is in the other file. + >>> # from my_ext_lib import GeoAccessor + >>> kdf = pp.DataFrame({"longitude": np.linspace(0,10), + ... "latitude": np.linspace(0, 20)}) + >>> kdf.longitude.geo.is_valid # doctest: +SKIP + True + """ + from pyspark.pandas import Series + + return _register_accessor(name, Series) + + +def register_index_accessor(name): + """ + Register a custom accessor with an Index + + Parameters + ---------- + name : str + name used when calling the accessor after its registered + + Returns + ------- + callable + A class decorator. + + See Also + -------- + register_dataframe_accessor: Register a custom accessor on DataFrame objects + register_series_accessor: Register a custom accessor on Series objects + + Notes + ----- + When accessed, your accessor will be initialiazed with the Koalas object the user is interacting + with. The code signature must be:: + + def __init__(self, koalas_obj): + # constructor logic + ... + + In the pandas API, if data passed to your accessor has an incorrect dtype, it's recommended to + raise an ``AttributeError`` for consistency purposes. In Koalas, ``ValueError`` is more + frequently used to annotate when a value's datatype is unexpected for a given method/function. + + Ultimately, you can structure this however you like, but Koalas would likely do something like + this: + + >>> pp.Series(['a', 'b']).dt + ... + Traceback (most recent call last): + ... + ValueError: Cannot call DatetimeMethods on type StringType + + Examples + -------- + In your library code:: + + from pyspark.pandas.extensions import register_index_accessor + + @register_index_accessor("foo") + class CustomAccessor: + + def __init__(self, koalas_obj): + self._obj = koalas_obj + self.item = "baz" + + @property + def bar(self): + # return item value + return self.item + + Then, in an ipython session:: + + >>> ## Import if the accessor is in the other file. + >>> # from my_ext_lib import CustomAccessor + >>> kdf = pp.DataFrame({"longitude": np.linspace(0,10), + ... "latitude": np.linspace(0, 20)}) + >>> kdf.index.foo.bar # doctest: +SKIP + 'baz' + """ + from pyspark.pandas import Index + + return _register_accessor(name, Index) diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py new file mode 100644 index 0000000000..4e7b9fa12a --- /dev/null +++ b/python/pyspark/pandas/frame.py @@ -0,0 +1,11976 @@ +# +# 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. +# + +""" +A wrapper class for Spark DataFrame to behave similar to pandas DataFrame. +""" +from collections import OrderedDict, defaultdict, namedtuple +from collections.abc import Mapping +from distutils.version import LooseVersion +import re +import warnings +import inspect +import json +import types +from functools import partial, reduce +import sys +from itertools import zip_longest +from typing import ( + Any, + Optional, + List, + Tuple, + Union, + Generic, + TypeVar, + Iterable, + Iterator, + Dict, + Callable, + cast, + TYPE_CHECKING, +) +import datetime + +import numpy as np +import pandas as pd +from pandas.api.types import is_list_like, is_dict_like, is_scalar +from pandas.api.extensions import ExtensionDtype +from pandas.tseries.frequencies import DateOffset, to_offset + +if TYPE_CHECKING: + from pandas.io.formats.style import Styler # noqa: F401 (SPARK-34943) + +if LooseVersion(pd.__version__) >= LooseVersion("0.24"): + from pandas.core.dtypes.common import infer_dtype_from_object +else: + from pandas.core.dtypes.common import _get_dtype_from_object as infer_dtype_from_object +from pandas.core.accessor import CachedAccessor +from pandas.core.dtypes.inference import is_sequence +import pyspark +from pyspark import StorageLevel +from pyspark import sql as spark +from pyspark.sql import Column, DataFrame as SparkDataFrame, functions as F +from pyspark.sql.functions import pandas_udf +from pyspark.sql.types import ( + BooleanType, + DoubleType, + FloatType, + NumericType, + StringType, + StructType, + StructField, + ArrayType, +) +from pyspark.sql.window import Window + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas.accessors import KoalasFrameMethods +from pyspark.pandas.config import option_context, get_option +from pyspark.pandas.spark import functions as SF +from pyspark.pandas.spark.accessors import SparkFrameMethods, CachedSparkFrameMethods +from pyspark.pandas.utils import ( + align_diff_frames, + column_labels_level, + combine_frames, + default_session, + is_name_like_tuple, + is_name_like_value, + is_testing, + name_like_string, + same_anchor, + scol_for, + validate_arguments_and_invoke_function, + validate_axis, + validate_bool_kwarg, + validate_how, + verify_temp_column_name, +) +from pyspark.pandas.spark.utils import as_nullable_spark_type, force_decimal_precision_scale +from pyspark.pandas.generic import Frame +from pyspark.pandas.internal import ( + InternalFrame, + HIDDEN_COLUMNS, + NATURAL_ORDER_COLUMN_NAME, + SPARK_INDEX_NAME_FORMAT, + SPARK_DEFAULT_INDEX_NAME, + SPARK_DEFAULT_SERIES_NAME, +) +from pyspark.pandas.missing.frame import _MissingPandasLikeDataFrame +from pyspark.pandas.ml import corr +from pyspark.pandas.typedef import ( + as_spark_type, + infer_return_type, + spark_type_to_pandas_dtype, + DataFrameType, + SeriesType, + Scalar, + ScalarType, +) +from pyspark.pandas.plot import KoalasPlotAccessor + +if TYPE_CHECKING: + from pyspark.pandas.indexes import Index # noqa: F401 (SPARK-34943) + from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943) + + +# These regular expression patterns are complied and defined here to avoid to compile the same +# pattern every time it is used in _repr_ and _repr_html_ in DataFrame. +# Two patterns basically seek the footer string from Pandas' +REPR_PATTERN = re.compile(r"\n\n\[(?P[0-9]+) rows x (?P[0-9]+) columns\]$") +REPR_HTML_PATTERN = re.compile( + r"\n\(?P[0-9]+) rows × (?P[0-9]+) columns\<\/p\>\n\<\/div\>$" +) + + +_flex_doc_FRAME = """ +Get {desc} of dataframe and other, element-wise (binary operator `{op_name}`). + +Equivalent to ``{equiv}``. With reverse version, `{reverse}`. + +Among flexible wrappers (`add`, `sub`, `mul`, `div`) to +arithmetic operators: `+`, `-`, `*`, `/`, `//`. + +Parameters +---------- +other : scalar + Any single data + +Returns +------- +DataFrame + Result of the arithmetic operation. + +Examples +-------- +>>> df = pp.DataFrame({{'angles': [0, 3, 4], +... 'degrees': [360, 180, 360]}}, +... index=['circle', 'triangle', 'rectangle'], +... columns=['angles', 'degrees']) +>>> df + angles degrees +circle 0 360 +triangle 3 180 +rectangle 4 360 + +Add a scalar with operator version which return the same +results. Also reverse version. + +>>> df + 1 + angles degrees +circle 1 361 +triangle 4 181 +rectangle 5 361 + +>>> df.add(1) + angles degrees +circle 1 361 +triangle 4 181 +rectangle 5 361 + +>>> df.add(df) + angles degrees +circle 0 720 +triangle 6 360 +rectangle 8 720 + +>>> df + df + df + angles degrees +circle 0 1080 +triangle 9 540 +rectangle 12 1080 + +>>> df.radd(1) + angles degrees +circle 1 361 +triangle 4 181 +rectangle 5 361 + +Divide and true divide by constant with reverse version. + +>>> df / 10 + angles degrees +circle 0.0 36.0 +triangle 0.3 18.0 +rectangle 0.4 36.0 + +>>> df.div(10) + angles degrees +circle 0.0 36.0 +triangle 0.3 18.0 +rectangle 0.4 36.0 + +>>> df.rdiv(10) + angles degrees +circle inf 0.027778 +triangle 3.333333 0.055556 +rectangle 2.500000 0.027778 + +>>> df.truediv(10) + angles degrees +circle 0.0 36.0 +triangle 0.3 18.0 +rectangle 0.4 36.0 + +>>> df.rtruediv(10) + angles degrees +circle inf 0.027778 +triangle 3.333333 0.055556 +rectangle 2.500000 0.027778 + +Subtract by constant with reverse version. + +>>> df - 1 + angles degrees +circle -1 359 +triangle 2 179 +rectangle 3 359 + +>>> df.sub(1) + angles degrees +circle -1 359 +triangle 2 179 +rectangle 3 359 + +>>> df.rsub(1) + angles degrees +circle 1 -359 +triangle -2 -179 +rectangle -3 -359 + +Multiply by constant with reverse version. + +>>> df * 1 + angles degrees +circle 0 360 +triangle 3 180 +rectangle 4 360 + +>>> df.mul(1) + angles degrees +circle 0 360 +triangle 3 180 +rectangle 4 360 + +>>> df.rmul(1) + angles degrees +circle 0 360 +triangle 3 180 +rectangle 4 360 + +Floor Divide by constant with reverse version. + +>>> df // 10 + angles degrees +circle 0.0 36.0 +triangle 0.0 18.0 +rectangle 0.0 36.0 + +>>> df.floordiv(10) + angles degrees +circle 0.0 36.0 +triangle 0.0 18.0 +rectangle 0.0 36.0 + +>>> df.rfloordiv(10) # doctest: +SKIP + angles degrees +circle inf 0.0 +triangle 3.0 0.0 +rectangle 2.0 0.0 + +Mod by constant with reverse version. + +>>> df % 2 + angles degrees +circle 0 0 +triangle 1 0 +rectangle 0 0 + +>>> df.mod(2) + angles degrees +circle 0 0 +triangle 1 0 +rectangle 0 0 + +>>> df.rmod(2) + angles degrees +circle NaN 2 +triangle 2.0 2 +rectangle 2.0 2 + +Power by constant with reverse version. + +>>> df ** 2 + angles degrees +circle 0.0 129600.0 +triangle 9.0 32400.0 +rectangle 16.0 129600.0 + +>>> df.pow(2) + angles degrees +circle 0.0 129600.0 +triangle 9.0 32400.0 +rectangle 16.0 129600.0 + +>>> df.rpow(2) + angles degrees +circle 1.0 2.348543e+108 +triangle 8.0 1.532496e+54 +rectangle 16.0 2.348543e+108 +""" + +T = TypeVar("T") + + +def _create_tuple_for_frame_type(params): + """ + This is a workaround to support variadic generic in DataFrame. + + See https://github.com/python/typing/issues/193 + we always wraps the given type hints by a tuple to mimic the variadic generic. + """ + from pyspark.pandas.typedef import NameTypeHolder + + if isinstance(params, zip): + params = [slice(name, tpe) for name, tpe in params] + + if isinstance(params, slice): + params = (params,) + + if ( + hasattr(params, "__len__") + and isinstance(params, Iterable) + and all(isinstance(param, slice) for param in params) + ): + for param in params: + if isinstance(param.start, str) and param.step is not None: + raise TypeError( + "Type hints should be specified as " + "DataFrame['name': type]; however, got %s" % param + ) + + name_classes = [] + for param in params: + new_class = type("NameType", (NameTypeHolder,), {}) + new_class.name = param.start + # When the given argument is a numpy's dtype instance. + new_class.tpe = param.stop.type if isinstance(param.stop, np.dtype) else param.stop + name_classes.append(new_class) + + return Tuple[tuple(name_classes)] + + if not isinstance(params, Iterable): + params = [params] + + new_params = [] + for param in params: + if isinstance(param, ExtensionDtype): + new_class = type("NameType", (NameTypeHolder,), {}) + new_class.tpe = param + new_params.append(new_class) + else: + new_params.append(param.type if isinstance(param, np.dtype) else param) + return Tuple[tuple(new_params)] + + +if (3, 5) <= sys.version_info < (3, 7): + from typing import GenericMeta # type: ignore + + # This is a workaround to support variadic generic in DataFrame in Python 3.5+. + # See https://github.com/python/typing/issues/193 + # We wrap the input params by a tuple to mimic variadic generic. + old_getitem = GenericMeta.__getitem__ # type: ignore + + def new_getitem(self, params): + if hasattr(self, "is_dataframe"): + return old_getitem(self, _create_tuple_for_frame_type(params)) + else: + return old_getitem(self, params) + + GenericMeta.__getitem__ = new_getitem # type: ignore + + +class DataFrame(Frame, Generic[T]): + """ + Koalas DataFrame that corresponds to pandas DataFrame logically. This holds Spark DataFrame + internally. + + :ivar _internal: an internal immutable Frame to manage metadata. + :type _internal: InternalFrame + + Parameters + ---------- + data : numpy ndarray (structured or homogeneous), dict, pandas DataFrame, Spark DataFrame \ + or Koalas Series + Dict can contain Series, arrays, constants, or list-like objects + If data is a dict, argument order is maintained for Python 3.6 + and later. + Note that if `data` is a pandas DataFrame, a Spark DataFrame, and a Koalas Series, + other arguments should not be used. + index : Index or array-like + Index to use for resulting frame. Will default to RangeIndex if + no indexing information part of input data and no index provided + columns : Index or array-like + Column labels to use for resulting frame. Will default to + RangeIndex (0, 1, 2, ..., n) if no column labels are provided + dtype : dtype, default None + Data type to force. Only a single dtype is allowed. If None, infer + copy : boolean, default False + Copy data from inputs. Only affects DataFrame / 2d ndarray input + + Examples + -------- + Constructing DataFrame from a dictionary. + + >>> d = {'col1': [1, 2], 'col2': [3, 4]} + >>> df = pp.DataFrame(data=d, columns=['col1', 'col2']) + >>> df + col1 col2 + 0 1 3 + 1 2 4 + + Constructing DataFrame from pandas DataFrame + + >>> df = pp.DataFrame(pd.DataFrame(data=d, columns=['col1', 'col2'])) + >>> df + col1 col2 + 0 1 3 + 1 2 4 + + Notice that the inferred dtype is int64. + + >>> df.dtypes + col1 int64 + col2 int64 + dtype: object + + To enforce a single dtype: + + >>> df = pp.DataFrame(data=d, dtype=np.int8) + >>> df.dtypes + col1 int8 + col2 int8 + dtype: object + + Constructing DataFrame from numpy ndarray: + + >>> df2 = pp.DataFrame(np.random.randint(low=0, high=10, size=(5, 5)), + ... columns=['a', 'b', 'c', 'd', 'e']) + >>> df2 # doctest: +SKIP + a b c d e + 0 3 1 4 9 8 + 1 4 8 4 8 4 + 2 7 6 5 6 7 + 3 8 7 9 1 0 + 4 2 5 4 3 9 + """ + + def __init__(self, data=None, index=None, columns=None, dtype=None, copy=False): + if isinstance(data, InternalFrame): + assert index is None + assert columns is None + assert dtype is None + assert not copy + internal = data + elif isinstance(data, spark.DataFrame): + assert index is None + assert columns is None + assert dtype is None + assert not copy + internal = InternalFrame(spark_frame=data, index_spark_columns=None) + elif isinstance(data, pp.Series): + assert index is None + assert columns is None + assert dtype is None + assert not copy + data = data.to_frame() + internal = data._internal + else: + if isinstance(data, pd.DataFrame): + assert index is None + assert columns is None + assert dtype is None + assert not copy + pdf = data + else: + pdf = pd.DataFrame(data=data, index=index, columns=columns, dtype=dtype, copy=copy) + internal = InternalFrame.from_pandas(pdf) + + object.__setattr__(self, "_internal_frame", internal) + + @property + def _ksers(self): + """ Return a dict of column label -> Series which anchors `self`. """ + from pyspark.pandas.series import Series + + if not hasattr(self, "_kseries"): + object.__setattr__( + self, + "_kseries", + {label: Series(data=self, index=label) for label in self._internal.column_labels}, + ) + else: + kseries = self._kseries + assert len(self._internal.column_labels) == len(kseries), ( + len(self._internal.column_labels), + len(kseries), + ) + if any(self is not kser._kdf for kser in kseries.values()): + # Refresh the dict to contain only Series anchoring `self`. + self._kseries = { + label: kseries[label] + if self is kseries[label]._kdf + else Series(data=self, index=label) + for label in self._internal.column_labels + } + return self._kseries + + @property + def _internal(self) -> InternalFrame: + return self._internal_frame + + def _update_internal_frame(self, internal: InternalFrame, requires_same_anchor: bool = True): + """ + Update InternalFrame with the given one. + + If the column_label is changed or the new InternalFrame is not the same `anchor`, + disconnect the link to the Series and create a new one. + + If `requires_same_anchor` is `False`, checking whether or not the same anchor is ignored + and force to update the InternalFrame, e.g., replacing the internal with the resolved_copy, + updating the underlying Spark DataFrame which need to combine a different Spark DataFrame. + + :param internal: the new InternalFrame + :param requires_same_anchor: whether checking the same anchor + """ + from pyspark.pandas.series import Series + + if hasattr(self, "_kseries"): + kseries = {} + + for old_label, new_label in zip_longest( + self._internal.column_labels, internal.column_labels + ): + if old_label is not None: + kser = self._ksers[old_label] + + renamed = old_label != new_label + not_same_anchor = requires_same_anchor and not same_anchor(internal, kser) + + if renamed or not_same_anchor: + kdf = DataFrame(self._internal.select_column(old_label)) # type: DataFrame + kser._update_anchor(kdf) + kser = None + else: + kser = None + if new_label is not None: + if kser is None: + kser = Series(data=self, index=new_label) + kseries[new_label] = kser + + self._kseries = kseries + + self._internal_frame = internal + + if hasattr(self, "_repr_pandas_cache"): + del self._repr_pandas_cache + + @property + def ndim(self) -> int: + """ + Return an int representing the number of array dimensions. + + return 2 for DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame([[1, 2], [4, 5], [7, 8]], + ... index=['cobra', 'viper', None], + ... columns=['max_speed', 'shield']) + >>> df + max_speed shield + cobra 1 2 + viper 4 5 + NaN 7 8 + >>> df.ndim + 2 + """ + return 2 + + @property + def axes(self) -> List: + """ + Return a list representing the axes of the DataFrame. + + It has the row axis labels and column axis labels as the only members. + They are returned in that order. + + Examples + -------- + + >>> df = pp.DataFrame({'col1': [1, 2], 'col2': [3, 4]}) + >>> df.axes + [Int64Index([0, 1], dtype='int64'), Index(['col1', 'col2'], dtype='object')] + """ + return [self.index, self.columns] + + def _reduce_for_stat_function(self, sfun, name, axis=None, numeric_only=True, **kwargs): + """ + Applies sfun to each column and returns a pd.Series where the number of rows equal the + number of columns. + + Parameters + ---------- + sfun : either an 1-arg function that takes a Column and returns a Column, or + a 2-arg function that takes a Column and its DataType and returns a Column. + axis: used only for sanity check because series only support index axis. + name : original pandas API name. + axis : axis to apply. 0 or 1, or 'index' or 'columns. + numeric_only : bool, default True + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. Only 'DataFrame.count' uses this parameter + currently. + """ + from inspect import signature + from pyspark.pandas.series import Series, first_series + + axis = validate_axis(axis) + if axis == 0: + min_count = kwargs.get("min_count", 0) + + exprs = [F.lit(None).cast(StringType()).alias(SPARK_DEFAULT_INDEX_NAME)] + new_column_labels = [] + num_args = len(signature(sfun).parameters) + for label in self._internal.column_labels: + spark_column = self._internal.spark_column_for(label) + spark_type = self._internal.spark_type_for(label) + + is_numeric_or_boolean = isinstance(spark_type, (NumericType, BooleanType)) + keep_column = not numeric_only or is_numeric_or_boolean + + if keep_column: + if num_args == 1: + # Only pass in the column if sfun accepts only one arg + scol = sfun(spark_column) + else: # must be 2 + assert num_args == 2 + # Pass in both the column and its data type if sfun accepts two args + scol = sfun(spark_column, spark_type) + + if min_count > 0: + scol = F.when( + Frame._count_expr(spark_column, spark_type) >= min_count, scol + ) + + exprs.append(scol.alias(name_like_string(label))) + new_column_labels.append(label) + + if len(exprs) == 1: + return Series([]) + + sdf = self._internal.spark_frame.select(*exprs) + + # The data is expected to be small so it's fine to transpose/use default index. + with pp.option_context("compute.max_rows", 1): + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, SPARK_DEFAULT_INDEX_NAME)], + column_labels=new_column_labels, + column_label_names=self._internal.column_label_names, + ) + return first_series(DataFrame(internal).transpose()) + + else: + # Here we execute with the first 1000 to get the return type. + # If the records were less than 1000, it uses pandas API directly for a shortcut. + limit = get_option("compute.shortcut_limit") + pdf = self.head(limit + 1)._to_internal_pandas() + pser = getattr(pdf, name)(axis=axis, numeric_only=numeric_only, **kwargs) + if len(pdf) <= limit: + return Series(pser) + + @pandas_udf(returnType=as_spark_type(pser.dtype.type)) + def calculate_columns_axis(*cols): + return getattr(pd.concat(cols, axis=1), name)( + axis=axis, numeric_only=numeric_only, **kwargs + ) + + column_name = verify_temp_column_name( + self._internal.spark_frame.select(self._internal.index_spark_columns), + "__calculate_columns_axis__", + ) + sdf = self._internal.spark_frame.select( + self._internal.index_spark_columns + + [calculate_columns_axis(*self._internal.data_spark_columns).alias(column_name)] + ) + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + return first_series(DataFrame(internal)).rename(pser.name) + + def _kser_for(self, label): + """ + Create Series with a proper column label. + + The given label must be verified to exist in `InternalFrame.column_labels`. + + For example, in some method, self is like: + + >>> self = pp.range(3) + + `self._kser_for(label)` can be used with `InternalFrame.column_labels`: + + >>> self._kser_for(self._internal.column_labels[0]) + 0 0 + 1 1 + 2 2 + Name: id, dtype: int64 + + `self._kser_for(label)` must not be used directly with user inputs. + In that case, `self[label]` should be used instead, which checks the label exists or not: + + >>> self['id'] + 0 0 + 1 1 + 2 2 + Name: id, dtype: int64 + """ + return self._ksers[label] + + def _apply_series_op(self, op, should_resolve: bool = False): + applied = [] + for label in self._internal.column_labels: + applied.append(op(self._kser_for(label))) + internal = self._internal.with_new_columns(applied) + if should_resolve: + internal = internal.resolved_copy + return DataFrame(internal) + + # Arithmetic Operators + def _map_series_op(self, op, other): + from pyspark.pandas.base import IndexOpsMixin + + if not isinstance(other, DataFrame) and ( + isinstance(other, IndexOpsMixin) or is_sequence(other) + ): + raise ValueError( + "%s with a sequence is currently not supported; " + "however, got %s." % (op, type(other).__name__) + ) + + if isinstance(other, DataFrame): + if self._internal.column_labels_level != other._internal.column_labels_level: + raise ValueError("cannot join with no overlapping index names") + + if not same_anchor(self, other): + # Different DataFrames + def apply_op(kdf, this_column_labels, that_column_labels): + for this_label, that_label in zip(this_column_labels, that_column_labels): + yield ( + getattr(kdf._kser_for(this_label), op)( + kdf._kser_for(that_label) + ).rename(this_label), + this_label, + ) + + return align_diff_frames(apply_op, self, other, fillna=True, how="full") + else: + applied = [] + column_labels = [] + for label in self._internal.column_labels: + if label in other._internal.column_labels: + applied.append(getattr(self._kser_for(label), op)(other._kser_for(label))) + else: + applied.append( + F.lit(None) + .cast(self._internal.spark_type_for(label)) + .alias(name_like_string(label)) + ) + column_labels.append(label) + for label in other._internal.column_labels: + if label not in column_labels: + applied.append( + F.lit(None) + .cast(other._internal.spark_type_for(label)) + .alias(name_like_string(label)) + ) + column_labels.append(label) + internal = self._internal.with_new_columns(applied, column_labels=column_labels) + return DataFrame(internal) + else: + return self._apply_series_op(lambda kser: getattr(kser, op)(other)) + + def __add__(self, other) -> "DataFrame": + return self._map_series_op("add", other) + + def __radd__(self, other) -> "DataFrame": + return self._map_series_op("radd", other) + + def __div__(self, other) -> "DataFrame": + return self._map_series_op("div", other) + + def __rdiv__(self, other) -> "DataFrame": + return self._map_series_op("rdiv", other) + + def __truediv__(self, other) -> "DataFrame": + return self._map_series_op("truediv", other) + + def __rtruediv__(self, other) -> "DataFrame": + return self._map_series_op("rtruediv", other) + + def __mul__(self, other) -> "DataFrame": + return self._map_series_op("mul", other) + + def __rmul__(self, other) -> "DataFrame": + return self._map_series_op("rmul", other) + + def __sub__(self, other) -> "DataFrame": + return self._map_series_op("sub", other) + + def __rsub__(self, other) -> "DataFrame": + return self._map_series_op("rsub", other) + + def __pow__(self, other) -> "DataFrame": + return self._map_series_op("pow", other) + + def __rpow__(self, other) -> "DataFrame": + return self._map_series_op("rpow", other) + + def __mod__(self, other) -> "DataFrame": + return self._map_series_op("mod", other) + + def __rmod__(self, other) -> "DataFrame": + return self._map_series_op("rmod", other) + + def __floordiv__(self, other) -> "DataFrame": + return self._map_series_op("floordiv", other) + + def __rfloordiv__(self, other) -> "DataFrame": + return self._map_series_op("rfloordiv", other) + + def __abs__(self) -> "DataFrame": + return self._apply_series_op(lambda kser: abs(kser)) + + def __neg__(self) -> "DataFrame": + return self._apply_series_op(lambda kser: -kser) + + def add(self, other) -> "DataFrame": + return self + other + + # create accessor for plot + plot = CachedAccessor("plot", KoalasPlotAccessor) + + # create accessor for Spark related methods. + spark = CachedAccessor("spark", SparkFrameMethods) + + # create accessor for Koalas specific methods. + koalas = CachedAccessor("koalas", KoalasFrameMethods) + + def hist(self, bins=10, **kwds): + return self.plot.hist(bins, **kwds) + + hist.__doc__ = KoalasPlotAccessor.hist.__doc__ + + def kde(self, bw_method=None, ind=None, **kwds): + return self.plot.kde(bw_method, ind, **kwds) + + kde.__doc__ = KoalasPlotAccessor.kde.__doc__ + + add.__doc__ = _flex_doc_FRAME.format( + desc="Addition", op_name="+", equiv="dataframe + other", reverse="radd" + ) + + def radd(self, other) -> "DataFrame": + return other + self + + radd.__doc__ = _flex_doc_FRAME.format( + desc="Addition", op_name="+", equiv="other + dataframe", reverse="add" + ) + + def div(self, other) -> "DataFrame": + return self / other + + div.__doc__ = _flex_doc_FRAME.format( + desc="Floating division", op_name="/", equiv="dataframe / other", reverse="rdiv" + ) + + divide = div + + def rdiv(self, other) -> "DataFrame": + return other / self + + rdiv.__doc__ = _flex_doc_FRAME.format( + desc="Floating division", op_name="/", equiv="other / dataframe", reverse="div" + ) + + def truediv(self, other) -> "DataFrame": + return self / other + + truediv.__doc__ = _flex_doc_FRAME.format( + desc="Floating division", op_name="/", equiv="dataframe / other", reverse="rtruediv" + ) + + def rtruediv(self, other) -> "DataFrame": + return other / self + + rtruediv.__doc__ = _flex_doc_FRAME.format( + desc="Floating division", op_name="/", equiv="other / dataframe", reverse="truediv" + ) + + def mul(self, other) -> "DataFrame": + return self * other + + mul.__doc__ = _flex_doc_FRAME.format( + desc="Multiplication", op_name="*", equiv="dataframe * other", reverse="rmul" + ) + + multiply = mul + + def rmul(self, other) -> "DataFrame": + return other * self + + rmul.__doc__ = _flex_doc_FRAME.format( + desc="Multiplication", op_name="*", equiv="other * dataframe", reverse="mul" + ) + + def sub(self, other) -> "DataFrame": + return self - other + + sub.__doc__ = _flex_doc_FRAME.format( + desc="Subtraction", op_name="-", equiv="dataframe - other", reverse="rsub" + ) + + subtract = sub + + def rsub(self, other) -> "DataFrame": + return other - self + + rsub.__doc__ = _flex_doc_FRAME.format( + desc="Subtraction", op_name="-", equiv="other - dataframe", reverse="sub" + ) + + def mod(self, other) -> "DataFrame": + return self % other + + mod.__doc__ = _flex_doc_FRAME.format( + desc="Modulo", op_name="%", equiv="dataframe % other", reverse="rmod" + ) + + def rmod(self, other) -> "DataFrame": + return other % self + + rmod.__doc__ = _flex_doc_FRAME.format( + desc="Modulo", op_name="%", equiv="other % dataframe", reverse="mod" + ) + + def pow(self, other) -> "DataFrame": + return self ** other + + pow.__doc__ = _flex_doc_FRAME.format( + desc="Exponential power of series", op_name="**", equiv="dataframe ** other", reverse="rpow" + ) + + def rpow(self, other) -> "DataFrame": + return other ** self + + rpow.__doc__ = _flex_doc_FRAME.format( + desc="Exponential power", op_name="**", equiv="other ** dataframe", reverse="pow" + ) + + def floordiv(self, other) -> "DataFrame": + return self // other + + floordiv.__doc__ = _flex_doc_FRAME.format( + desc="Integer division", op_name="//", equiv="dataframe // other", reverse="rfloordiv" + ) + + def rfloordiv(self, other) -> "DataFrame": + return other // self + + rfloordiv.__doc__ = _flex_doc_FRAME.format( + desc="Integer division", op_name="//", equiv="other // dataframe", reverse="floordiv" + ) + + # Comparison Operators + def __eq__(self, other) -> "DataFrame": # type: ignore + return self._map_series_op("eq", other) + + def __ne__(self, other) -> "DataFrame": # type: ignore + return self._map_series_op("ne", other) + + def __lt__(self, other) -> "DataFrame": + return self._map_series_op("lt", other) + + def __le__(self, other) -> "DataFrame": + return self._map_series_op("le", other) + + def __ge__(self, other) -> "DataFrame": + return self._map_series_op("ge", other) + + def __gt__(self, other) -> "DataFrame": + return self._map_series_op("gt", other) + + def eq(self, other) -> "DataFrame": + """ + Compare if the current value is equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.eq(1) + a b + a True True + b False False + c False True + d False False + """ + return self == other + + equals = eq + + def gt(self, other) -> "DataFrame": + """ + Compare if the current value is greater than the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.gt(2) + a b + a False False + b False False + c True False + d True False + """ + return self > other + + def ge(self, other) -> "DataFrame": + """ + Compare if the current value is greater than or equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.ge(1) + a b + a True True + b True False + c True True + d True False + """ + return self >= other + + def lt(self, other) -> "DataFrame": + """ + Compare if the current value is less than the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.lt(1) + a b + a False False + b False False + c False False + d False False + """ + return self < other + + def le(self, other) -> "DataFrame": + """ + Compare if the current value is less than or equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.le(2) + a b + a True True + b True False + c False True + d False False + """ + return self <= other + + def ne(self, other) -> "DataFrame": + """ + Compare if the current value is not equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.ne(1) + a b + a False False + b True True + c True False + d True True + """ + return self != other + + def applymap(self, func) -> "DataFrame": + """ + Apply a function to a Dataframe elementwise. + + This method applies a function that accepts and returns a scalar + to every element of a DataFrame. + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def square(x) -> np.int32: + ... return x ** 2 + + Koalas uses return type hint and does not try to infer the type. + + Parameters + ---------- + func : callable + Python function, returns a single value from a single value. + + Returns + ------- + DataFrame + Transformed DataFrame. + + Examples + -------- + >>> df = pp.DataFrame([[1, 2.12], [3.356, 4.567]]) + >>> df + 0 1 + 0 1.000 2.120 + 1 3.356 4.567 + + >>> def str_len(x) -> int: + ... return len(str(x)) + >>> df.applymap(str_len) + 0 1 + 0 3 4 + 1 5 5 + + >>> def power(x) -> float: + ... return x ** 2 + >>> df.applymap(power) + 0 1 + 0 1.000000 4.494400 + 1 11.262736 20.857489 + + You can omit the type hint and let Koalas infer its type. + + >>> df.applymap(lambda x: x ** 2) + 0 1 + 0 1.000000 4.494400 + 1 11.262736 20.857489 + """ + + # TODO: We can implement shortcut theoretically since it creates new DataFrame + # anyway and we don't have to worry about operations on different DataFrames. + return self._apply_series_op(lambda kser: kser.apply(func)) + + # TODO: not all arguments are implemented comparing to pandas' for now. + def aggregate( + self, func: Union[List[str], Dict[Any, List[str]]] + ) -> Union["Series", "DataFrame", "Index"]: + """Aggregate using one or more operations over the specified axis. + + Parameters + ---------- + func : dict or a list + a dict mapping from column name (string) to + aggregate functions (list of strings). + If a list is given, the aggregation is performed against + all columns. + + Returns + ------- + DataFrame + + Notes + ----- + `agg` is an alias for `aggregate`. Use the alias. + + See Also + -------- + DataFrame.apply : Invoke function on DataFrame. + DataFrame.transform : Only perform transforming type operations. + DataFrame.groupby : Perform operations over groups. + Series.aggregate : The equivalent function for Series. + + Examples + -------- + >>> df = pp.DataFrame([[1, 2, 3], + ... [4, 5, 6], + ... [7, 8, 9], + ... [np.nan, np.nan, np.nan]], + ... columns=['A', 'B', 'C']) + + >>> df + A B C + 0 1.0 2.0 3.0 + 1 4.0 5.0 6.0 + 2 7.0 8.0 9.0 + 3 NaN NaN NaN + + Aggregate these functions over the rows. + + >>> df.agg(['sum', 'min'])[['A', 'B', 'C']].sort_index() + A B C + min 1.0 2.0 3.0 + sum 12.0 15.0 18.0 + + Different aggregations per column. + + >>> df.agg({'A' : ['sum', 'min'], 'B' : ['min', 'max']})[['A', 'B']].sort_index() + A B + max NaN 8.0 + min 1.0 2.0 + sum 12.0 NaN + + For multi-index columns: + + >>> df.columns = pd.MultiIndex.from_tuples([("X", "A"), ("X", "B"), ("Y", "C")]) + >>> df.agg(['sum', 'min'])[[("X", "A"), ("X", "B"), ("Y", "C")]].sort_index() + X Y + A B C + min 1.0 2.0 3.0 + sum 12.0 15.0 18.0 + + >>> aggregated = df.agg({("X", "A") : ['sum', 'min'], ("X", "B") : ['min', 'max']}) + >>> aggregated[[("X", "A"), ("X", "B")]].sort_index() # doctest: +NORMALIZE_WHITESPACE + X + A B + max NaN 8.0 + min 1.0 2.0 + sum 12.0 NaN + """ + from pyspark.pandas.groupby import GroupBy + + if isinstance(func, list): + if all((isinstance(f, str) for f in func)): + func = dict([(column, func) for column in self.columns]) + else: + raise ValueError( + "If the given function is a list, it " + "should only contains function names as strings." + ) + + if not isinstance(func, dict) or not all( + is_name_like_value(key) + and ( + isinstance(value, str) + or (isinstance(value, list) and all(isinstance(v, str) for v in value)) + ) + for key, value in func.items() + ): + raise ValueError( + "aggs must be a dict mapping from column name to aggregate " + "functions (string or list of strings)." + ) + + with option_context("compute.default_index_type", "distributed"): + kdf = DataFrame(GroupBy._spark_groupby(self, func)) # type: DataFrame + + # The codes below basically converts: + # + # A B + # sum min min max + # 0 12.0 1.0 2.0 8.0 + # + # to: + # A B + # max NaN 8.0 + # min 1.0 2.0 + # sum 12.0 NaN + # + # Aggregated output is usually pretty much small. + + if LooseVersion(pyspark.__version__) >= LooseVersion("2.4"): + return kdf.stack().droplevel(0)[list(func.keys())] + else: + pdf = kdf._to_internal_pandas().stack() + pdf.index = pdf.index.droplevel() + return pp.from_pandas(pdf[list(func.keys())]) + + agg = aggregate + + def corr(self, method="pearson") -> Union["Series", "DataFrame", "Index"]: + """ + Compute pairwise correlation of columns, excluding NA/null values. + + Parameters + ---------- + method : {'pearson', 'spearman'} + * pearson : standard correlation coefficient + * spearman : Spearman rank correlation + + Returns + ------- + y : DataFrame + + See Also + -------- + Series.corr + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats']) + >>> df.corr('pearson') + dogs cats + dogs 1.000000 -0.851064 + cats -0.851064 1.000000 + + >>> df.corr('spearman') + dogs cats + dogs 1.000000 -0.948683 + cats -0.948683 1.000000 + + Notes + ----- + There are behavior differences between Koalas and pandas. + + * the `method` argument only accepts 'pearson', 'spearman' + * the data should not contain NaNs. Koalas will return an error. + * Koalas doesn't support the following argument(s). + + * `min_periods` argument is not supported + """ + return pp.from_pandas(corr(self, method)) + + def iteritems(self) -> Iterator: + """ + Iterator over (column name, Series) pairs. + + Iterates over the DataFrame columns, returning a tuple with + the column name and the content as a Series. + + Returns + ------- + label : object + The column names for the DataFrame being iterated over. + content : Series + The column entries belonging to each label, as a Series. + + Examples + -------- + >>> df = pp.DataFrame({'species': ['bear', 'bear', 'marsupial'], + ... 'population': [1864, 22000, 80000]}, + ... index=['panda', 'polar', 'koala'], + ... columns=['species', 'population']) + >>> df + species population + panda bear 1864 + polar bear 22000 + koala marsupial 80000 + + >>> for label, content in df.iteritems(): + ... print('label:', label) + ... print('content:', content.to_string()) + ... + label: species + content: panda bear + polar bear + koala marsupial + label: population + content: panda 1864 + polar 22000 + koala 80000 + """ + return ( + (label if len(label) > 1 else label[0], self._kser_for(label)) + for label in self._internal.column_labels + ) + + def iterrows(self) -> Iterator: + """ + Iterate over DataFrame rows as (index, Series) pairs. + + Yields + ------ + index : label or tuple of label + The index of the row. A tuple for a `MultiIndex`. + data : pandas.Series + The data of the row as a Series. + + it : generator + A generator that iterates over the rows of the frame. + + Notes + ----- + + 1. Because ``iterrows`` returns a Series for each row, + it does **not** preserve dtypes across the rows (dtypes are + preserved across columns for DataFrames). For example, + + >>> df = pp.DataFrame([[1, 1.5]], columns=['int', 'float']) + >>> row = next(df.iterrows())[1] + >>> row + int 1.0 + float 1.5 + Name: 0, dtype: float64 + >>> print(row['int'].dtype) + float64 + >>> print(df['int'].dtype) + int64 + + To preserve dtypes while iterating over the rows, it is better + to use :meth:`itertuples` which returns namedtuples of the values + and which is generally faster than ``iterrows``. + + 2. You should **never modify** something you are iterating over. + This is not guaranteed to work in all cases. Depending on the + data types, the iterator returns a copy and not a view, and writing + to it will have no effect. + """ + + columns = self.columns + internal_index_columns = self._internal.index_spark_column_names + internal_data_columns = self._internal.data_spark_column_names + + def extract_kv_from_spark_row(row): + k = ( + row[internal_index_columns[0]] + if len(internal_index_columns) == 1 + else tuple(row[c] for c in internal_index_columns) + ) + v = [row[c] for c in internal_data_columns] + return k, v + + for k, v in map( + extract_kv_from_spark_row, self._internal.resolved_copy.spark_frame.toLocalIterator() + ): + s = pd.Series(v, index=columns, name=k) + yield k, s + + def itertuples(self, index: bool = True, name: Optional[str] = "Koalas") -> Iterator: + """ + Iterate over DataFrame rows as namedtuples. + + Parameters + ---------- + index : bool, default True + If True, return the index as the first element of the tuple. + name : str or None, default "Koalas" + The name of the returned namedtuples or None to return regular + tuples. + + Returns + ------- + iterator + An object to iterate over namedtuples for each row in the + DataFrame with the first field possibly being the index and + following fields being the column values. + + See Also + -------- + DataFrame.iterrows : Iterate over DataFrame rows as (index, Series) + pairs. + DataFrame.items : Iterate over (column name, Series) pairs. + + Notes + ----- + The column names will be renamed to positional names if they are + invalid Python identifiers, repeated, or start with an underscore. + On python versions < 3.7 regular tuples are returned for DataFrames + with a large number of columns (>254). + + Examples + -------- + >>> df = pp.DataFrame({'num_legs': [4, 2], 'num_wings': [0, 2]}, + ... index=['dog', 'hawk']) + >>> df + num_legs num_wings + dog 4 0 + hawk 2 2 + + >>> for row in df.itertuples(): + ... print(row) + ... + Koalas(Index='dog', num_legs=4, num_wings=0) + Koalas(Index='hawk', num_legs=2, num_wings=2) + + By setting the `index` parameter to False we can remove the index + as the first element of the tuple: + + >>> for row in df.itertuples(index=False): + ... print(row) + ... + Koalas(num_legs=4, num_wings=0) + Koalas(num_legs=2, num_wings=2) + + With the `name` parameter set we set a custom name for the yielded + namedtuples: + + >>> for row in df.itertuples(name='Animal'): + ... print(row) + ... + Animal(Index='dog', num_legs=4, num_wings=0) + Animal(Index='hawk', num_legs=2, num_wings=2) + """ + fields = list(self.columns) + if index: + fields.insert(0, "Index") + + index_spark_column_names = self._internal.index_spark_column_names + data_spark_column_names = self._internal.data_spark_column_names + + def extract_kv_from_spark_row(row): + k = ( + row[index_spark_column_names[0]] + if len(index_spark_column_names) == 1 + else tuple(row[c] for c in index_spark_column_names) + ) + v = [row[c] for c in data_spark_column_names] + return k, v + + can_return_named_tuples = sys.version_info >= (3, 7) or len(self.columns) + index < 255 + + if name is not None and can_return_named_tuples: + itertuple = namedtuple(name, fields, rename=True) # type: ignore + for k, v in map( + extract_kv_from_spark_row, + self._internal.resolved_copy.spark_frame.toLocalIterator(), + ): + yield itertuple._make(([k] if index else []) + list(v)) + else: + for k, v in map( + extract_kv_from_spark_row, + self._internal.resolved_copy.spark_frame.toLocalIterator(), + ): + yield tuple(([k] if index else []) + list(v)) + + def items(self) -> Iterator: + """This is an alias of ``iteritems``.""" + return self.iteritems() + + def to_clipboard(self, excel=True, sep=None, **kwargs) -> None: + """ + Copy object to the system clipboard. + + Write a text representation of object to the system clipboard. + This can be pasted into Excel, for example. + + .. note:: This method should only be used if the resulting DataFrame is expected + to be small, as all the data is loaded into the driver's memory. + + Parameters + ---------- + excel : bool, default True + - True, use the provided separator, writing in a csv format for + allowing easy pasting into excel. + - False, write a string representation of the object to the + clipboard. + + sep : str, default ``'\\t'`` + Field delimiter. + **kwargs + These parameters will be passed to DataFrame.to_csv. + + Notes + ----- + Requirements for your platform. + + - Linux : `xclip`, or `xsel` (with `gtk` or `PyQt4` modules) + - Windows : none + - OS X : none + + See Also + -------- + read_clipboard : Read text from clipboard. + + Examples + -------- + Copy the contents of a DataFrame to the clipboard. + + >>> df = pp.DataFrame([[1, 2, 3], [4, 5, 6]], columns=['A', 'B', 'C']) # doctest: +SKIP + >>> df.to_clipboard(sep=',') # doctest: +SKIP + ... # Wrote the following to the system clipboard: + ... # ,A,B,C + ... # 0,1,2,3 + ... # 1,4,5,6 + + We can omit the index by passing the keyword `index` and setting + it to false. + + >>> df.to_clipboard(sep=',', index=False) # doctest: +SKIP + ... # Wrote the following to the system clipboard: + ... # A,B,C + ... # 1,2,3 + ... # 4,5,6 + + This function also works for Series: + + >>> df = pp.Series([1, 2, 3, 4, 5, 6, 7], name='x') # doctest: +SKIP + >>> df.to_clipboard(sep=',') # doctest: +SKIP + ... # Wrote the following to the system clipboard: + ... # 0, 1 + ... # 1, 2 + ... # 2, 3 + ... # 3, 4 + ... # 4, 5 + ... # 5, 6 + ... # 6, 7 + """ + + args = locals() + kdf = self + return validate_arguments_and_invoke_function( + kdf._to_internal_pandas(), self.to_clipboard, pd.DataFrame.to_clipboard, args + ) + + def to_html( + self, + buf=None, + columns=None, + col_space=None, + header=True, + index=True, + na_rep="NaN", + formatters=None, + float_format=None, + sparsify=None, + index_names=True, + justify=None, + max_rows=None, + max_cols=None, + show_dimensions=False, + decimal=".", + bold_rows=True, + classes=None, + escape=True, + notebook=False, + border=None, + table_id=None, + render_links=False, + ) -> Optional[str]: + """ + Render a DataFrame as an HTML table. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. If the input + is large, set max_rows parameter. + + Parameters + ---------- + buf : StringIO-like, optional + Buffer to write to. + columns : sequence, optional, default None + The subset of columns to write. Writes all columns by default. + col_space : int, optional + The minimum width of each column. + header : bool, optional + Write out the column names. If a list of strings is given, it + is assumed to be aliases for the column names + index : bool, optional, default True + Whether to print index (row) labels. + na_rep : str, optional, default 'NaN' + String representation of NAN to use. + formatters : list or dict of one-param. functions, optional + Formatter functions to apply to columns' elements by position or + name. + The result of each function must be a unicode string. + List must be of length equal to the number of columns. + float_format : one-parameter function, optional, default None + Formatter function to apply to columns' elements if they are + floats. The result of this function must be a unicode string. + sparsify : bool, optional, default True + Set to False for a DataFrame with a hierarchical index to print + every multiindex key at each row. + index_names : bool, optional, default True + Prints the names of the indexes. + justify : str, default None + How to justify the column labels. If None uses the option from + the print configuration (controlled by set_option), 'right' out + of the box. Valid values are + + * left + * right + * center + * justify + * justify-all + * start + * end + * inherit + * match-parent + * initial + * unset. + max_rows : int, optional + Maximum number of rows to display in the console. + max_cols : int, optional + Maximum number of columns to display in the console. + show_dimensions : bool, default False + Display DataFrame dimensions (number of rows by number of columns). + decimal : str, default '.' + Character recognized as decimal separator, e.g. ',' in Europe. + bold_rows : bool, default True + Make the row labels bold in the output. + classes : str or list or tuple, default None + CSS class(es) to apply to the resulting html table. + escape : bool, default True + Convert the characters <, >, and & to HTML-safe sequences. + notebook : {True, False}, default False + Whether the generated HTML is for IPython Notebook. + border : int + A ``border=border`` attribute is included in the opening + `` tag. Default ``pd.options.html.border``. + table_id : str, optional + A css id is included in the opening `
` tag if specified. + render_links : bool, default False + Convert URLs to HTML links (only works with pandas 0.24+). + + Returns + ------- + str (or unicode, depending on data and options) + String representation of the dataframe. + + See Also + -------- + to_string : Convert DataFrame to a string. + """ + # Make sure locals() call is at the top of the function so we don't capture local variables. + args = locals() + if max_rows is not None: + kdf = self.head(max_rows) + else: + kdf = self + + return validate_arguments_and_invoke_function( + kdf._to_internal_pandas(), self.to_html, pd.DataFrame.to_html, args + ) + + def to_string( + self, + buf=None, + columns=None, + col_space=None, + header=True, + index=True, + na_rep="NaN", + formatters=None, + float_format=None, + sparsify=None, + index_names=True, + justify=None, + max_rows=None, + max_cols=None, + show_dimensions=False, + decimal=".", + line_width=None, + ) -> Optional[str]: + """ + Render a DataFrame to a console-friendly tabular output. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. If the input + is large, set max_rows parameter. + + Parameters + ---------- + buf : StringIO-like, optional + Buffer to write to. + columns : sequence, optional, default None + The subset of columns to write. Writes all columns by default. + col_space : int, optional + The minimum width of each column. + header : bool, optional + Write out the column names. If a list of strings is given, it + is assumed to be aliases for the column names + index : bool, optional, default True + Whether to print index (row) labels. + na_rep : str, optional, default 'NaN' + String representation of NAN to use. + formatters : list or dict of one-param. functions, optional + Formatter functions to apply to columns' elements by position or + name. + The result of each function must be a unicode string. + List must be of length equal to the number of columns. + float_format : one-parameter function, optional, default None + Formatter function to apply to columns' elements if they are + floats. The result of this function must be a unicode string. + sparsify : bool, optional, default True + Set to False for a DataFrame with a hierarchical index to print + every multiindex key at each row. + index_names : bool, optional, default True + Prints the names of the indexes. + justify : str, default None + How to justify the column labels. If None uses the option from + the print configuration (controlled by set_option), 'right' out + of the box. Valid values are + + * left + * right + * center + * justify + * justify-all + * start + * end + * inherit + * match-parent + * initial + * unset. + max_rows : int, optional + Maximum number of rows to display in the console. + max_cols : int, optional + Maximum number of columns to display in the console. + show_dimensions : bool, default False + Display DataFrame dimensions (number of rows by number of columns). + decimal : str, default '.' + Character recognized as decimal separator, e.g. ',' in Europe. + line_width : int, optional + Width to wrap a line in characters. + + Returns + ------- + str (or unicode, depending on data and options) + String representation of the dataframe. + + See Also + -------- + to_html : Convert DataFrame to HTML. + + Examples + -------- + >>> df = pp.DataFrame({'col1': [1, 2, 3], 'col2': [4, 5, 6]}, columns=['col1', 'col2']) + >>> print(df.to_string()) + col1 col2 + 0 1 4 + 1 2 5 + 2 3 6 + + >>> print(df.to_string(max_rows=2)) + col1 col2 + 0 1 4 + 1 2 5 + """ + # Make sure locals() call is at the top of the function so we don't capture local variables. + args = locals() + if max_rows is not None: + kdf = self.head(max_rows) + else: + kdf = self + + return validate_arguments_and_invoke_function( + kdf._to_internal_pandas(), self.to_string, pd.DataFrame.to_string, args + ) + + def to_dict(self, orient="dict", into=dict) -> Union[List, Mapping]: + """ + Convert the DataFrame to a dictionary. + + The type of the key-value pairs can be customized with the parameters + (see below). + + .. note:: This method should only be used if the resulting pandas DataFrame is expected + to be small, as all the data is loaded into the driver's memory. + + Parameters + ---------- + orient : str {'dict', 'list', 'series', 'split', 'records', 'index'} + Determines the type of the values of the dictionary. + + - 'dict' (default) : dict like {column -> {index -> value}} + - 'list' : dict like {column -> [values]} + - 'series' : dict like {column -> Series(values)} + - 'split' : dict like + {'index' -> [index], 'columns' -> [columns], 'data' -> [values]} + - 'records' : list like + [{column -> value}, ... , {column -> value}] + - 'index' : dict like {index -> {column -> value}} + + Abbreviations are allowed. `s` indicates `series` and `sp` + indicates `split`. + + into : class, default dict + The collections.abc.Mapping subclass used for all Mappings + in the return value. Can be the actual class or an empty + instance of the mapping type you want. If you want a + collections.defaultdict, you must pass it initialized. + + Returns + ------- + dict, list or collections.abc.Mapping + Return a collections.abc.Mapping object representing the DataFrame. + The resulting transformation depends on the `orient` parameter. + + Examples + -------- + >>> df = pp.DataFrame({'col1': [1, 2], + ... 'col2': [0.5, 0.75]}, + ... index=['row1', 'row2'], + ... columns=['col1', 'col2']) + >>> df + col1 col2 + row1 1 0.50 + row2 2 0.75 + + >>> df_dict = df.to_dict() + >>> sorted([(key, sorted(values.items())) for key, values in df_dict.items()]) + [('col1', [('row1', 1), ('row2', 2)]), ('col2', [('row1', 0.5), ('row2', 0.75)])] + + You can specify the return orientation. + + >>> df_dict = df.to_dict('series') + >>> sorted(df_dict.items()) + [('col1', row1 1 + row2 2 + Name: col1, dtype: int64), ('col2', row1 0.50 + row2 0.75 + Name: col2, dtype: float64)] + + >>> df_dict = df.to_dict('split') + >>> sorted(df_dict.items()) # doctest: +ELLIPSIS + [('columns', ['col1', 'col2']), ('data', [[1..., 0.75]]), ('index', ['row1', 'row2'])] + + >>> df_dict = df.to_dict('records') + >>> [sorted(values.items()) for values in df_dict] # doctest: +ELLIPSIS + [[('col1', 1...), ('col2', 0.5)], [('col1', 2...), ('col2', 0.75)]] + + >>> df_dict = df.to_dict('index') + >>> sorted([(key, sorted(values.items())) for key, values in df_dict.items()]) + [('row1', [('col1', 1), ('col2', 0.5)]), ('row2', [('col1', 2), ('col2', 0.75)])] + + You can also specify the mapping type. + + >>> from collections import OrderedDict, defaultdict + >>> df.to_dict(into=OrderedDict) + OrderedDict([('col1', OrderedDict([('row1', 1), ('row2', 2)])), \ +('col2', OrderedDict([('row1', 0.5), ('row2', 0.75)]))]) + + If you want a `defaultdict`, you need to initialize it: + + >>> dd = defaultdict(list) + >>> df.to_dict('records', into=dd) # doctest: +ELLIPSIS + [defaultdict(, {'col..., 'col...}), \ +defaultdict(, {'col..., 'col...})] + """ + # Make sure locals() call is at the top of the function so we don't capture local variables. + args = locals() + kdf = self + return validate_arguments_and_invoke_function( + kdf._to_internal_pandas(), self.to_dict, pd.DataFrame.to_dict, args + ) + + def to_latex( + self, + buf=None, + columns=None, + col_space=None, + header=True, + index=True, + na_rep="NaN", + formatters=None, + float_format=None, + sparsify=None, + index_names=True, + bold_rows=False, + column_format=None, + longtable=None, + escape=None, + encoding=None, + decimal=".", + multicolumn=None, + multicolumn_format=None, + multirow=None, + ) -> Optional[str]: + r""" + Render an object to a LaTeX tabular environment table. + + Render an object to a tabular environment table. You can splice this into a LaTeX + document. Requires usepackage{booktabs}. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. If the input + is large, consider alternative formats. + + Parameters + ---------- + buf : file descriptor or None + Buffer to write to. If None, the output is returned as a string. + columns : list of label, optional + The subset of columns to write. Writes all columns by default. + col_space : int, optional + The minimum width of each column. + header : bool or list of str, default True + Write out the column names. If a list of strings is given, it is assumed to be aliases + for the column names. + index : bool, default True + Write row names (index). + na_rep : str, default ‘NaN’ + Missing data representation. + formatters : list of functions or dict of {str: function}, optional + Formatter functions to apply to columns’ elements by position or name. The result of + each function must be a unicode string. List must be of length equal to the number of + columns. + float_format : str, optional + Format string for floating point numbers. + sparsify : bool, optional + Set to False for a DataFrame with a hierarchical index to print every multiindex key at + each row. By default, the value will be read from the config module. + index_names : bool, default True + Prints the names of the indexes. + bold_rows : bool, default False + Make the row labels bold in the output. + column_format : str, optional + The columns format as specified in LaTeX table format e.g. ‘rcl’ for 3 columns. By + default, ‘l’ will be used for all columns except columns of numbers, which default + to ‘r’. + longtable : bool, optional + By default, the value will be read from the pandas config module. Use a longtable + environment instead of tabular. Requires adding a usepackage{longtable} to your LaTeX + preamble. + escape : bool, optional + By default, the value will be read from the pandas config module. When set to False + prevents from escaping latex special characters in column names. + encoding : str, optional + A string representing the encoding to use in the output file, defaults to ‘ascii’ on + Python 2 and ‘utf-8’ on Python 3. + decimal : str, default ‘.’ + Character recognized as decimal separator, e.g. ‘,’ in Europe. + multicolumn : bool, default True + Use multicolumn to enhance MultiIndex columns. The default will be read from the config + module. + multicolumn_format : str, default ‘l’ + The alignment for multicolumns, similar to column_format The default will be read from + the config module. + multirow : bool, default False + Use multirow to enhance MultiIndex rows. Requires adding a usepackage{multirow} to your + LaTeX preamble. Will print centered labels (instead of top-aligned) across the contained + rows, separating groups via clines. The default will be read from the pandas config + module. + + Returns + ------- + str or None + If buf is None, returns the resulting LateX format as a string. Otherwise returns None. + + See Also + -------- + DataFrame.to_string : Render a DataFrame to a console-friendly + tabular output. + DataFrame.to_html : Render a DataFrame as an HTML table. + + + Examples + -------- + >>> df = pp.DataFrame({'name': ['Raphael', 'Donatello'], + ... 'mask': ['red', 'purple'], + ... 'weapon': ['sai', 'bo staff']}, + ... columns=['name', 'mask', 'weapon']) + >>> print(df.to_latex(index=False)) # doctest: +NORMALIZE_WHITESPACE + \begin{tabular}{lll} + \toprule + name & mask & weapon \\ + \midrule + Raphael & red & sai \\ + Donatello & purple & bo staff \\ + \bottomrule + \end{tabular} + + """ + + args = locals() + kdf = self + return validate_arguments_and_invoke_function( + kdf._to_internal_pandas(), self.to_latex, pd.DataFrame.to_latex, args + ) + + # TODO: enable doctests once we drop Spark 2.3.x (due to type coercion logic + # when creating arrays) + def transpose(self) -> "DataFrame": + """ + Transpose index and columns. + + Reflect the DataFrame over its main diagonal by writing rows as columns + and vice-versa. The property :attr:`.T` is an accessor to the method + :meth:`transpose`. + + .. note:: This method is based on an expensive operation due to the nature + of big data. Internally it needs to generate each row for each value, and + then group twice - it is a huge operation. To prevent misusage, this method + has the 'compute.max_rows' default limit of input length, and raises a ValueError. + + >>> from pyspark.pandas.config import option_context + >>> with option_context('compute.max_rows', 1000): # doctest: +NORMALIZE_WHITESPACE + ... pp.DataFrame({'a': range(1001)}).transpose() + Traceback (most recent call last): + ... + ValueError: Current DataFrame has more then the given limit 1000 rows. + Please set 'compute.max_rows' by using 'pyspark.pandas.config.set_option' + to retrieve to retrieve more than 1000 rows. Note that, before changing the + 'compute.max_rows', this operation is considerably expensive. + + Returns + ------- + DataFrame + The transposed DataFrame. + + Notes + ----- + Transposing a DataFrame with mixed dtypes will result in a homogeneous + DataFrame with the coerced dtype. For instance, if int and float have + to be placed in same column, it becomes float. If type coercion is not + possible, it fails. + + Also, note that the values in index should be unique because they become + unique column names. + + In addition, if Spark 2.3 is used, the types should always be exactly same. + + Examples + -------- + **Square DataFrame with homogeneous dtype** + + >>> d1 = {'col1': [1, 2], 'col2': [3, 4]} + >>> df1 = pp.DataFrame(data=d1, columns=['col1', 'col2']) + >>> df1 + col1 col2 + 0 1 3 + 1 2 4 + + >>> df1_transposed = df1.T.sort_index() # doctest: +SKIP + >>> df1_transposed # doctest: +SKIP + 0 1 + col1 1 2 + col2 3 4 + + When the dtype is homogeneous in the original DataFrame, we get a + transposed DataFrame with the same dtype: + + >>> df1.dtypes + col1 int64 + col2 int64 + dtype: object + >>> df1_transposed.dtypes # doctest: +SKIP + 0 int64 + 1 int64 + dtype: object + + **Non-square DataFrame with mixed dtypes** + + >>> d2 = {'score': [9.5, 8], + ... 'kids': [0, 0], + ... 'age': [12, 22]} + >>> df2 = pp.DataFrame(data=d2, columns=['score', 'kids', 'age']) + >>> df2 + score kids age + 0 9.5 0 12 + 1 8.0 0 22 + + >>> df2_transposed = df2.T.sort_index() # doctest: +SKIP + >>> df2_transposed # doctest: +SKIP + 0 1 + age 12.0 22.0 + kids 0.0 0.0 + score 9.5 8.0 + + When the DataFrame has mixed dtypes, we get a transposed DataFrame with + the coerced dtype: + + >>> df2.dtypes + score float64 + kids int64 + age int64 + dtype: object + + >>> df2_transposed.dtypes # doctest: +SKIP + 0 float64 + 1 float64 + dtype: object + """ + max_compute_count = get_option("compute.max_rows") + if max_compute_count is not None: + pdf = self.head(max_compute_count + 1)._to_internal_pandas() + if len(pdf) > max_compute_count: + raise ValueError( + "Current DataFrame has more then the given limit {0} rows. " + "Please set 'compute.max_rows' by using 'pyspark.pandas.config.set_option' " + "to retrieve to retrieve more than {0} rows. Note that, before changing the " + "'compute.max_rows', this operation is considerably expensive.".format( + max_compute_count + ) + ) + return DataFrame(pdf.transpose()) + + # Explode the data to be pairs. + # + # For instance, if the current input DataFrame is as below: + # + # +------+------+------+------+------+ + # |index1|index2|(a,x1)|(a,x2)|(b,x3)| + # +------+------+------+------+------+ + # | y1| z1| 1| 0| 0| + # | y2| z2| 0| 50| 0| + # | y3| z3| 3| 2| 1| + # +------+------+------+------+------+ + # + # Output of `exploded_df` becomes as below: + # + # +-----------------+-----------------+-----------------+-----+ + # | index|__index_level_0__|__index_level_1__|value| + # +-----------------+-----------------+-----------------+-----+ + # |{"a":["y1","z1"]}| a| x1| 1| + # |{"a":["y1","z1"]}| a| x2| 0| + # |{"a":["y1","z1"]}| b| x3| 0| + # |{"a":["y2","z2"]}| a| x1| 0| + # |{"a":["y2","z2"]}| a| x2| 50| + # |{"a":["y2","z2"]}| b| x3| 0| + # |{"a":["y3","z3"]}| a| x1| 3| + # |{"a":["y3","z3"]}| a| x2| 2| + # |{"a":["y3","z3"]}| b| x3| 1| + # +-----------------+-----------------+-----------------+-----+ + pairs = F.explode( + F.array( + *[ + F.struct( + [ + F.lit(col).alias(SPARK_INDEX_NAME_FORMAT(i)) + for i, col in enumerate(label) + ] + + [self._internal.spark_column_for(label).alias("value")] + ) + for label in self._internal.column_labels + ] + ) + ) + + exploded_df = self._internal.spark_frame.withColumn("pairs", pairs).select( + [ + F.to_json( + F.struct( + F.array([scol for scol in self._internal.index_spark_columns]).alias("a") + ) + ).alias("index"), + F.col("pairs.*"), + ] + ) + + # After that, executes pivot with key and its index column. + # Note that index column should contain unique values since column names + # should be unique. + internal_index_columns = [ + SPARK_INDEX_NAME_FORMAT(i) for i in range(self._internal.column_labels_level) + ] + pivoted_df = exploded_df.groupBy(internal_index_columns).pivot("index") + + transposed_df = pivoted_df.agg(F.first(F.col("value"))) + + new_data_columns = list( + filter(lambda x: x not in internal_index_columns, transposed_df.columns) + ) + + column_labels = [ + None if len(label) == 1 and label[0] is None else label + for label in (tuple(json.loads(col)["a"]) for col in new_data_columns) + ] + + internal = InternalFrame( + spark_frame=transposed_df, + index_spark_columns=[scol_for(transposed_df, col) for col in internal_index_columns], + index_names=self._internal.column_label_names, + column_labels=column_labels, + data_spark_columns=[scol_for(transposed_df, col) for col in new_data_columns], + column_label_names=self._internal.index_names, + ) + + return DataFrame(internal) + + T = property(transpose) + + def apply_batch(self, func, args=(), **kwds) -> "DataFrame": + warnings.warn( + "DataFrame.apply_batch is deprecated as of DataFrame.koalas.apply_batch. " + "Please use the API instead.", + FutureWarning, + ) + return self.koalas.apply_batch(func, args=args, **kwds) + + apply_batch.__doc__ = KoalasFrameMethods.apply_batch.__doc__ + + # TODO: Remove this API when Koalas 2.0.0. + def map_in_pandas(self, func) -> "DataFrame": + warnings.warn( + "DataFrame.map_in_pandas is deprecated as of DataFrame.koalas.apply_batch. " + "Please use the API instead.", + FutureWarning, + ) + return self.koalas.apply_batch(func) + + map_in_pandas.__doc__ = KoalasFrameMethods.apply_batch.__doc__ + + def apply(self, func, axis=0, args=(), **kwds) -> Union["Series", "DataFrame", "Index"]: + """ + Apply a function along an axis of the DataFrame. + + Objects passed to the function are Series objects whose index is + either the DataFrame's index (``axis=0``) or the DataFrame's columns + (``axis=1``). + + See also `Transform and apply a function + `_. + + .. note:: when `axis` is 0 or 'index', the `func` is unable to access + to the whole input series. Koalas internally splits the input series into multiple + batches and calls `func` with each batch multiple times. Therefore, operations + such as global aggregations are impossible. See the example below. + + >>> # This case does not return the length of whole series but of the batch internally + ... # used. + ... def length(s) -> int: + ... return len(s) + ... + >>> df = pp.DataFrame({'A': range(1000)}) + >>> df.apply(length, axis=0) # doctest: +SKIP + 0 83 + 1 83 + 2 83 + ... + 10 83 + 11 83 + dtype: int32 + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify the return type as `Series` or scalar value in ``func``, + for instance, as below: + + >>> def square(s) -> pp.Series[np.int32]: + ... return s ** 2 + + Koalas uses return type hint and does not try to infer the type. + + In case when axis is 1, it requires to specify `DataFrame` or scalar value + with type hints as below: + + >>> def plus_one(x) -> pp.DataFrame[float, float]: + ... return x + 1 + + If the return type is specified as `DataFrame`, the output column names become + `c0, c1, c2 ... cn`. These names are positionally mapped to the returned + DataFrame in ``func``. + + To specify the column names, you can assign them in a pandas friendly style as below: + + >>> def plus_one(x) -> pp.DataFrame["a": float, "b": float]: + ... return x + 1 + + >>> pdf = pd.DataFrame({'a': [1, 2, 3], 'b': [3, 4, 5]}) + >>> def plus_one(x) -> pp.DataFrame[zip(pdf.dtypes, pdf.columns)]: + ... return x + 1 + + However, this way switches the index type to default index type in the output + because the type hint cannot express the index type at this moment. Use + `reset_index()` to keep index as a workaround. + + When the given function has the return type annotated, the original index of the + DataFrame will be lost and then a default index will be attached to the result. + Please be careful about configuring the default index. See also `Default Index Type + `_. + + Parameters + ---------- + func : function + Function to apply to each column or row. + axis : {0 or 'index', 1 or 'columns'}, default 0 + Axis along which the function is applied: + + * 0 or 'index': apply function to each column. + * 1 or 'columns': apply function to each row. + args : tuple + Positional arguments to pass to `func` in addition to the + array/series. + **kwds + Additional keyword arguments to pass as keywords arguments to + `func`. + + Returns + ------- + Series or DataFrame + Result of applying ``func`` along the given axis of the + DataFrame. + + See Also + -------- + DataFrame.applymap : For elementwise operations. + DataFrame.aggregate : Only perform aggregating type operations. + DataFrame.transform : Only perform transforming type operations. + Series.apply : The equivalent function for Series. + + Examples + -------- + >>> df = pp.DataFrame([[4, 9]] * 3, columns=['A', 'B']) + >>> df + A B + 0 4 9 + 1 4 9 + 2 4 9 + + Using a numpy universal function (in this case the same as + ``np.sqrt(df)``): + + >>> def sqrt(x) -> pp.Series[float]: + ... return np.sqrt(x) + ... + >>> df.apply(sqrt, axis=0) + A B + 0 2.0 3.0 + 1 2.0 3.0 + 2 2.0 3.0 + + You can omit the type hint and let Koalas infer its type. + + >>> df.apply(np.sqrt, axis=0) + A B + 0 2.0 3.0 + 1 2.0 3.0 + 2 2.0 3.0 + + When `axis` is 1 or 'columns', it applies the function for each row. + + >>> def summation(x) -> np.int64: + ... return np.sum(x) + ... + >>> df.apply(summation, axis=1) + 0 13 + 1 13 + 2 13 + dtype: int64 + + Likewise, you can omit the type hint and let Koalas infer its type. + + >>> df.apply(np.sum, axis=1) + 0 13 + 1 13 + 2 13 + dtype: int64 + + >>> df.apply(max, axis=1) + 0 9 + 1 9 + 2 9 + dtype: int64 + + Returning a list-like will result in a Series + + >>> df.apply(lambda x: [1, 2], axis=1) + 0 [1, 2] + 1 [1, 2] + 2 [1, 2] + dtype: object + + In order to specify the types when `axis` is '1', it should use DataFrame[...] + annotation. In this case, the column names are automatically generated. + + >>> def identify(x) -> pp.DataFrame['A': np.int64, 'B': np.int64]: + ... return x + ... + >>> df.apply(identify, axis=1) + A B + 0 4 9 + 1 4 9 + 2 4 9 + + You can also specify extra arguments. + + >>> def plus_two(a, b, c) -> pp.DataFrame[np.int64, np.int64]: + ... return a + b + c + ... + >>> df.apply(plus_two, axis=1, args=(1,), c=3) + c0 c1 + 0 8 13 + 1 8 13 + 2 8 13 + """ + from pyspark.pandas.groupby import GroupBy + from pyspark.pandas.series import first_series + + if not isinstance(func, types.FunctionType): + assert callable(func), "the first argument should be a callable function." + f = func + func = lambda *args, **kwargs: f(*args, **kwargs) + + axis = validate_axis(axis) + should_return_series = False + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + should_infer_schema = return_sig is None + should_use_map_in_pandas = LooseVersion(pyspark.__version__) >= "3.0" + + def apply_func(pdf): + pdf_or_pser = pdf.apply(func, axis=axis, args=args, **kwds) + if isinstance(pdf_or_pser, pd.Series): + return pdf_or_pser.to_frame() + else: + return pdf_or_pser + + self_applied = DataFrame(self._internal.resolved_copy) # type: "DataFrame" + + column_labels = None # type: Optional[List[Tuple]] + if should_infer_schema: + # Here we execute with the first 1000 to get the return type. + # If the records were less than 1000, it uses pandas API directly for a shortcut. + limit = get_option("compute.shortcut_limit") + pdf = self_applied.head(limit + 1)._to_internal_pandas() + applied = pdf.apply(func, axis=axis, args=args, **kwds) + kser_or_kdf = pp.from_pandas(applied) + if len(pdf) <= limit: + return kser_or_kdf + + kdf = kser_or_kdf + if isinstance(kser_or_kdf, pp.Series): + should_return_series = True + kdf = kser_or_kdf._kdf + + return_schema = force_decimal_precision_scale( + as_nullable_spark_type(kdf._internal.to_internal_spark_frame.schema) + ) + + if should_use_map_in_pandas: + output_func = GroupBy._make_pandas_df_builder_func( + self_applied, apply_func, return_schema, retain_index=True + ) + sdf = self_applied._internal.to_internal_spark_frame.mapInPandas( + lambda iterator: map(output_func, iterator), schema=return_schema + ) + else: + sdf = GroupBy._spark_group_map_apply( + self_applied, + apply_func, + (F.spark_partition_id(),), + return_schema, + retain_index=True, + ) + + # If schema is inferred, we can restore indexes too. + internal = kdf._internal.with_new_sdf(sdf) + else: + return_type = infer_return_type(func) + require_index_axis = isinstance(return_type, SeriesType) + require_column_axis = isinstance(return_type, DataFrameType) + + if require_index_axis: + if axis != 0: + raise TypeError( + "The given function should specify a scalar or a series as its type " + "hints when axis is 0 or 'index'; however, the return type " + "was %s" % return_sig + ) + return_schema = cast(SeriesType, return_type).spark_type + fields_types = zip( + self_applied.columns, [return_schema] * len(self_applied.columns) + ) + return_schema = StructType([StructField(c, t) for c, t in fields_types]) + data_dtypes = [cast(SeriesType, return_type).dtype] * len(self_applied.columns) + elif require_column_axis: + if axis != 1: + raise TypeError( + "The given function should specify a scalar or a frame as its type " + "hints when axis is 1 or 'column'; however, the return type " + "was %s" % return_sig + ) + return_schema = cast(DataFrameType, return_type).spark_type + data_dtypes = cast(DataFrameType, return_type).dtypes + else: + # any axis is fine. + should_return_series = True + return_schema = cast(ScalarType, return_type).spark_type + return_schema = StructType([StructField(SPARK_DEFAULT_SERIES_NAME, return_schema)]) + data_dtypes = [cast(ScalarType, return_type).dtype] + column_labels = [None] + + if should_use_map_in_pandas: + output_func = GroupBy._make_pandas_df_builder_func( + self_applied, apply_func, return_schema, retain_index=False + ) + sdf = self_applied._internal.to_internal_spark_frame.mapInPandas( + lambda iterator: map(output_func, iterator), schema=return_schema + ) + else: + sdf = GroupBy._spark_group_map_apply( + self_applied, + apply_func, + (F.spark_partition_id(),), + return_schema, + retain_index=False, + ) + + # Otherwise, it loses index. + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=None, + column_labels=column_labels, + data_dtypes=data_dtypes, + ) + + result = DataFrame(internal) # type: "DataFrame" + if should_return_series: + return first_series(result) + else: + return result + + def transform(self, func, axis=0, *args, **kwargs) -> "DataFrame": + """ + Call ``func`` on self producing a Series with transformed values + and that has the same length as its input. + + See also `Transform and apply a function + `_. + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def square(x) -> pp.Series[np.int32]: + ... return x ** 2 + + Koalas uses return type hint and does not try to infer the type. + + .. note:: the series within ``func`` is actually multiple pandas series as the + segments of the whole Koalas series; therefore, the length of each series + is not guaranteed. As an example, an aggregation against each series + does work as a global aggregation but an aggregation of each segment. See + below: + + >>> def func(x) -> pp.Series[np.int32]: + ... return x + sum(x) + + Parameters + ---------- + func : function + Function to use for transforming the data. It must work when pandas Series + is passed. + axis : int, default 0 or 'index' + Can only be set to 0 at the moment. + *args + Positional arguments to pass to func. + **kwargs + Keyword arguments to pass to func. + + Returns + ------- + DataFrame + A DataFrame that must have the same length as self. + + Raises + ------ + Exception : If the returned DataFrame has a different length than self. + + See Also + -------- + DataFrame.aggregate : Only perform aggregating type operations. + DataFrame.apply : Invoke function on DataFrame. + Series.transform : The equivalent function for Series. + + Examples + -------- + >>> df = pp.DataFrame({'A': range(3), 'B': range(1, 4)}, columns=['A', 'B']) + >>> df + A B + 0 0 1 + 1 1 2 + 2 2 3 + + >>> def square(x) -> pp.Series[np.int32]: + ... return x ** 2 + >>> df.transform(square) + A B + 0 0 1 + 1 1 4 + 2 4 9 + + You can omit the type hint and let Koalas infer its type. + + >>> df.transform(lambda x: x ** 2) + A B + 0 0 1 + 1 1 4 + 2 4 9 + + For multi-index columns: + + >>> df.columns = [('X', 'A'), ('X', 'B')] + >>> df.transform(square) # doctest: +NORMALIZE_WHITESPACE + X + A B + 0 0 1 + 1 1 4 + 2 4 9 + + >>> (df * -1).transform(abs) # doctest: +NORMALIZE_WHITESPACE + X + A B + 0 0 1 + 1 1 2 + 2 2 3 + + You can also specify extra arguments. + + >>> def calculation(x, y, z) -> pp.Series[int]: + ... return x ** y + z + >>> df.transform(calculation, y=10, z=20) # doctest: +NORMALIZE_WHITESPACE + X + A B + 0 20 21 + 1 21 1044 + 2 1044 59069 + """ + if not isinstance(func, types.FunctionType): + assert callable(func), "the first argument should be a callable function." + f = func + func = lambda *args, **kwargs: f(*args, **kwargs) + + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + should_infer_schema = return_sig is None + + if should_infer_schema: + # Here we execute with the first 1000 to get the return type. + # If the records were less than 1000, it uses pandas API directly for a shortcut. + limit = get_option("compute.shortcut_limit") + pdf = self.head(limit + 1)._to_internal_pandas() + transformed = pdf.transform(func, axis, *args, **kwargs) + kdf = DataFrame(transformed) # type: "DataFrame" + if len(pdf) <= limit: + return kdf + + applied = [] + for input_label, output_label in zip( + self._internal.column_labels, kdf._internal.column_labels + ): + kser = self._kser_for(input_label) + dtype = kdf._internal.dtype_for(output_label) + return_schema = force_decimal_precision_scale( + as_nullable_spark_type(kdf._internal.spark_type_for(output_label)) + ) + applied.append( + kser.koalas._transform_batch( + func=lambda c: func(c, *args, **kwargs), + return_type=SeriesType(dtype, return_schema), + ) + ) + + internal = self._internal.with_new_columns( + applied, data_dtypes=kdf._internal.data_dtypes + ) + return DataFrame(internal) + else: + return self._apply_series_op( + lambda kser: kser.koalas.transform_batch(func, *args, **kwargs) + ) + + def transform_batch(self, func, *args, **kwargs) -> "DataFrame": + warnings.warn( + "DataFrame.transform_batch is deprecated as of DataFrame.koalas.transform_batch. " + "Please use the API instead.", + FutureWarning, + ) + return self.koalas.transform_batch(func, *args, **kwargs) + + transform_batch.__doc__ = KoalasFrameMethods.transform_batch.__doc__ + + def pop(self, item) -> "DataFrame": + """ + Return item and drop from frame. Raise KeyError if not found. + + Parameters + ---------- + item : str + Label of column to be popped. + + Returns + ------- + Series + + Examples + -------- + >>> df = pp.DataFrame([('falcon', 'bird', 389.0), + ... ('parrot', 'bird', 24.0), + ... ('lion', 'mammal', 80.5), + ... ('monkey','mammal', np.nan)], + ... columns=('name', 'class', 'max_speed')) + + >>> df + name class max_speed + 0 falcon bird 389.0 + 1 parrot bird 24.0 + 2 lion mammal 80.5 + 3 monkey mammal NaN + + >>> df.pop('class') + 0 bird + 1 bird + 2 mammal + 3 mammal + Name: class, dtype: object + + >>> df + name max_speed + 0 falcon 389.0 + 1 parrot 24.0 + 2 lion 80.5 + 3 monkey NaN + + Also support for MultiIndex + + >>> df = pp.DataFrame([('falcon', 'bird', 389.0), + ... ('parrot', 'bird', 24.0), + ... ('lion', 'mammal', 80.5), + ... ('monkey','mammal', np.nan)], + ... columns=('name', 'class', 'max_speed')) + >>> columns = [('a', 'name'), ('a', 'class'), ('b', 'max_speed')] + >>> df.columns = pd.MultiIndex.from_tuples(columns) + >>> df + a b + name class max_speed + 0 falcon bird 389.0 + 1 parrot bird 24.0 + 2 lion mammal 80.5 + 3 monkey mammal NaN + + >>> df.pop('a') + name class + 0 falcon bird + 1 parrot bird + 2 lion mammal + 3 monkey mammal + + >>> df + b + max_speed + 0 389.0 + 1 24.0 + 2 80.5 + 3 NaN + """ + result = self[item] + self._update_internal_frame(self.drop(item)._internal) + return result + + # TODO: add axis parameter can work when '1' or 'columns' + def xs(self, key, axis=0, level=None) -> Union["DataFrame", "Series"]: + """ + Return cross-section from the DataFrame. + + This method takes a `key` argument to select data at a particular + level of a MultiIndex. + + Parameters + ---------- + key : label or tuple of label + Label contained in the index, or partially in a MultiIndex. + axis : 0 or 'index', default 0 + Axis to retrieve cross-section on. + currently only support 0 or 'index' + level : object, defaults to first n levels (n=1 or len(key)) + In case of a key partially contained in a MultiIndex, indicate + which levels are used. Levels can be referred by label or position. + + Returns + ------- + DataFrame or Series + Cross-section from the original DataFrame + corresponding to the selected index levels. + + See Also + -------- + DataFrame.loc : Access a group of rows and columns + by label(s) or a boolean array. + DataFrame.iloc : Purely integer-location based indexing + for selection by position. + + Examples + -------- + >>> d = {'num_legs': [4, 4, 2, 2], + ... 'num_wings': [0, 0, 2, 2], + ... 'class': ['mammal', 'mammal', 'mammal', 'bird'], + ... 'animal': ['cat', 'dog', 'bat', 'penguin'], + ... 'locomotion': ['walks', 'walks', 'flies', 'walks']} + >>> df = pp.DataFrame(data=d) + >>> df = df.set_index(['class', 'animal', 'locomotion']) + >>> df # doctest: +NORMALIZE_WHITESPACE + num_legs num_wings + class animal locomotion + mammal cat walks 4 0 + dog walks 4 0 + bat flies 2 2 + bird penguin walks 2 2 + + Get values at specified index + + >>> df.xs('mammal') # doctest: +NORMALIZE_WHITESPACE + num_legs num_wings + animal locomotion + cat walks 4 0 + dog walks 4 0 + bat flies 2 2 + + Get values at several indexes + + >>> df.xs(('mammal', 'dog')) # doctest: +NORMALIZE_WHITESPACE + num_legs num_wings + locomotion + walks 4 0 + + >>> df.xs(('mammal', 'dog', 'walks')) # doctest: +NORMALIZE_WHITESPACE + num_legs 4 + num_wings 0 + Name: (mammal, dog, walks), dtype: int64 + + Get values at specified index and level + + >>> df.xs('cat', level=1) # doctest: +NORMALIZE_WHITESPACE + num_legs num_wings + class locomotion + mammal walks 4 0 + """ + from pyspark.pandas.series import first_series + + if not is_name_like_value(key): + raise ValueError("'key' should be a scalar value or tuple that contains scalar values") + + if level is not None and is_name_like_tuple(key): + raise KeyError(key) + + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + if not is_name_like_tuple(key): + key = (key,) + if len(key) > self._internal.index_level: + raise KeyError( + "Key length ({}) exceeds index depth ({})".format( + len(key), self._internal.index_level + ) + ) + if level is None: + level = 0 + + rows = [ + self._internal.index_spark_columns[lvl] == index for lvl, index in enumerate(key, level) + ] + internal = self._internal.with_filter(reduce(lambda x, y: x & y, rows)) + + if len(key) == self._internal.index_level: + kdf = DataFrame(internal) # type: DataFrame + pdf = kdf.head(2)._to_internal_pandas() + if len(pdf) == 0: + raise KeyError(key) + elif len(pdf) > 1: + return kdf + else: + return first_series(DataFrame(pdf.transpose())) + else: + index_spark_columns = ( + internal.index_spark_columns[:level] + + internal.index_spark_columns[level + len(key):] + ) + index_names = internal.index_names[:level] + internal.index_names[level + len(key):] + index_dtypes = internal.index_dtypes[:level] + internal.index_dtypes[level + len(key):] + + internal = internal.copy( + index_spark_columns=index_spark_columns, + index_names=index_names, + index_dtypes=index_dtypes, + ).resolved_copy + return DataFrame(internal) + + def between_time( + self, + start_time: Union[datetime.time, str], + end_time: Union[datetime.time, str], + include_start: bool = True, + include_end: bool = True, + axis: Union[int, str] = 0, + ) -> "DataFrame": + """ + Select values between particular times of the day (e.g., 9:00-9:30 AM). + + By setting ``start_time`` to be later than ``end_time``, + you can get the times that are *not* between the two times. + + Parameters + ---------- + start_time : datetime.time or str + Initial time as a time filter limit. + end_time : datetime.time or str + End time as a time filter limit. + include_start : bool, default True + Whether the start time needs to be included in the result. + include_end : bool, default True + Whether the end time needs to be included in the result. + axis : {0 or 'index', 1 or 'columns'}, default 0 + Determine range time on index or columns value. + + Returns + ------- + DataFrame + Data from the original object filtered to the specified dates range. + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + See Also + -------- + at_time : Select values at a particular time of the day. + first : Select initial periods of time series based on a date offset. + last : Select final periods of time series based on a date offset. + DatetimeIndex.indexer_between_time : Get just the index locations for + values between particular times of the day. + + Examples + -------- + >>> idx = pd.date_range('2018-04-09', periods=4, freq='1D20min') + >>> kdf = pp.DataFrame({'A': [1, 2, 3, 4]}, index=idx) + >>> kdf + A + 2018-04-09 00:00:00 1 + 2018-04-10 00:20:00 2 + 2018-04-11 00:40:00 3 + 2018-04-12 01:00:00 4 + + >>> kdf.between_time('0:15', '0:45') + A + 2018-04-10 00:20:00 2 + 2018-04-11 00:40:00 3 + + You get the times that are *not* between two times by setting + ``start_time`` later than ``end_time``: + + >>> kdf.between_time('0:45', '0:15') + A + 2018-04-09 00:00:00 1 + 2018-04-12 01:00:00 4 + """ + axis = validate_axis(axis) + + if axis != 0: + raise NotImplementedError("between_time currently only works for axis=0") + + if not isinstance(self.index, pp.DatetimeIndex): + raise TypeError("Index must be DatetimeIndex") + + kdf = self.copy() + kdf.index.name = verify_temp_column_name(kdf, "__index_name__") + return_types = [kdf.index.dtype] + list(kdf.dtypes) + + def pandas_between_time(pdf) -> pp.DataFrame[return_types]: # type: ignore + return pdf.between_time(start_time, end_time, include_start, include_end).reset_index() + + # apply_batch will remove the index of the Koalas DataFrame and attach a default index, + # which will never be used. So use "distributed" index as a dummy to avoid overhead. + with option_context("compute.default_index_type", "distributed"): + kdf = kdf.koalas.apply_batch(pandas_between_time) + + return DataFrame( + self._internal.copy( + spark_frame=kdf._internal.spark_frame, + index_spark_columns=kdf._internal.data_spark_columns[:1], + data_spark_columns=kdf._internal.data_spark_columns[1:], + ) + ) + + # TODO: implement axis=1 + def at_time( + self, time: Union[datetime.time, str], asof: bool = False, axis: Union[int, str] = 0 + ) -> "DataFrame": + """ + Select values at particular time of day (e.g., 9:30AM). + + Parameters + ---------- + time : datetime.time or str + axis : {0 or 'index', 1 or 'columns'}, default 0 + + Returns + ------- + DataFrame + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + See Also + -------- + between_time : Select values between particular times of the day. + DatetimeIndex.indexer_at_time : Get just the index locations for + values at particular time of the day. + + Examples + -------- + >>> idx = pd.date_range('2018-04-09', periods=4, freq='12H') + >>> kdf = pp.DataFrame({'A': [1, 2, 3, 4]}, index=idx) + >>> kdf + A + 2018-04-09 00:00:00 1 + 2018-04-09 12:00:00 2 + 2018-04-10 00:00:00 3 + 2018-04-10 12:00:00 4 + + >>> kdf.at_time('12:00') + A + 2018-04-09 12:00:00 2 + 2018-04-10 12:00:00 4 + """ + if asof: + raise NotImplementedError("'asof' argument is not supported") + + axis = validate_axis(axis) + + if axis != 0: + raise NotImplementedError("at_time currently only works for axis=0") + + if not isinstance(self.index, pp.DatetimeIndex): + raise TypeError("Index must be DatetimeIndex") + + kdf = self.copy() + kdf.index.name = verify_temp_column_name(kdf, "__index_name__") + return_types = [kdf.index.dtype] + list(kdf.dtypes) + + if LooseVersion(pd.__version__) < LooseVersion("0.24"): + + def pandas_at_time(pdf) -> pp.DataFrame[return_types]: # type: ignore + return pdf.at_time(time, asof).reset_index() + + else: + + def pandas_at_time(pdf) -> pp.DataFrame[return_types]: # type: ignore + return pdf.at_time(time, asof, axis).reset_index() + + # apply_batch will remove the index of the Koalas DataFrame and attach a default index, + # which will never be used. So use "distributed" index as a dummy to avoid overhead. + with option_context("compute.default_index_type", "distributed"): + kdf = kdf.koalas.apply_batch(pandas_at_time) + + return DataFrame( + self._internal.copy( + spark_frame=kdf._internal.spark_frame, + index_spark_columns=kdf._internal.data_spark_columns[:1], + data_spark_columns=kdf._internal.data_spark_columns[1:], + ) + ) + + def where(self, cond, other=np.nan) -> "DataFrame": + """ + Replace values where the condition is False. + + Parameters + ---------- + cond : boolean DataFrame + Where cond is True, keep the original value. Where False, + replace with corresponding value from other. + other : scalar, DataFrame + Entries where cond is False are replaced with corresponding value from other. + + Returns + ------- + DataFrame + + Examples + -------- + + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + >>> df1 = pp.DataFrame({'A': [0, 1, 2, 3, 4], 'B':[100, 200, 300, 400, 500]}) + >>> df2 = pp.DataFrame({'A': [0, -1, -2, -3, -4], 'B':[-100, -200, -300, -400, -500]}) + >>> df1 + A B + 0 0 100 + 1 1 200 + 2 2 300 + 3 3 400 + 4 4 500 + >>> df2 + A B + 0 0 -100 + 1 -1 -200 + 2 -2 -300 + 3 -3 -400 + 4 -4 -500 + + >>> df1.where(df1 > 0).sort_index() + A B + 0 NaN 100.0 + 1 1.0 200.0 + 2 2.0 300.0 + 3 3.0 400.0 + 4 4.0 500.0 + + >>> df1.where(df1 > 1, 10).sort_index() + A B + 0 10 100 + 1 10 200 + 2 2 300 + 3 3 400 + 4 4 500 + + >>> df1.where(df1 > 1, df1 + 100).sort_index() + A B + 0 100 100 + 1 101 200 + 2 2 300 + 3 3 400 + 4 4 500 + + >>> df1.where(df1 > 1, df2).sort_index() + A B + 0 0 100 + 1 -1 200 + 2 2 300 + 3 3 400 + 4 4 500 + + When the column name of cond is different from self, it treats all values are False + + >>> cond = pp.DataFrame({'C': [0, -1, -2, -3, -4], 'D':[4, 3, 2, 1, 0]}) % 3 == 0 + >>> cond + C D + 0 True False + 1 False True + 2 False False + 3 True False + 4 False True + + >>> df1.where(cond).sort_index() + A B + 0 NaN NaN + 1 NaN NaN + 2 NaN NaN + 3 NaN NaN + 4 NaN NaN + + When the type of cond is Series, it just check boolean regardless of column name + + >>> cond = pp.Series([1, 2]) > 1 + >>> cond + 0 False + 1 True + dtype: bool + + >>> df1.where(cond).sort_index() + A B + 0 NaN NaN + 1 1.0 200.0 + 2 NaN NaN + 3 NaN NaN + 4 NaN NaN + + >>> reset_option("compute.ops_on_diff_frames") + """ + from pyspark.pandas.series import Series + + tmp_cond_col_name = "__tmp_cond_col_{}__".format + tmp_other_col_name = "__tmp_other_col_{}__".format + + kdf = self.copy() + + tmp_cond_col_names = [ + tmp_cond_col_name(name_like_string(label)) for label in self._internal.column_labels + ] + if isinstance(cond, DataFrame): + cond = cond[ + [ + ( + cond._internal.spark_column_for(label) + if label in cond._internal.column_labels + else F.lit(False) + ).alias(name) + for label, name in zip(self._internal.column_labels, tmp_cond_col_names) + ] + ] + kdf[tmp_cond_col_names] = cond + elif isinstance(cond, Series): + cond = cond.to_frame() + cond = cond[ + [cond._internal.data_spark_columns[0].alias(name) for name in tmp_cond_col_names] + ] + kdf[tmp_cond_col_names] = cond + else: + raise ValueError("type of cond must be a DataFrame or Series") + + tmp_other_col_names = [ + tmp_other_col_name(name_like_string(label)) for label in self._internal.column_labels + ] + if isinstance(other, DataFrame): + other = other[ + [ + ( + other._internal.spark_column_for(label) + if label in other._internal.column_labels + else F.lit(np.nan) + ).alias(name) + for label, name in zip(self._internal.column_labels, tmp_other_col_names) + ] + ] + kdf[tmp_other_col_names] = other + elif isinstance(other, Series): + other = other.to_frame() + other = other[ + [other._internal.data_spark_columns[0].alias(name) for name in tmp_other_col_names] + ] + kdf[tmp_other_col_names] = other + else: + for label in self._internal.column_labels: + kdf[tmp_other_col_name(name_like_string(label))] = other + + # above logic make spark dataframe looks like below: + # +-----------------+---+---+------------------+-------------------+------------------+--... + # |__index_level_0__| A| B|__tmp_cond_col_A__|__tmp_other_col_A__|__tmp_cond_col_B__|__... + # +-----------------+---+---+------------------+-------------------+------------------+--... + # | 0| 0|100| true| 0| false| ... + # | 1| 1|200| false| -1| false| ... + # | 3| 3|400| true| -3| false| ... + # | 2| 2|300| false| -2| true| ... + # | 4| 4|500| false| -4| false| ... + # +-----------------+---+---+------------------+-------------------+------------------+--... + + data_spark_columns = [] + for label in self._internal.column_labels: + data_spark_columns.append( + F.when( + kdf[tmp_cond_col_name(name_like_string(label))].spark.column, + kdf._internal.spark_column_for(label), + ) + .otherwise(kdf[tmp_other_col_name(name_like_string(label))].spark.column) + .alias(kdf._internal.spark_column_name_for(label)) + ) + + return DataFrame( + kdf._internal.with_new_columns( + data_spark_columns, column_labels=self._internal.column_labels # TODO: dtypes? + ) + ) + + def mask(self, cond, other=np.nan) -> "DataFrame": + """ + Replace values where the condition is True. + + Parameters + ---------- + cond : boolean DataFrame + Where cond is False, keep the original value. Where True, + replace with corresponding value from other. + other : scalar, DataFrame + Entries where cond is True are replaced with corresponding value from other. + + Returns + ------- + DataFrame + + Examples + -------- + + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + >>> df1 = pp.DataFrame({'A': [0, 1, 2, 3, 4], 'B':[100, 200, 300, 400, 500]}) + >>> df2 = pp.DataFrame({'A': [0, -1, -2, -3, -4], 'B':[-100, -200, -300, -400, -500]}) + >>> df1 + A B + 0 0 100 + 1 1 200 + 2 2 300 + 3 3 400 + 4 4 500 + >>> df2 + A B + 0 0 -100 + 1 -1 -200 + 2 -2 -300 + 3 -3 -400 + 4 -4 -500 + + >>> df1.mask(df1 > 0).sort_index() + A B + 0 0.0 NaN + 1 NaN NaN + 2 NaN NaN + 3 NaN NaN + 4 NaN NaN + + >>> df1.mask(df1 > 1, 10).sort_index() + A B + 0 0 10 + 1 1 10 + 2 10 10 + 3 10 10 + 4 10 10 + + >>> df1.mask(df1 > 1, df1 + 100).sort_index() + A B + 0 0 200 + 1 1 300 + 2 102 400 + 3 103 500 + 4 104 600 + + >>> df1.mask(df1 > 1, df2).sort_index() + A B + 0 0 -100 + 1 1 -200 + 2 -2 -300 + 3 -3 -400 + 4 -4 -500 + + >>> reset_option("compute.ops_on_diff_frames") + """ + from pyspark.pandas.series import Series + + if not isinstance(cond, (DataFrame, Series)): + raise ValueError("type of cond must be a DataFrame or Series") + + cond_inversed = cond._apply_series_op(lambda kser: ~kser) + return self.where(cond_inversed, other) + + @property + def index(self) -> "Index": + """The index (row labels) Column of the DataFrame. + + Currently not supported when the DataFrame has no index. + + See Also + -------- + Index + """ + from pyspark.pandas.indexes.base import Index + + return Index._new_instance(self) + + @property + def empty(self) -> bool: + """ + Returns true if the current DataFrame is empty. Otherwise, returns false. + + Examples + -------- + >>> pp.range(10).empty + False + + >>> pp.range(0).empty + True + + >>> pp.DataFrame({}, index=list('abc')).empty + True + """ + return ( + len(self._internal.column_labels) == 0 + or self._internal.resolved_copy.spark_frame.rdd.isEmpty() + ) + + @property + def style(self) -> "Styler": + """ + Property returning a Styler object containing methods for + building a styled HTML representation for the DataFrame. + + .. note:: currently it collects top 1000 rows and return its + pandas `pandas.io.formats.style.Styler` instance. + + Examples + -------- + >>> pp.range(1001).style # doctest: +ELLIPSIS + + """ + max_results = get_option("compute.max_rows") + pdf = self.head(max_results + 1)._to_internal_pandas() + if len(pdf) > max_results: + warnings.warn("'style' property will only use top %s rows." % max_results, UserWarning) + return pdf.head(max_results).style + + def set_index(self, keys, drop=True, append=False, inplace=False) -> Optional["DataFrame"]: + """Set the DataFrame index (row labels) using one or more existing columns. + + Set the DataFrame index (row labels) using one or more existing + columns or arrays (of the correct length). The index can replace the + existing index or expand on it. + + Parameters + ---------- + keys : label or array-like or list of labels/arrays + This parameter can be either a single column key, a single array of + the same length as the calling DataFrame, or a list containing an + arbitrary combination of column keys and arrays. Here, "array" + encompasses :class:`Series`, :class:`Index` and ``np.ndarray``. + drop : bool, default True + Delete columns to be used as the new index. + append : bool, default False + Whether to append columns to existing index. + inplace : bool, default False + Modify the DataFrame in place (do not create a new object). + + Returns + ------- + DataFrame + Changed row labels. + + See Also + -------- + DataFrame.reset_index : Opposite of set_index. + + Examples + -------- + >>> df = pp.DataFrame({'month': [1, 4, 7, 10], + ... 'year': [2012, 2014, 2013, 2014], + ... 'sale': [55, 40, 84, 31]}, + ... columns=['month', 'year', 'sale']) + >>> df + month year sale + 0 1 2012 55 + 1 4 2014 40 + 2 7 2013 84 + 3 10 2014 31 + + Set the index to become the 'month' column: + + >>> df.set_index('month') # doctest: +NORMALIZE_WHITESPACE + year sale + month + 1 2012 55 + 4 2014 40 + 7 2013 84 + 10 2014 31 + + Create a MultiIndex using columns 'year' and 'month': + + >>> df.set_index(['year', 'month']) # doctest: +NORMALIZE_WHITESPACE + sale + year month + 2012 1 55 + 2014 4 40 + 2013 7 84 + 2014 10 31 + """ + inplace = validate_bool_kwarg(inplace, "inplace") + if is_name_like_tuple(keys): + keys = [keys] + elif is_name_like_value(keys): + keys = [(keys,)] + else: + keys = [key if is_name_like_tuple(key) else (key,) for key in keys] + columns = set(self._internal.column_labels) + for key in keys: + if key not in columns: + raise KeyError(name_like_string(key)) + + if drop: + column_labels = [label for label in self._internal.column_labels if label not in keys] + else: + column_labels = self._internal.column_labels + if append: + index_spark_columns = self._internal.index_spark_columns + [ + self._internal.spark_column_for(label) for label in keys + ] + index_names = self._internal.index_names + keys + index_dtypes = self._internal.index_dtypes + [ + self._internal.dtype_for(label) for label in keys + ] + else: + index_spark_columns = [self._internal.spark_column_for(label) for label in keys] + index_names = keys + index_dtypes = [self._internal.dtype_for(label) for label in keys] + + internal = self._internal.copy( + index_spark_columns=index_spark_columns, + index_names=index_names, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=[self._internal.spark_column_for(label) for label in column_labels], + data_dtypes=[self._internal.dtype_for(label) for label in column_labels], + ) + + if inplace: + self._update_internal_frame(internal) + return None + else: + return DataFrame(internal) + + def reset_index( + self, level=None, drop=False, inplace=False, col_level=0, col_fill="" + ) -> Optional["DataFrame"]: + """Reset the index, or a level of it. + + For DataFrame with multi-level index, return new DataFrame with labeling information in + the columns under the index names, defaulting to 'level_0', 'level_1', etc. if any are None. + For a standard index, the index name will be used (if set), otherwise a default 'index' or + 'level_0' (if 'index' is already taken) will be used. + + Parameters + ---------- + level : int, str, tuple, or list, default None + Only remove the given levels from the index. Removes all levels by + default. + drop : bool, default False + Do not try to insert index into dataframe columns. This resets + the index to the default integer index. + inplace : bool, default False + Modify the DataFrame in place (do not create a new object). + col_level : int or str, default 0 + If the columns have multiple levels, determines which level the + labels are inserted into. By default it is inserted into the first + level. + col_fill : object, default '' + If the columns have multiple levels, determines how the other + levels are named. If None then the index name is repeated. + + Returns + ------- + DataFrame + DataFrame with the new index. + + See Also + -------- + DataFrame.set_index : Opposite of reset_index. + + Examples + -------- + >>> df = pp.DataFrame([('bird', 389.0), + ... ('bird', 24.0), + ... ('mammal', 80.5), + ... ('mammal', np.nan)], + ... index=['falcon', 'parrot', 'lion', 'monkey'], + ... columns=('class', 'max_speed')) + >>> df + class max_speed + falcon bird 389.0 + parrot bird 24.0 + lion mammal 80.5 + monkey mammal NaN + + When we reset the index, the old index is added as a column. Unlike pandas, Koalas + does not automatically add a sequential index. The following 0, 1, 2, 3 are only + there when we display the DataFrame. + + >>> df.reset_index() + index class max_speed + 0 falcon bird 389.0 + 1 parrot bird 24.0 + 2 lion mammal 80.5 + 3 monkey mammal NaN + + We can use the `drop` parameter to avoid the old index being added as + a column: + + >>> df.reset_index(drop=True) + class max_speed + 0 bird 389.0 + 1 bird 24.0 + 2 mammal 80.5 + 3 mammal NaN + + You can also use `reset_index` with `MultiIndex`. + + >>> index = pd.MultiIndex.from_tuples([('bird', 'falcon'), + ... ('bird', 'parrot'), + ... ('mammal', 'lion'), + ... ('mammal', 'monkey')], + ... names=['class', 'name']) + >>> columns = pd.MultiIndex.from_tuples([('speed', 'max'), + ... ('species', 'type')]) + >>> df = pp.DataFrame([(389.0, 'fly'), + ... ( 24.0, 'fly'), + ... ( 80.5, 'run'), + ... (np.nan, 'jump')], + ... index=index, + ... columns=columns) + >>> df # doctest: +NORMALIZE_WHITESPACE + speed species + max type + class name + bird falcon 389.0 fly + parrot 24.0 fly + mammal lion 80.5 run + monkey NaN jump + + If the index has multiple levels, we can reset a subset of them: + + >>> df.reset_index(level='class') # doctest: +NORMALIZE_WHITESPACE + class speed species + max type + name + falcon bird 389.0 fly + parrot bird 24.0 fly + lion mammal 80.5 run + monkey mammal NaN jump + + If we are not dropping the index, by default, it is placed in the top + level. We can place it in another level: + + >>> df.reset_index(level='class', col_level=1) # doctest: +NORMALIZE_WHITESPACE + speed species + class max type + name + falcon bird 389.0 fly + parrot bird 24.0 fly + lion mammal 80.5 run + monkey mammal NaN jump + + When the index is inserted under another level, we can specify under + which one with the parameter `col_fill`: + + >>> df.reset_index(level='class', col_level=1, + ... col_fill='species') # doctest: +NORMALIZE_WHITESPACE + species speed species + class max type + name + falcon bird 389.0 fly + parrot bird 24.0 fly + lion mammal 80.5 run + monkey mammal NaN jump + + If we specify a nonexistent level for `col_fill`, it is created: + + >>> df.reset_index(level='class', col_level=1, + ... col_fill='genus') # doctest: +NORMALIZE_WHITESPACE + genus speed species + class max type + name + falcon bird 389.0 fly + parrot bird 24.0 fly + lion mammal 80.5 run + monkey mammal NaN jump + """ + inplace = validate_bool_kwarg(inplace, "inplace") + multi_index = self._internal.index_level > 1 + + def rename(index): + if multi_index: + return ("level_{}".format(index),) + else: + if ("index",) not in self._internal.column_labels: + return ("index",) + else: + return ("level_{}".format(index),) + + if level is None: + new_column_labels = [ + name if name is not None else rename(i) + for i, name in enumerate(self._internal.index_names) + ] + new_data_spark_columns = [ + scol.alias(name_like_string(label)) + for scol, label in zip(self._internal.index_spark_columns, new_column_labels) + ] + new_data_dtypes = self._internal.index_dtypes + + index_spark_columns = [] + index_names = [] + index_dtypes = [] + else: + if is_list_like(level): + level = list(level) + if isinstance(level, int) or is_name_like_tuple(level): + level = [level] + elif is_name_like_value(level): + level = [(level,)] + else: + level = [ + lvl if isinstance(lvl, int) or is_name_like_tuple(lvl) else (lvl,) + for lvl in level + ] + + if all(isinstance(l, int) for l in level): + for lev in level: + if lev >= self._internal.index_level: + raise IndexError( + "Too many levels: Index has only {} level, not {}".format( + self._internal.index_level, lev + 1 + ) + ) + idx = level + elif all(is_name_like_tuple(lev) for lev in level): + idx = [] + for l in level: + try: + i = self._internal.index_names.index(l) + idx.append(i) + except ValueError: + if multi_index: + raise KeyError("Level unknown not found") + else: + raise KeyError( + "Level unknown must be same as name ({})".format( + name_like_string(self._internal.index_names[0]) + ) + ) + else: + raise ValueError("Level should be all int or all string.") + idx.sort() + + new_column_labels = [] + new_data_spark_columns = [] + new_data_dtypes = [] + + index_spark_columns = self._internal.index_spark_columns.copy() + index_names = self._internal.index_names.copy() + index_dtypes = self._internal.index_dtypes.copy() + + for i in idx[::-1]: + name = index_names.pop(i) + new_column_labels.insert(0, name if name is not None else rename(i)) + + scol = index_spark_columns.pop(i) + new_data_spark_columns.insert(0, scol.alias(name_like_string(name))) + + new_data_dtypes.insert(0, index_dtypes.pop(i)) + + if drop: + new_data_spark_columns = [] + new_column_labels = [] + new_data_dtypes = [] + + for label in new_column_labels: + if label in self._internal.column_labels: + raise ValueError("cannot insert {}, already exists".format(name_like_string(label))) + + if self._internal.column_labels_level > 1: + column_depth = len(self._internal.column_labels[0]) + if col_level >= column_depth: + raise IndexError( + "Too many levels: Index has only {} levels, not {}".format( + column_depth, col_level + 1 + ) + ) + if any(col_level + len(label) > column_depth for label in new_column_labels): + raise ValueError("Item must have length equal to number of levels.") + new_column_labels = [ + tuple( + ([col_fill] * col_level) + + list(label) + + ([col_fill] * (column_depth - (len(label) + col_level))) + ) + for label in new_column_labels + ] + + internal = self._internal.copy( + index_spark_columns=index_spark_columns, + index_names=index_names, + index_dtypes=index_dtypes, + column_labels=new_column_labels + self._internal.column_labels, + data_spark_columns=new_data_spark_columns + self._internal.data_spark_columns, + data_dtypes=new_data_dtypes + self._internal.data_dtypes, + ) + + if inplace: + self._update_internal_frame(internal) + return None + else: + return DataFrame(internal) + + def isnull(self) -> "DataFrame": + """ + Detects missing values for items in the current Dataframe. + + Return a boolean same-sized Dataframe indicating if the values are NA. + NA values, such as None or numpy.NaN, gets mapped to True values. + Everything else gets mapped to False values. + + See Also + -------- + DataFrame.notnull + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, None), (.6, None), (.2, .1)]) + >>> df.isnull() + 0 1 + 0 False False + 1 False True + 2 False True + 3 False False + + >>> df = pp.DataFrame([[None, 'bee', None], ['dog', None, 'fly']]) + >>> df.isnull() + 0 1 2 + 0 True False True + 1 False True False + """ + return self._apply_series_op(lambda kser: kser.isnull()) + + isna = isnull + + def notnull(self) -> "DataFrame": + """ + Detects non-missing values for items in the current Dataframe. + + This function takes a dataframe and indicates whether it's + values are valid (not missing, which is ``NaN`` in numeric + datatypes, ``None`` or ``NaN`` in objects and ``NaT`` in datetimelike). + + See Also + -------- + DataFrame.isnull + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, None), (.6, None), (.2, .1)]) + >>> df.notnull() + 0 1 + 0 True True + 1 True False + 2 True False + 3 True True + + >>> df = pp.DataFrame([['ant', 'bee', 'cat'], ['dog', None, 'fly']]) + >>> df.notnull() + 0 1 2 + 0 True True True + 1 True False True + """ + return self._apply_series_op(lambda kser: kser.notnull()) + + notna = notnull + + def insert( + self, + loc: int, + column, + value: Union[Scalar, "Series", Iterable], + allow_duplicates: bool = False, + ) -> None: + """ + Insert column into DataFrame at specified location. + + Raises a ValueError if `column` is already contained in the DataFrame, + unless `allow_duplicates` is set to True. + + Parameters + ---------- + loc : int + Insertion index. Must verify 0 <= loc <= len(columns). + column : str, number, or hashable object + Label of the inserted column. + value : int, Series, or array-like + allow_duplicates : bool, optional + + Examples + -------- + >>> kdf = pp.DataFrame([1, 2, 3]) + >>> kdf.sort_index() + 0 + 0 1 + 1 2 + 2 3 + >>> kdf.insert(0, 'x', 4) + >>> kdf.sort_index() + x 0 + 0 4 1 + 1 4 2 + 2 4 3 + + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + + >>> kdf.insert(1, 'y', [5, 6, 7]) + >>> kdf.sort_index() + x y 0 + 0 4 5 1 + 1 4 6 2 + 2 4 7 3 + + >>> kdf.insert(2, 'z', pp.Series([8, 9, 10])) + >>> kdf.sort_index() + x y z 0 + 0 4 5 8 1 + 1 4 6 9 2 + 2 4 7 10 3 + + >>> reset_option("compute.ops_on_diff_frames") + """ + if not isinstance(loc, int): + raise TypeError("loc must be int") + + assert 0 <= loc <= len(self.columns) + assert allow_duplicates is False + + if not is_name_like_value(column): + raise ValueError( + '"column" should be a scalar value or tuple that contains scalar values' + ) + + if is_name_like_tuple(column): + if len(column) != len(self.columns.levels): + # To be consistent with pandas + raise ValueError('"column" must have length equal to number of column levels.') + + if column in self.columns: + raise ValueError("cannot insert %s, already exists" % column) + + kdf = self.copy() + kdf[column] = value + columns = kdf.columns[:-1].insert(loc, kdf.columns[-1]) + kdf = kdf[columns] + self._update_internal_frame(kdf._internal) + + # TODO: add frep and axis parameter + def shift(self, periods=1, fill_value=None) -> "DataFrame": + """ + Shift DataFrame by desired number of periods. + + .. note:: the current implementation of shift uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + periods : int + Number of periods to shift. Can be positive or negative. + fill_value : object, optional + The scalar value to use for newly introduced missing values. + The default depends on the dtype of self. For numeric data, np.nan is used. + + Returns + ------- + Copy of input DataFrame, shifted. + + Examples + -------- + >>> df = pp.DataFrame({'Col1': [10, 20, 15, 30, 45], + ... 'Col2': [13, 23, 18, 33, 48], + ... 'Col3': [17, 27, 22, 37, 52]}, + ... columns=['Col1', 'Col2', 'Col3']) + + >>> df.shift(periods=3) + Col1 Col2 Col3 + 0 NaN NaN NaN + 1 NaN NaN NaN + 2 NaN NaN NaN + 3 10.0 13.0 17.0 + 4 20.0 23.0 27.0 + + >>> df.shift(periods=3, fill_value=0) + Col1 Col2 Col3 + 0 0 0 0 + 1 0 0 0 + 2 0 0 0 + 3 10 13 17 + 4 20 23 27 + + """ + return self._apply_series_op( + lambda kser: kser._shift(periods, fill_value), should_resolve=True + ) + + # TODO: axis should support 1 or 'columns' either at this moment + def diff(self, periods: int = 1, axis: Union[int, str] = 0) -> "DataFrame": + """ + First discrete difference of element. + + Calculates the difference of a DataFrame element compared with another element in the + DataFrame (default is the element in the same column of the previous row). + + .. note:: the current implementation of diff uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + periods : int, default 1 + Periods to shift for calculating difference, accepts negative values. + axis : int, default 0 or 'index' + Can only be set to 0 at the moment. + + Returns + ------- + diffed : DataFrame + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 2, 3, 4, 5, 6], + ... 'b': [1, 1, 2, 3, 5, 8], + ... 'c': [1, 4, 9, 16, 25, 36]}, columns=['a', 'b', 'c']) + >>> df + a b c + 0 1 1 1 + 1 2 1 4 + 2 3 2 9 + 3 4 3 16 + 4 5 5 25 + 5 6 8 36 + + >>> df.diff() + a b c + 0 NaN NaN NaN + 1 1.0 0.0 3.0 + 2 1.0 1.0 5.0 + 3 1.0 1.0 7.0 + 4 1.0 2.0 9.0 + 5 1.0 3.0 11.0 + + Difference with previous column + + >>> df.diff(periods=3) + a b c + 0 NaN NaN NaN + 1 NaN NaN NaN + 2 NaN NaN NaN + 3 3.0 2.0 15.0 + 4 3.0 4.0 21.0 + 5 3.0 6.0 27.0 + + Difference with following row + + >>> df.diff(periods=-1) + a b c + 0 -1.0 0.0 -3.0 + 1 -1.0 -1.0 -5.0 + 2 -1.0 -1.0 -7.0 + 3 -1.0 -2.0 -9.0 + 4 -1.0 -3.0 -11.0 + 5 NaN NaN NaN + """ + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + return self._apply_series_op(lambda kser: kser._diff(periods), should_resolve=True) + + # TODO: axis should support 1 or 'columns' either at this moment + def nunique( + self, + axis: Union[int, str] = 0, + dropna: bool = True, + approx: bool = False, + rsd: float = 0.05, + ) -> "Series": + """ + Return number of unique elements in the object. + + Excludes NA values by default. + + Parameters + ---------- + axis : int, default 0 or 'index' + Can only be set to 0 at the moment. + dropna : bool, default True + Don’t include NaN in the count. + approx: bool, default False + If False, will use the exact algorithm and return the exact number of unique. + If True, it uses the HyperLogLog approximate algorithm, which is significantly faster + for large amount of data. + Note: This parameter is specific to Koalas and is not found in pandas. + rsd: float, default 0.05 + Maximum estimation error allowed in the HyperLogLog algorithm. + Note: Just like ``approx`` this parameter is specific to Koalas. + + Returns + ------- + The number of unique values per column as a Koalas Series. + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2, 3], 'B': [np.nan, 3, np.nan]}) + >>> df.nunique() + A 3 + B 1 + dtype: int64 + + >>> df.nunique(dropna=False) + A 3 + B 2 + dtype: int64 + + On big data, we recommend using the approximate algorithm to speed up this function. + The result will be very close to the exact unique count. + + >>> df.nunique(approx=True) + A 3 + B 1 + dtype: int64 + """ + from pyspark.pandas.series import first_series + + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + sdf = self._internal.spark_frame.select( + [F.lit(None).cast(StringType()).alias(SPARK_DEFAULT_INDEX_NAME)] + + [ + self._kser_for(label)._nunique(dropna, approx, rsd) + for label in self._internal.column_labels + ] + ) + + # The data is expected to be small so it's fine to transpose/use default index. + with pp.option_context("compute.max_rows", 1): + internal = self._internal.copy( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, SPARK_DEFAULT_INDEX_NAME)], + index_names=[None], + index_dtypes=[None], + data_spark_columns=[ + scol_for(sdf, col) for col in self._internal.data_spark_column_names + ], + data_dtypes=None, + ) + return first_series(DataFrame(internal).transpose()) + + def round(self, decimals=0) -> "DataFrame": + """ + Round a DataFrame to a variable number of decimal places. + + Parameters + ---------- + decimals : int, dict, Series + Number of decimal places to round each column to. If an int is + given, round each column to the same number of places. + Otherwise dict and Series round to variable numbers of places. + Column names should be in the keys if `decimals` is a + dict-like, or in the index if `decimals` is a Series. Any + columns not included in `decimals` will be left as is. Elements + of `decimals` which are not columns of the input will be + ignored. + + .. note:: If `decimals` is a Series, it is expected to be small, + as all the data is loaded into the driver's memory. + + Returns + ------- + DataFrame + + See Also + -------- + Series.round + + Examples + -------- + >>> df = pp.DataFrame({'A':[0.028208, 0.038683, 0.877076], + ... 'B':[0.992815, 0.645646, 0.149370], + ... 'C':[0.173891, 0.577595, 0.491027]}, + ... columns=['A', 'B', 'C'], + ... index=['first', 'second', 'third']) + >>> df + A B C + first 0.028208 0.992815 0.173891 + second 0.038683 0.645646 0.577595 + third 0.877076 0.149370 0.491027 + + >>> df.round(2) + A B C + first 0.03 0.99 0.17 + second 0.04 0.65 0.58 + third 0.88 0.15 0.49 + + >>> df.round({'A': 1, 'C': 2}) + A B C + first 0.0 0.992815 0.17 + second 0.0 0.645646 0.58 + third 0.9 0.149370 0.49 + + >>> decimals = pp.Series([1, 0, 2], index=['A', 'B', 'C']) + >>> df.round(decimals) + A B C + first 0.0 1.0 0.17 + second 0.0 1.0 0.58 + third 0.9 0.0 0.49 + """ + if isinstance(decimals, pp.Series): + decimals = { + k if isinstance(k, tuple) else (k,): v + for k, v in decimals._to_internal_pandas().items() + } + elif isinstance(decimals, dict): + decimals = {k if is_name_like_tuple(k) else (k,): v for k, v in decimals.items()} + elif isinstance(decimals, int): + decimals = {k: decimals for k in self._internal.column_labels} + else: + raise ValueError("decimals must be an integer, a dict-like or a Series") + + def op(kser): + label = kser._column_label + if label in decimals: + return F.round(kser.spark.column, decimals[label]).alias( + kser._internal.data_spark_column_names[0] + ) + else: + return kser + + return self._apply_series_op(op) + + def _mark_duplicates(self, subset=None, keep="first"): + if subset is None: + subset = self._internal.column_labels + else: + if is_name_like_tuple(subset): + subset = [subset] + elif is_name_like_value(subset): + subset = [(subset,)] + else: + subset = [sub if is_name_like_tuple(sub) else (sub,) for sub in subset] + diff = set(subset).difference(set(self._internal.column_labels)) + if len(diff) > 0: + raise KeyError(", ".join([name_like_string(d) for d in diff])) + group_cols = [self._internal.spark_column_name_for(label) for label in subset] + + sdf = self._internal.resolved_copy.spark_frame + + column = verify_temp_column_name(sdf, "__duplicated__") + + if keep == "first" or keep == "last": + if keep == "first": + ord_func = spark.functions.asc + else: + ord_func = spark.functions.desc + window = ( + Window.partitionBy(group_cols) + .orderBy(ord_func(NATURAL_ORDER_COLUMN_NAME)) + .rowsBetween(Window.unboundedPreceding, Window.currentRow) + ) + sdf = sdf.withColumn(column, F.row_number().over(window) > 1) + elif not keep: + window = Window.partitionBy(group_cols).rowsBetween( + Window.unboundedPreceding, Window.unboundedFollowing + ) + sdf = sdf.withColumn(column, F.count("*").over(window) > 1) + else: + raise ValueError("'keep' only supports 'first', 'last' and False") + return sdf, column + + def duplicated(self, subset=None, keep="first") -> "Series": + """ + Return boolean Series denoting duplicate rows, optionally only considering certain columns. + + Parameters + ---------- + subset : column label or sequence of labels, optional + Only consider certain columns for identifying duplicates, + by default use all of the columns + keep : {'first', 'last', False}, default 'first' + - ``first`` : Mark duplicates as ``True`` except for the first occurrence. + - ``last`` : Mark duplicates as ``True`` except for the last occurrence. + - False : Mark all duplicates as ``True``. + + Returns + ------- + duplicated : Series + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 1, 3], 'b': [1, 1, 1, 4], 'c': [1, 1, 1, 5]}, + ... columns = ['a', 'b', 'c']) + >>> df + a b c + 0 1 1 1 + 1 1 1 1 + 2 1 1 1 + 3 3 4 5 + + >>> df.duplicated().sort_index() + 0 False + 1 True + 2 True + 3 False + dtype: bool + + Mark duplicates as ``True`` except for the last occurrence. + + >>> df.duplicated(keep='last').sort_index() + 0 True + 1 True + 2 False + 3 False + dtype: bool + + Mark all duplicates as ``True``. + + >>> df.duplicated(keep=False).sort_index() + 0 True + 1 True + 2 True + 3 False + dtype: bool + """ + from pyspark.pandas.series import first_series + + sdf, column = self._mark_duplicates(subset, keep) + + sdf = sdf.select( + self._internal.index_spark_columns + + [scol_for(sdf, column).alias(SPARK_DEFAULT_SERIES_NAME)] + ) + return first_series( + DataFrame( + InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + column_labels=[None], # type: ignore + data_spark_columns=[scol_for(sdf, SPARK_DEFAULT_SERIES_NAME)], + ) + ) + ) + + # TODO: support other as DataFrame or array-like + def dot(self, other: "Series") -> "Series": + """ + Compute the matrix multiplication between the DataFrame and other. + + This method computes the matrix product between the DataFrame and the + values of an other Series + + It can also be called using ``self @ other`` in Python >= 3.5. + + .. note:: This method is based on an expensive operation due to the nature + of big data. Internally it needs to generate each row for each value, and + then group twice - it is a huge operation. To prevent misusage, this method + has the 'compute.max_rows' default limit of input length, and raises a ValueError. + + >>> from pyspark.pandas.config import option_context + >>> with option_context( + ... 'compute.max_rows', 1000, "compute.ops_on_diff_frames", True + ... ): # doctest: +NORMALIZE_WHITESPACE + ... kdf = pp.DataFrame({'a': range(1001)}) + ... kser = pp.Series([2], index=['a']) + ... kdf.dot(kser) + Traceback (most recent call last): + ... + ValueError: Current DataFrame has more then the given limit 1000 rows. + Please set 'compute.max_rows' by using 'pyspark.pandas.config.set_option' + to retrieve to retrieve more than 1000 rows. Note that, before changing the + 'compute.max_rows', this operation is considerably expensive. + + Parameters + ---------- + other : Series + The other object to compute the matrix product with. + + Returns + ------- + Series + Return the matrix product between self and other as a Series. + + See Also + -------- + Series.dot: Similar method for Series. + + Notes + ----- + The dimensions of DataFrame and other must be compatible in order to + compute the matrix multiplication. In addition, the column names of + DataFrame and the index of other must contain the same values, as they + will be aligned prior to the multiplication. + + The dot method for Series computes the inner product, instead of the + matrix product here. + + Examples + -------- + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + >>> kdf = pp.DataFrame([[0, 1, -2, -1], [1, 1, 1, 1]]) + >>> kser = pp.Series([1, 1, 2, 1]) + >>> kdf.dot(kser) + 0 -4 + 1 5 + dtype: int64 + + Note how shuffling of the objects does not change the result. + + >>> kser2 = kser.reindex([1, 0, 2, 3]) + >>> kdf.dot(kser2) + 0 -4 + 1 5 + dtype: int64 + >>> kdf @ kser2 + 0 -4 + 1 5 + dtype: int64 + >>> reset_option("compute.ops_on_diff_frames") + """ + if not isinstance(other, pp.Series): + raise TypeError("Unsupported type {}".format(type(other).__name__)) + else: + return cast(pp.Series, other.dot(self.transpose())).rename(None) + + def __matmul__(self, other): + """ + Matrix multiplication using binary `@` operator in Python>=3.5. + """ + return self.dot(other) + + def to_koalas(self, index_col: Optional[Union[str, List[str]]] = None) -> "DataFrame": + """ + Converts the existing DataFrame into a Koalas DataFrame. + + This method is monkey-patched into Spark's DataFrame and can be used + to convert a Spark DataFrame into a Koalas DataFrame. If running on + an existing Koalas DataFrame, the method returns itself. + + If a Koalas DataFrame is converted to a Spark DataFrame and then back + to Koalas, it will lose the index information and the original index + will be turned into a normal column. + + Parameters + ---------- + index_col: str or list of str, optional, default: None + Index column of table in Spark. + + See Also + -------- + DataFrame.to_spark + + Examples + -------- + >>> df = pp.DataFrame({'col1': [1, 2], 'col2': [3, 4]}, columns=['col1', 'col2']) + >>> df + col1 col2 + 0 1 3 + 1 2 4 + + >>> spark_df = df.to_spark() + >>> spark_df + DataFrame[col1: bigint, col2: bigint] + + >>> kdf = spark_df.to_koalas() + >>> kdf + col1 col2 + 0 1 3 + 1 2 4 + + We can specify the index columns. + + >>> kdf = spark_df.to_koalas(index_col='col1') + >>> kdf # doctest: +NORMALIZE_WHITESPACE + col2 + col1 + 1 3 + 2 4 + + Calling to_koalas on a Koalas DataFrame simply returns itself. + + >>> df.to_koalas() + col1 col2 + 0 1 3 + 1 2 4 + """ + if isinstance(self, DataFrame): + return self + else: + assert isinstance(self, spark.DataFrame), type(self) + from pyspark.pandas.namespace import _get_index_map + + index_spark_columns, index_names = _get_index_map(self, index_col) + internal = InternalFrame( + spark_frame=self, index_spark_columns=index_spark_columns, index_names=index_names + ) + return DataFrame(internal) + + def cache(self) -> "CachedDataFrame": + warnings.warn( + "DataFrame.cache is deprecated as of DataFrame.spark.cache. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.cache() + + cache.__doc__ = SparkFrameMethods.cache.__doc__ + + def persist(self, storage_level=StorageLevel.MEMORY_AND_DISK) -> "CachedDataFrame": + warnings.warn( + "DataFrame.persist is deprecated as of DataFrame.spark.persist. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.persist(storage_level) + + persist.__doc__ = SparkFrameMethods.persist.__doc__ + + def hint(self, name: str, *parameters) -> "DataFrame": + warnings.warn( + "DataFrame.hint is deprecated as of DataFrame.spark.hint. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.hint(name, *parameters) + + hint.__doc__ = SparkFrameMethods.hint.__doc__ + + def to_table( + self, + name: str, + format: Optional[str] = None, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> None: + return self.spark.to_table(name, format, mode, partition_cols, index_col, **options) + + to_table.__doc__ = SparkFrameMethods.to_table.__doc__ + + def to_delta( + self, + path: str, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> None: + """ + Write the DataFrame out as a Delta Lake table. + + Parameters + ---------- + path : str, required + Path to write to. + mode : str {'append', 'overwrite', 'ignore', 'error', 'errorifexists'}, default + 'overwrite'. Specifies the behavior of the save operation when the destination + exists already. + + - 'append': Append the new data to existing data. + - 'overwrite': Overwrite existing data. + - 'ignore': Silently ignore this operation if data already exists. + - 'error' or 'errorifexists': Throw an exception if data already exists. + + partition_cols : str or list of str, optional, default None + Names of partitioning columns + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + options : dict + All other options passed directly into Delta Lake. + + See Also + -------- + read_delta + DataFrame.to_parquet + DataFrame.to_table + DataFrame.to_spark_io + + Examples + -------- + + >>> df = pp.DataFrame(dict( + ... date=list(pd.date_range('2012-1-1 12:00:00', periods=3, freq='M')), + ... country=['KR', 'US', 'JP'], + ... code=[1, 2 ,3]), columns=['date', 'country', 'code']) + >>> df + date country code + 0 2012-01-31 12:00:00 KR 1 + 1 2012-02-29 12:00:00 US 2 + 2 2012-03-31 12:00:00 JP 3 + + Create a new Delta Lake table, partitioned by one column: + + >>> df.to_delta('%s/to_delta/foo' % path, partition_cols='date') + + Partitioned by two columns: + + >>> df.to_delta('%s/to_delta/bar' % path, partition_cols=['date', 'country']) + + Overwrite an existing table's partitions, using the 'replaceWhere' capability in Delta: + + >>> df.to_delta('%s/to_delta/bar' % path, + ... mode='overwrite', replaceWhere='date >= "2012-01-01"') + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + self.spark.to_spark_io( + path=path, + mode=mode, + format="delta", + partition_cols=partition_cols, + index_col=index_col, + **options, + ) + + def to_parquet( + self, + path: str, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + compression: Optional[str] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> None: + """ + Write the DataFrame out as a Parquet file or directory. + + Parameters + ---------- + path : str, required + Path to write to. + mode : str {'append', 'overwrite', 'ignore', 'error', 'errorifexists'}, + default 'overwrite'. Specifies the behavior of the save operation when the + destination exists already. + + - 'append': Append the new data to existing data. + - 'overwrite': Overwrite existing data. + - 'ignore': Silently ignore this operation if data already exists. + - 'error' or 'errorifexists': Throw an exception if data already exists. + + partition_cols : str or list of str, optional, default None + Names of partitioning columns + compression : str {'none', 'uncompressed', 'snappy', 'gzip', 'lzo', 'brotli', 'lz4', 'zstd'} + Compression codec to use when saving to file. If None is set, it uses the + value specified in `spark.sql.parquet.compression.codec`. + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + options : dict + All other options passed directly into Spark's data source. + + See Also + -------- + read_parquet + DataFrame.to_delta + DataFrame.to_table + DataFrame.to_spark_io + + Examples + -------- + >>> df = pp.DataFrame(dict( + ... date=list(pd.date_range('2012-1-1 12:00:00', periods=3, freq='M')), + ... country=['KR', 'US', 'JP'], + ... code=[1, 2 ,3]), columns=['date', 'country', 'code']) + >>> df + date country code + 0 2012-01-31 12:00:00 KR 1 + 1 2012-02-29 12:00:00 US 2 + 2 2012-03-31 12:00:00 JP 3 + + >>> df.to_parquet('%s/to_parquet/foo.parquet' % path, partition_cols='date') + + >>> df.to_parquet( + ... '%s/to_parquet/foo.parquet' % path, + ... mode = 'overwrite', + ... partition_cols=['date', 'country']) + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + builder = self.to_spark(index_col=index_col).write.mode(mode) + if partition_cols is not None: + builder.partitionBy(partition_cols) + builder._set_opts(compression=compression) + builder.options(**options).format("parquet").save(path) + + def to_orc( + self, + path: str, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> None: + """ + Write the DataFrame out as a ORC file or directory. + + Parameters + ---------- + path : str, required + Path to write to. + mode : str {'append', 'overwrite', 'ignore', 'error', 'errorifexists'}, + default 'overwrite'. Specifies the behavior of the save operation when the + destination exists already. + + - 'append': Append the new data to existing data. + - 'overwrite': Overwrite existing data. + - 'ignore': Silently ignore this operation if data already exists. + - 'error' or 'errorifexists': Throw an exception if data already exists. + + partition_cols : str or list of str, optional, default None + Names of partitioning columns + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + options : dict + All other options passed directly into Spark's data source. + + See Also + -------- + read_orc + DataFrame.to_delta + DataFrame.to_parquet + DataFrame.to_table + DataFrame.to_spark_io + + Examples + -------- + >>> df = pp.DataFrame(dict( + ... date=list(pd.date_range('2012-1-1 12:00:00', periods=3, freq='M')), + ... country=['KR', 'US', 'JP'], + ... code=[1, 2 ,3]), columns=['date', 'country', 'code']) + >>> df + date country code + 0 2012-01-31 12:00:00 KR 1 + 1 2012-02-29 12:00:00 US 2 + 2 2012-03-31 12:00:00 JP 3 + + >>> df.to_orc('%s/to_orc/foo.orc' % path, partition_cols='date') + + >>> df.to_orc( + ... '%s/to_orc/foo.orc' % path, + ... mode = 'overwrite', + ... partition_cols=['date', 'country']) + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + self.spark.to_spark_io( + path=path, + mode=mode, + format="orc", + partition_cols=partition_cols, + index_col=index_col, + **options, + ) + + def to_spark_io( + self, + path: Optional[str] = None, + format: Optional[str] = None, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> None: + return self.spark.to_spark_io(path, format, mode, partition_cols, index_col, **options) + + to_spark_io.__doc__ = SparkFrameMethods.to_spark_io.__doc__ + + def to_spark(self, index_col: Optional[Union[str, List[str]]] = None) -> SparkDataFrame: + return self.spark.frame(index_col) + + to_spark.__doc__ = SparkFrameMethods.__doc__ + + def to_pandas(self) -> pd.DataFrame: + """ + Return a pandas DataFrame. + + .. note:: This method should only be used if the resulting pandas DataFrame is expected + to be small, as all the data is loaded into the driver's memory. + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats']) + >>> df.to_pandas() + dogs cats + 0 0.2 0.3 + 1 0.0 0.6 + 2 0.6 0.0 + 3 0.2 0.1 + """ + return self._internal.to_pandas_frame.copy() + + # Alias to maintain backward compatibility with Spark + def toPandas(self) -> pd.DataFrame: + warnings.warn( + "DataFrame.toPandas is deprecated as of DataFrame.to_pandas. " + "Please use the API instead.", + FutureWarning, + ) + return self.to_pandas() + + toPandas.__doc__ = to_pandas.__doc__ + + def assign(self, **kwargs) -> "DataFrame": + """ + Assign new columns to a DataFrame. + + Returns a new object with all original columns in addition to new ones. + Existing columns that are re-assigned will be overwritten. + + Parameters + ---------- + **kwargs : dict of {str: callable, Series or Index} + The column names are keywords. If the values are + callable, they are computed on the DataFrame and + assigned to the new columns. The callable must not + change input DataFrame (though Koalas doesn't check it). + If the values are not callable, (e.g. a Series or a literal), + they are simply assigned. + + Returns + ------- + DataFrame + A new DataFrame with the new columns in addition to + all the existing columns. + + Examples + -------- + >>> df = pp.DataFrame({'temp_c': [17.0, 25.0]}, + ... index=['Portland', 'Berkeley']) + >>> df + temp_c + Portland 17.0 + Berkeley 25.0 + + Where the value is a callable, evaluated on `df`: + + >>> df.assign(temp_f=lambda x: x.temp_c * 9 / 5 + 32) + temp_c temp_f + Portland 17.0 62.6 + Berkeley 25.0 77.0 + + Alternatively, the same behavior can be achieved by directly + referencing an existing Series or sequence and you can also + create multiple columns within the same assign. + + >>> assigned = df.assign(temp_f=df['temp_c'] * 9 / 5 + 32, + ... temp_k=df['temp_c'] + 273.15, + ... temp_idx=df.index) + >>> assigned[['temp_c', 'temp_f', 'temp_k', 'temp_idx']] + temp_c temp_f temp_k temp_idx + Portland 17.0 62.6 290.15 Portland + Berkeley 25.0 77.0 298.15 Berkeley + + Notes + ----- + Assigning multiple columns within the same ``assign`` is possible + but you cannot refer to newly created or modified columns. This + feature is supported in pandas for Python 3.6 and later but not in + Koalas. In Koalas, all items are computed first, and then assigned. + """ + return self._assign(kwargs) + + def _assign(self, kwargs): + assert isinstance(kwargs, dict) + from pyspark.pandas.indexes import MultiIndex + from pyspark.pandas.series import IndexOpsMixin + + for k, v in kwargs.items(): + is_invalid_assignee = ( + not (isinstance(v, (IndexOpsMixin, spark.Column)) or callable(v) or is_scalar(v)) + ) or isinstance(v, MultiIndex) + if is_invalid_assignee: + raise TypeError( + "Column assignment doesn't support type " "{0}".format(type(v).__name__) + ) + if callable(v): + kwargs[k] = v(self) + + pairs = { + (k if is_name_like_tuple(k) else (k,)): ( + (v.spark.column, v.dtype) + if isinstance(v, IndexOpsMixin) and not isinstance(v, MultiIndex) + else (v, None) + if isinstance(v, spark.Column) + else (F.lit(v), None) + ) + for k, v in kwargs.items() + } + + scols = [] + data_dtypes = [] + for label in self._internal.column_labels: + for i in range(len(label)): + if label[: len(label) - i] in pairs: + scol, dtype = pairs[label[: len(label) - i]] + scol = scol.alias(self._internal.spark_column_name_for(label)) + break + else: + scol = self._internal.spark_column_for(label) + dtype = self._internal.dtype_for(label) + scols.append(scol) + data_dtypes.append(dtype) + + column_labels = self._internal.column_labels.copy() + for label, (scol, dtype) in pairs.items(): + if label not in set(i[: len(label)] for i in self._internal.column_labels): + scols.append(scol.alias(name_like_string(label))) + column_labels.append(label) + data_dtypes.append(dtype) + + level = self._internal.column_labels_level + column_labels = [ + tuple(list(label) + ([""] * (level - len(label)))) for label in column_labels + ] + + internal = self._internal.with_new_columns( + scols, column_labels=column_labels, data_dtypes=data_dtypes + ) + return DataFrame(internal) + + @staticmethod + def from_records( + data: Union[np.array, List[tuple], dict, pd.DataFrame], + index: Union[str, list, np.array] = None, + exclude: list = None, + columns: list = None, + coerce_float: bool = False, + nrows: int = None, + ) -> "DataFrame": + """ + Convert structured or record ndarray to DataFrame. + + Parameters + ---------- + data : ndarray (structured dtype), list of tuples, dict, or DataFrame + index : string, list of fields, array-like + Field of array to use as the index, alternately a specific set of input labels to use + exclude : sequence, default None + Columns or fields to exclude + columns : sequence, default None + Column names to use. If the passed data do not have names associated with them, this + argument provides names for the columns. Otherwise this argument indicates the order of + the columns in the result (any names not found in the data will become all-NA columns) + coerce_float : boolean, default False + Attempt to convert values of non-string, non-numeric objects (like decimal.Decimal) to + floating point, useful for SQL result sets + nrows : int, default None + Number of rows to read if data is an iterator + + Returns + ------- + df : DataFrame + + Examples + -------- + Use dict as input + + >>> pp.DataFrame.from_records({'A': [1, 2, 3]}) + A + 0 1 + 1 2 + 2 3 + + Use list of tuples as input + + >>> pp.DataFrame.from_records([(1, 2), (3, 4)]) + 0 1 + 0 1 2 + 1 3 4 + + Use NumPy array as input + + >>> pp.DataFrame.from_records(np.eye(3)) + 0 1 2 + 0 1.0 0.0 0.0 + 1 0.0 1.0 0.0 + 2 0.0 0.0 1.0 + """ + return DataFrame( + pd.DataFrame.from_records(data, index, exclude, columns, coerce_float, nrows) + ) + + def to_records(self, index=True, column_dtypes=None, index_dtypes=None) -> np.recarray: + """ + Convert DataFrame to a NumPy record array. + + Index will be included as the first field of the record array if + requested. + + .. note:: This method should only be used if the resulting NumPy ndarray is + expected to be small, as all the data is loaded into the driver's memory. + + Parameters + ---------- + index : bool, default True + Include index in resulting record array, stored in 'index' + field or using the index label, if set. + column_dtypes : str, type, dict, default None + If a string or type, the data type to store all columns. If + a dictionary, a mapping of column names and indices (zero-indexed) + to specific data types. + index_dtypes : str, type, dict, default None + If a string or type, the data type to store all index levels. If + a dictionary, a mapping of index level names and indices + (zero-indexed) to specific data types. + This mapping is applied only if `index=True`. + + Returns + ------- + numpy.recarray + NumPy ndarray with the DataFrame labels as fields and each row + of the DataFrame as entries. + + See Also + -------- + DataFrame.from_records: Convert structured or record ndarray + to DataFrame. + numpy.recarray: An ndarray that allows field access using + attributes, analogous to typed columns in a + spreadsheet. + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2], 'B': [0.5, 0.75]}, + ... index=['a', 'b']) + >>> df + A B + a 1 0.50 + b 2 0.75 + + >>> df.to_records() # doctest: +SKIP + rec.array([('a', 1, 0.5 ), ('b', 2, 0.75)], + dtype=[('index', 'O'), ('A', '>> df.to_records(index=False) # doctest: +SKIP + rec.array([(1, 0.5 ), (2, 0.75)], + dtype=[('A', '>> df.to_records(column_dtypes={"A": "int32"}) # doctest: +SKIP + rec.array([('a', 1, 0.5 ), ('b', 2, 0.75)], + dtype=[('index', 'O'), ('A', '>> df.to_records(index_dtypes=" "DataFrame": + """ + Make a copy of this object's indices and data. + + Parameters + ---------- + deep : None + this parameter is not supported but just dummy parameter to match pandas. + + Returns + ------- + copy : DataFrame + + Examples + -------- + >>> df = pp.DataFrame({'x': [1, 2], 'y': [3, 4], 'z': [5, 6], 'w': [7, 8]}, + ... columns=['x', 'y', 'z', 'w']) + >>> df + x y z w + 0 1 3 5 7 + 1 2 4 6 8 + >>> df_copy = df.copy() + >>> df_copy + x y z w + 0 1 3 5 7 + 1 2 4 6 8 + """ + return DataFrame(self._internal) + + def dropna( + self, axis=0, how="any", thresh=None, subset=None, inplace=False + ) -> Optional["DataFrame"]: + """ + Remove missing values. + + Parameters + ---------- + axis : {0 or 'index'}, default 0 + Determine if rows or columns which contain missing values are + removed. + + * 0, or 'index' : Drop rows which contain missing values. + how : {'any', 'all'}, default 'any' + Determine if row or column is removed from DataFrame, when we have + at least one NA or all NA. + + * 'any' : If any NA values are present, drop that row or column. + * 'all' : If all values are NA, drop that row or column. + + thresh : int, optional + Require that many non-NA values. + subset : array-like, optional + Labels along other axis to consider, e.g. if you are dropping rows + these would be a list of columns to include. + inplace : bool, default False + If True, do operation inplace and return None. + + Returns + ------- + DataFrame + DataFrame with NA entries dropped from it. + + See Also + -------- + DataFrame.drop : Drop specified labels from columns. + DataFrame.isnull: Indicate missing values. + DataFrame.notnull : Indicate existing (non-missing) values. + + Examples + -------- + >>> df = pp.DataFrame({"name": ['Alfred', 'Batman', 'Catwoman'], + ... "toy": [None, 'Batmobile', 'Bullwhip'], + ... "born": [None, "1940-04-25", None]}, + ... columns=['name', 'toy', 'born']) + >>> df + name toy born + 0 Alfred None None + 1 Batman Batmobile 1940-04-25 + 2 Catwoman Bullwhip None + + Drop the rows where at least one element is missing. + + >>> df.dropna() + name toy born + 1 Batman Batmobile 1940-04-25 + + Drop the columns where at least one element is missing. + + >>> df.dropna(axis='columns') + name + 0 Alfred + 1 Batman + 2 Catwoman + + Drop the rows where all elements are missing. + + >>> df.dropna(how='all') + name toy born + 0 Alfred None None + 1 Batman Batmobile 1940-04-25 + 2 Catwoman Bullwhip None + + Keep only the rows with at least 2 non-NA values. + + >>> df.dropna(thresh=2) + name toy born + 1 Batman Batmobile 1940-04-25 + 2 Catwoman Bullwhip None + + Define in which columns to look for missing values. + + >>> df.dropna(subset=['name', 'born']) + name toy born + 1 Batman Batmobile 1940-04-25 + + Keep the DataFrame with valid entries in the same variable. + + >>> df.dropna(inplace=True) + >>> df + name toy born + 1 Batman Batmobile 1940-04-25 + """ + axis = validate_axis(axis) + inplace = validate_bool_kwarg(inplace, "inplace") + + if thresh is None: + if how is None: + raise TypeError("must specify how or thresh") + elif how not in ("any", "all"): + raise ValueError("invalid how option: {h}".format(h=how)) + + if subset is not None: + if isinstance(subset, str): + labels = [(subset,)] # type: Optional[List[Tuple]] + elif isinstance(subset, tuple): + labels = [subset] + else: + labels = [sub if isinstance(sub, tuple) else (sub,) for sub in subset] + else: + labels = None + + if axis == 0: + if labels is not None: + invalids = [label for label in labels if label not in self._internal.column_labels] + if len(invalids) > 0: + raise KeyError(invalids) + else: + labels = self._internal.column_labels + + cnt = reduce( + lambda x, y: x + y, + [ + F.when(self._kser_for(label).notna().spark.column, 1).otherwise(0) + for label in labels + ], + F.lit(0), + ) + if thresh is not None: + pred = cnt >= F.lit(int(thresh)) + elif how == "any": + pred = cnt == F.lit(len(labels)) + elif how == "all": + pred = cnt > F.lit(0) + + internal = self._internal.with_filter(pred) + if inplace: + self._update_internal_frame(internal) + return None + else: + return DataFrame(internal) + else: + assert axis == 1 + + internal = self._internal.resolved_copy + + if labels is not None: + if any(len(lbl) != internal.index_level for lbl in labels): + raise ValueError( + "The length of each subset must be the same as the index size." + ) + + cond = reduce( + lambda x, y: x | y, + [ + reduce( + lambda x, y: x & y, + [ + scol == F.lit(l) + for l, scol in zip(lbl, internal.index_spark_columns) + ], + ) + for lbl in labels + ], + ) + + internal = internal.with_filter(cond) + + null_counts = [] + for label in internal.column_labels: + scol = internal.spark_column_for(label) + if isinstance(internal.spark_type_for(label), (FloatType, DoubleType)): + cond = scol.isNull() | F.isnan(scol) + else: + cond = scol.isNull() + null_counts.append( + F.sum(F.when(~cond, 1).otherwise(0)).alias(name_like_string(label)) + ) + + counts = internal.spark_frame.select(null_counts + [F.count("*")]).head() + + if thresh is not None: + column_labels = [ + label + for label, cnt in zip(internal.column_labels, counts) + if (cnt or 0) >= int(thresh) + ] + elif how == "any": + column_labels = [ + label + for label, cnt in zip(internal.column_labels, counts) + if (cnt or 0) == counts[-1] + ] + elif how == "all": + column_labels = [ + label for label, cnt in zip(internal.column_labels, counts) if (cnt or 0) > 0 + ] + + kdf = self[column_labels] + if inplace: + self._update_internal_frame(kdf._internal) + return None + else: + return kdf + + # TODO: add 'limit' when value parameter exists + def fillna( + self, value=None, method=None, axis=None, inplace=False, limit=None + ) -> Optional["DataFrame"]: + """Fill NA/NaN values. + + .. note:: the current implementation of 'method' parameter in fillna uses Spark's Window + without specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + value : scalar, dict, Series + Value to use to fill holes. alternately a dict/Series of values + specifying which value to use for each column. + DataFrame is not supported. + method : {'backfill', 'bfill', 'pad', 'ffill', None}, default None + Method to use for filling holes in reindexed Series pad / ffill: propagate last valid + observation forward to next valid backfill / bfill: + use NEXT valid observation to fill gap + axis : {0 or `index`} + 1 and `columns` are not supported. + inplace : boolean, default False + Fill in place (do not create a new object) + limit : int, default None + If method is specified, this is the maximum number of consecutive NaN values to + forward/backward fill. In other words, if there is a gap with more than this number of + consecutive NaNs, it will only be partially filled. If method is not specified, + this is the maximum number of entries along the entire axis where NaNs will be filled. + Must be greater than 0 if not None + + Returns + ------- + DataFrame + DataFrame with NA entries filled. + + Examples + -------- + >>> df = pp.DataFrame({ + ... 'A': [None, 3, None, None], + ... 'B': [2, 4, None, 3], + ... 'C': [None, None, None, 1], + ... 'D': [0, 1, 5, 4] + ... }, + ... columns=['A', 'B', 'C', 'D']) + >>> df + A B C D + 0 NaN 2.0 NaN 0 + 1 3.0 4.0 NaN 1 + 2 NaN NaN NaN 5 + 3 NaN 3.0 1.0 4 + + Replace all NaN elements with 0s. + + >>> df.fillna(0) + A B C D + 0 0.0 2.0 0.0 0 + 1 3.0 4.0 0.0 1 + 2 0.0 0.0 0.0 5 + 3 0.0 3.0 1.0 4 + + We can also propagate non-null values forward or backward. + + >>> df.fillna(method='ffill') + A B C D + 0 NaN 2.0 NaN 0 + 1 3.0 4.0 NaN 1 + 2 3.0 4.0 NaN 5 + 3 3.0 3.0 1.0 4 + + Replace all NaN elements in column 'A', 'B', 'C', and 'D', with 0, 1, + 2, and 3 respectively. + + >>> values = {'A': 0, 'B': 1, 'C': 2, 'D': 3} + >>> df.fillna(value=values) + A B C D + 0 0.0 2.0 2.0 0 + 1 3.0 4.0 2.0 1 + 2 0.0 1.0 2.0 5 + 3 0.0 3.0 1.0 4 + """ + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError("fillna currently only works for axis=0 or axis='index'") + + if value is not None: + if not isinstance(value, (float, int, str, bool, dict, pd.Series)): + raise TypeError("Unsupported type %s" % type(value).__name__) + if limit is not None: + raise ValueError("limit parameter for value is not support now") + if isinstance(value, pd.Series): + value = value.to_dict() + if isinstance(value, dict): + for v in value.values(): + if not isinstance(v, (float, int, str, bool)): + raise TypeError("Unsupported type %s" % type(v).__name__) + value = {k if is_name_like_tuple(k) else (k,): v for k, v in value.items()} + + def op(kser): + label = kser._column_label + for k, v in value.items(): + if k == label[: len(k)]: + return kser._fillna( + value=value[k], method=method, axis=axis, limit=limit + ) + else: + return kser + + else: + op = lambda kser: kser._fillna(value=value, method=method, axis=axis, limit=limit) + elif method is not None: + op = lambda kser: kser._fillna(value=value, method=method, axis=axis, limit=limit) + else: + raise ValueError("Must specify a fillna 'value' or 'method' parameter.") + + kdf = self._apply_series_op(op, should_resolve=(method is not None)) + + inplace = validate_bool_kwarg(inplace, "inplace") + if inplace: + self._update_internal_frame(kdf._internal, requires_same_anchor=False) + return None + else: + return kdf + + def replace( + self, to_replace=None, value=None, inplace=False, limit=None, regex=False, method="pad", + ) -> Optional["DataFrame"]: + """ + Returns a new DataFrame replacing a value with another value. + + Parameters + ---------- + to_replace : int, float, string, list, tuple or dict + Value to be replaced. + value : int, float, string, list or tuple + Value to use to replace holes. The replacement value must be an int, float, + or string. + If value is a list or tuple, value should be of the same length with to_replace. + inplace : boolean, default False + Fill in place (do not create a new object) + + Returns + ------- + DataFrame + Object after replacement. + + Examples + -------- + >>> df = pp.DataFrame({"name": ['Ironman', 'Captain America', 'Thor', 'Hulk'], + ... "weapon": ['Mark-45', 'Shield', 'Mjolnir', 'Smash']}, + ... columns=['name', 'weapon']) + >>> df + name weapon + 0 Ironman Mark-45 + 1 Captain America Shield + 2 Thor Mjolnir + 3 Hulk Smash + + Scalar `to_replace` and `value` + + >>> df.replace('Ironman', 'War-Machine') + name weapon + 0 War-Machine Mark-45 + 1 Captain America Shield + 2 Thor Mjolnir + 3 Hulk Smash + + List like `to_replace` and `value` + + >>> df.replace(['Ironman', 'Captain America'], ['Rescue', 'Hawkeye'], inplace=True) + >>> df + name weapon + 0 Rescue Mark-45 + 1 Hawkeye Shield + 2 Thor Mjolnir + 3 Hulk Smash + + Dicts can be used to specify different replacement values for different existing values + To use a dict in this way the value parameter should be None + + >>> df.replace({'Mjolnir': 'Stormbuster'}) + name weapon + 0 Rescue Mark-45 + 1 Hawkeye Shield + 2 Thor Stormbuster + 3 Hulk Smash + + Dict can specify that different values should be replaced in different columns + The value parameter should not be None in this case + + >>> df.replace({'weapon': 'Mjolnir'}, 'Stormbuster') + name weapon + 0 Rescue Mark-45 + 1 Hawkeye Shield + 2 Thor Stormbuster + 3 Hulk Smash + + Nested dictionaries + The value parameter should be None to use a nested dict in this way + + >>> df.replace({'weapon': {'Mjolnir': 'Stormbuster'}}) + name weapon + 0 Rescue Mark-45 + 1 Hawkeye Shield + 2 Thor Stormbuster + 3 Hulk Smash + """ + if method != "pad": + raise NotImplementedError("replace currently works only for method='pad") + if limit is not None: + raise NotImplementedError("replace currently works only when limit=None") + if regex is not False: + raise NotImplementedError("replace currently doesn't supports regex") + inplace = validate_bool_kwarg(inplace, "inplace") + + if value is not None and not isinstance(value, (int, float, str, list, tuple, dict)): + raise TypeError("Unsupported type {}".format(type(value).__name__)) + if to_replace is not None and not isinstance( + to_replace, (int, float, str, list, tuple, dict) + ): + raise TypeError("Unsupported type {}".format(type(to_replace).__name__)) + + if isinstance(value, (list, tuple)) and isinstance(to_replace, (list, tuple)): + if len(value) != len(to_replace): + raise ValueError("Length of to_replace and value must be same") + + if isinstance(to_replace, dict) and ( + value is not None or all(isinstance(i, dict) for i in to_replace.values()) + ): + + def op(kser): + if kser.name in to_replace: + return kser.replace(to_replace=to_replace[kser.name], value=value, regex=regex) + else: + return kser + + else: + op = lambda kser: kser.replace(to_replace=to_replace, value=value, regex=regex) + + kdf = self._apply_series_op(op) + if inplace: + self._update_internal_frame(kdf._internal) + return None + else: + return kdf + + def clip(self, lower: Union[float, int] = None, upper: Union[float, int] = None) -> "DataFrame": + """ + Trim values at input threshold(s). + + Assigns values outside boundary to boundary values. + + Parameters + ---------- + lower : float or int, default None + Minimum threshold value. All values below this threshold will be set to it. + upper : float or int, default None + Maximum threshold value. All values above this threshold will be set to it. + + Returns + ------- + DataFrame + DataFrame with the values outside the clip boundaries replaced. + + Examples + -------- + >>> pp.DataFrame({'A': [0, 2, 4]}).clip(1, 3) + A + 0 1 + 1 2 + 2 3 + + Notes + ----- + One difference between this implementation and pandas is that running + pd.DataFrame({'A': ['a', 'b']}).clip(0, 1) will crash with "TypeError: '<=' not supported + between instances of 'str' and 'int'" while pp.DataFrame({'A': ['a', 'b']}).clip(0, 1) + will output the original DataFrame, simply ignoring the incompatible types. + """ + if is_list_like(lower) or is_list_like(upper): + raise ValueError( + "List-like value are not supported for 'lower' and 'upper' at the " + "moment" + ) + + if lower is None and upper is None: + return self + + return self._apply_series_op(lambda kser: kser.clip(lower=lower, upper=upper)) + + def head(self, n: int = 5) -> "DataFrame": + """ + Return the first `n` rows. + + This function returns the first `n` rows for the object based + on position. It is useful for quickly testing if your object + has the right type of data in it. + + Parameters + ---------- + n : int, default 5 + Number of rows to select. + + Returns + ------- + obj_head : same type as caller + The first `n` rows of the caller object. + + Examples + -------- + >>> df = pp.DataFrame({'animal':['alligator', 'bee', 'falcon', 'lion', + ... 'monkey', 'parrot', 'shark', 'whale', 'zebra']}) + >>> df + animal + 0 alligator + 1 bee + 2 falcon + 3 lion + 4 monkey + 5 parrot + 6 shark + 7 whale + 8 zebra + + Viewing the first 5 lines + + >>> df.head() + animal + 0 alligator + 1 bee + 2 falcon + 3 lion + 4 monkey + + Viewing the first `n` lines (three in this case) + + >>> df.head(3) + animal + 0 alligator + 1 bee + 2 falcon + """ + if n < 0: + n = len(self) + n + if n <= 0: + return DataFrame(self._internal.with_filter(F.lit(False))) + else: + sdf = self._internal.resolved_copy.spark_frame + if get_option("compute.ordered_head"): + sdf = sdf.orderBy(NATURAL_ORDER_COLUMN_NAME) + return DataFrame(self._internal.with_new_sdf(sdf.limit(n))) + + def last(self, offset: Union[str, DateOffset]) -> "DataFrame": + """ + Select final periods of time series data based on a date offset. + + When having a DataFrame with dates as index, this function can + select the last few rows based on a date offset. + + Parameters + ---------- + offset : str or DateOffset + The offset length of the data that will be selected. For instance, + '3D' will display all the rows having their index within the last 3 days. + + Returns + ------- + DataFrame + A subset of the caller. + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + Examples + -------- + + >>> index = pd.date_range('2018-04-09', periods=4, freq='2D') + >>> kdf = pp.DataFrame({'A': [1, 2, 3, 4]}, index=index) + >>> kdf + A + 2018-04-09 1 + 2018-04-11 2 + 2018-04-13 3 + 2018-04-15 4 + + Get the rows for the last 3 days: + + >>> kdf.last('3D') + A + 2018-04-13 3 + 2018-04-15 4 + + Notice the data for 3 last calendar days were returned, not the last + 3 observed days in the dataset, and therefore data for 2018-04-11 was + not returned. + """ + # Check index type should be format DateTime + if not isinstance(self.index, pp.DatetimeIndex): + raise TypeError("'last' only supports a DatetimeIndex") + + offset = to_offset(offset) + from_date = self.index.max() - offset + + return cast(DataFrame, self.loc[from_date:]) + + def first(self, offset: Union[str, DateOffset]) -> "DataFrame": + """ + Select first periods of time series data based on a date offset. + + When having a DataFrame with dates as index, this function can + select the first few rows based on a date offset. + + Parameters + ---------- + offset : str or DateOffset + The offset length of the data that will be selected. For instance, + '3D' will display all the rows having their index within the first 3 days. + + Returns + ------- + DataFrame + A subset of the caller. + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + Examples + -------- + + >>> index = pd.date_range('2018-04-09', periods=4, freq='2D') + >>> kdf = pp.DataFrame({'A': [1, 2, 3, 4]}, index=index) + >>> kdf + A + 2018-04-09 1 + 2018-04-11 2 + 2018-04-13 3 + 2018-04-15 4 + + Get the rows for the last 3 days: + + >>> kdf.first('3D') + A + 2018-04-09 1 + 2018-04-11 2 + + Notice the data for 3 first calendar days were returned, not the first + 3 observed days in the dataset, and therefore data for 2018-04-13 was + not returned. + """ + # Check index type should be format DatetimeIndex + if not isinstance(self.index, pp.DatetimeIndex): + raise TypeError("'first' only supports a DatetimeIndex") + + offset = to_offset(offset) + to_date = self.index.min() + offset + + return cast(DataFrame, self.loc[:to_date]) + + def pivot_table( + self, values=None, index=None, columns=None, aggfunc="mean", fill_value=None + ) -> "DataFrame": + """ + Create a spreadsheet-style pivot table as a DataFrame. The levels in + the pivot table will be stored in MultiIndex objects (hierarchical + indexes) on the index and columns of the result DataFrame. + + Parameters + ---------- + values : column to aggregate. + They should be either a list less than three or a string. + index : column (string) or list of columns + If an array is passed, it must be the same length as the data. + The list should contain string. + columns : column + Columns used in the pivot operation. Only one column is supported and + it should be a string. + aggfunc : function (string), dict, default mean + If dict is passed, the key is column to aggregate and value + is function or list of functions. + fill_value : scalar, default None + Value to replace missing values with. + + Returns + ------- + table : DataFrame + + Examples + -------- + >>> df = pp.DataFrame({"A": ["foo", "foo", "foo", "foo", "foo", + ... "bar", "bar", "bar", "bar"], + ... "B": ["one", "one", "one", "two", "two", + ... "one", "one", "two", "two"], + ... "C": ["small", "large", "large", "small", + ... "small", "large", "small", "small", + ... "large"], + ... "D": [1, 2, 2, 3, 3, 4, 5, 6, 7], + ... "E": [2, 4, 5, 5, 6, 6, 8, 9, 9]}, + ... columns=['A', 'B', 'C', 'D', 'E']) + >>> df + A B C D E + 0 foo one small 1 2 + 1 foo one large 2 4 + 2 foo one large 2 5 + 3 foo two small 3 5 + 4 foo two small 3 6 + 5 bar one large 4 6 + 6 bar one small 5 8 + 7 bar two small 6 9 + 8 bar two large 7 9 + + This first example aggregates values by taking the sum. + + >>> table = df.pivot_table(values='D', index=['A', 'B'], + ... columns='C', aggfunc='sum') + >>> table.sort_index() # doctest: +NORMALIZE_WHITESPACE + C large small + A B + bar one 4.0 5 + two 7.0 6 + foo one 4.0 1 + two NaN 6 + + We can also fill missing values using the `fill_value` parameter. + + >>> table = df.pivot_table(values='D', index=['A', 'B'], + ... columns='C', aggfunc='sum', fill_value=0) + >>> table.sort_index() # doctest: +NORMALIZE_WHITESPACE + C large small + A B + bar one 4 5 + two 7 6 + foo one 4 1 + two 0 6 + + We can also calculate multiple types of aggregations for any given + value column. + + >>> table = df.pivot_table(values=['D'], index =['C'], + ... columns="A", aggfunc={'D': 'mean'}) + >>> table.sort_index() # doctest: +NORMALIZE_WHITESPACE + D + A bar foo + C + large 5.5 2.000000 + small 5.5 2.333333 + + The next example aggregates on multiple values. + + >>> table = df.pivot_table(index=['C'], columns="A", values=['D', 'E'], + ... aggfunc={'D': 'mean', 'E': 'sum'}) + >>> table.sort_index() # doctest: +NORMALIZE_WHITESPACE + D E + A bar foo bar foo + C + large 5.5 2.000000 15 9 + small 5.5 2.333333 17 13 + """ + if not is_name_like_value(columns): + raise ValueError("columns should be one column name.") + + if not is_name_like_value(values) and not ( + isinstance(values, list) and all(is_name_like_value(v) for v in values) + ): + raise ValueError("values should be one column or list of columns.") + + if not isinstance(aggfunc, str) and ( + not isinstance(aggfunc, dict) + or not all( + is_name_like_value(key) and isinstance(value, str) for key, value in aggfunc.items() + ) + ): + raise ValueError( + "aggfunc must be a dict mapping from column name " + "to aggregate functions (string)." + ) + + if isinstance(aggfunc, dict) and index is None: + raise NotImplementedError( + "pivot_table doesn't support aggfunc" " as dict and without index." + ) + if isinstance(values, list) and index is None: + raise NotImplementedError("values can't be a list without index.") + + if columns not in self.columns: + raise ValueError("Wrong columns {}.".format(name_like_string(columns))) + if not is_name_like_tuple(columns): + columns = (columns,) + + if isinstance(values, list): + values = [col if is_name_like_tuple(col) else (col,) for col in values] + if not all( + isinstance(self._internal.spark_type_for(col), NumericType) for col in values + ): + raise TypeError("values should be a numeric type.") + else: + values = values if is_name_like_tuple(values) else (values,) + if not isinstance(self._internal.spark_type_for(values), NumericType): + raise TypeError("values should be a numeric type.") + + if isinstance(aggfunc, str): + if isinstance(values, list): + agg_cols = [ + F.expr( + "{1}(`{0}`) as `{0}`".format( + self._internal.spark_column_name_for(value), aggfunc + ) + ) + for value in values + ] + else: + agg_cols = [ + F.expr( + "{1}(`{0}`) as `{0}`".format( + self._internal.spark_column_name_for(values), aggfunc + ) + ) + ] + elif isinstance(aggfunc, dict): + aggfunc = { + key if is_name_like_tuple(key) else (key,): value for key, value in aggfunc.items() + } + agg_cols = [ + F.expr( + "{1}(`{0}`) as `{0}`".format(self._internal.spark_column_name_for(key), value) + ) + for key, value in aggfunc.items() + ] + agg_columns = [key for key, _ in aggfunc.items()] + + if set(agg_columns) != set(values): + raise ValueError("Columns in aggfunc must be the same as values.") + + sdf = self._internal.resolved_copy.spark_frame + if index is None: + sdf = ( + sdf.groupBy() + .pivot(pivot_col=self._internal.spark_column_name_for(columns)) + .agg(*agg_cols) + ) + + elif isinstance(index, list): + index = [label if is_name_like_tuple(label) else (label,) for label in index] + sdf = ( + sdf.groupBy([self._internal.spark_column_name_for(label) for label in index]) + .pivot(pivot_col=self._internal.spark_column_name_for(columns)) + .agg(*agg_cols) + ) + else: + raise ValueError("index should be a None or a list of columns.") + + if fill_value is not None and isinstance(fill_value, (int, float)): + sdf = sdf.fillna(fill_value) + + if index is not None: + index_columns = [self._internal.spark_column_name_for(label) for label in index] + index_dtypes = [self._internal.dtype_for(label) for label in index] + + if isinstance(values, list): + data_columns = [column for column in sdf.columns if column not in index_columns] + + if len(values) > 1: + # If we have two values, Spark will return column's name + # in this format: column_values, where column contains + # their values in the DataFrame and values is + # the column list passed to the pivot_table(). + # E.g. if column is b and values is ['b','e'], + # then ['2_b', '2_e', '3_b', '3_e']. + + # We sort the columns of Spark DataFrame by values. + data_columns.sort(key=lambda x: x.split("_", 1)[1]) + sdf = sdf.select(index_columns + data_columns) + + column_name_to_index = dict( + zip(self._internal.data_spark_column_names, self._internal.column_labels) + ) + column_labels = [ + tuple(list(column_name_to_index[name.split("_")[1]]) + [name.split("_")[0]]) + for name in data_columns + ] + column_label_names = ([None] * column_labels_level(values)) + [columns] + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_columns], + index_names=index, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + column_label_names=column_label_names, # type: ignore + ) + kdf = DataFrame(internal) # type: "DataFrame" + else: + column_labels = [tuple(list(values[0]) + [column]) for column in data_columns] + column_label_names = ([None] * len(values[0])) + [columns] + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_columns], + index_names=index, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + column_label_names=column_label_names, # type: ignore + ) + kdf = DataFrame(internal) + else: + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_columns], + index_names=index, + index_dtypes=index_dtypes, + column_label_names=[columns], + ) + kdf = DataFrame(internal) + else: + if isinstance(values, list): + index_values = values[-1] + else: + index_values = values + index_map = OrderedDict() # type: Dict[str, Optional[Tuple]] + for i, index_value in enumerate(index_values): + colname = SPARK_INDEX_NAME_FORMAT(i) + sdf = sdf.withColumn(colname, F.lit(index_value)) + index_map[colname] = None + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_map.keys()], + index_names=list(index_map.values()), + column_label_names=[columns], + ) + kdf = DataFrame(internal) + + kdf_columns = kdf.columns + if isinstance(kdf_columns, pd.MultiIndex): + kdf.columns = kdf_columns.set_levels( + kdf_columns.levels[-1].astype( + spark_type_to_pandas_dtype(self._kser_for(columns).spark.data_type) + ), + level=-1, + ) + else: + kdf.columns = kdf_columns.astype( + spark_type_to_pandas_dtype(self._kser_for(columns).spark.data_type) + ) + + return kdf + + def pivot(self, index=None, columns=None, values=None) -> "DataFrame": + """ + Return reshaped DataFrame organized by given index / column values. + + Reshape data (produce a "pivot" table) based on column values. Uses + unique values from specified `index` / `columns` to form axes of the + resulting DataFrame. This function does not support data + aggregation. + + Parameters + ---------- + index : string, optional + Column to use to make new frame's index. If None, uses + existing index. + columns : string + Column to use to make new frame's columns. + values : string, object or a list of the previous + Column(s) to use for populating new frame's values. + + Returns + ------- + DataFrame + Returns reshaped DataFrame. + + See Also + -------- + DataFrame.pivot_table : Generalization of pivot that can handle + duplicate values for one index/column pair. + + Examples + -------- + >>> df = pp.DataFrame({'foo': ['one', 'one', 'one', 'two', 'two', + ... 'two'], + ... 'bar': ['A', 'B', 'C', 'A', 'B', 'C'], + ... 'baz': [1, 2, 3, 4, 5, 6], + ... 'zoo': ['x', 'y', 'z', 'q', 'w', 't']}, + ... columns=['foo', 'bar', 'baz', 'zoo']) + >>> df + foo bar baz zoo + 0 one A 1 x + 1 one B 2 y + 2 one C 3 z + 3 two A 4 q + 4 two B 5 w + 5 two C 6 t + + >>> df.pivot(index='foo', columns='bar', values='baz').sort_index() + ... # doctest: +NORMALIZE_WHITESPACE + bar A B C + foo + one 1 2 3 + two 4 5 6 + + >>> df.pivot(columns='bar', values='baz').sort_index() # doctest: +NORMALIZE_WHITESPACE + bar A B C + 0 1.0 NaN NaN + 1 NaN 2.0 NaN + 2 NaN NaN 3.0 + 3 4.0 NaN NaN + 4 NaN 5.0 NaN + 5 NaN NaN 6.0 + + Notice that, unlike pandas raises an ValueError when duplicated values are found, + Koalas' pivot still works with its first value it meets during operation because pivot + is an expensive operation and it is preferred to permissively execute over failing fast + when processing large data. + + >>> df = pp.DataFrame({"foo": ['one', 'one', 'two', 'two'], + ... "bar": ['A', 'A', 'B', 'C'], + ... "baz": [1, 2, 3, 4]}, columns=['foo', 'bar', 'baz']) + >>> df + foo bar baz + 0 one A 1 + 1 one A 2 + 2 two B 3 + 3 two C 4 + + >>> df.pivot(index='foo', columns='bar', values='baz').sort_index() + ... # doctest: +NORMALIZE_WHITESPACE + bar A B C + foo + one 1.0 NaN NaN + two NaN 3.0 4.0 + + It also support multi-index and multi-index column. + >>> df.columns = pd.MultiIndex.from_tuples([('a', 'foo'), ('a', 'bar'), ('b', 'baz')]) + + >>> df = df.set_index(('a', 'bar'), append=True) + >>> df # doctest: +NORMALIZE_WHITESPACE + a b + foo baz + (a, bar) + 0 A one 1 + 1 A one 2 + 2 B two 3 + 3 C two 4 + + >>> df.pivot(columns=('a', 'foo'), values=('b', 'baz')).sort_index() + ... # doctest: +NORMALIZE_WHITESPACE + ('a', 'foo') one two + (a, bar) + 0 A 1.0 NaN + 1 A 2.0 NaN + 2 B NaN 3.0 + 3 C NaN 4.0 + + """ + if columns is None: + raise ValueError("columns should be set.") + + if values is None: + raise ValueError("values should be set.") + + should_use_existing_index = index is not None + if should_use_existing_index: + df = self + index = [index] + else: + # The index after `reset_index()` will never be used, so use "distributed" index + # as a dummy to avoid overhead. + with option_context("compute.default_index_type", "distributed"): + df = self.reset_index() + index = df._internal.column_labels[: self._internal.index_level] + + df = df.pivot_table(index=index, columns=columns, values=values, aggfunc="first") + + if should_use_existing_index: + return df + else: + internal = df._internal.copy(index_names=self._internal.index_names) + return DataFrame(internal) + + @property + def columns(self) -> pd.Index: + """The column labels of the DataFrame.""" + names = [ + name if name is None or len(name) > 1 else name[0] + for name in self._internal.column_label_names + ] + if self._internal.column_labels_level > 1: + columns = pd.MultiIndex.from_tuples(self._internal.column_labels, names=names) + else: + columns = pd.Index([label[0] for label in self._internal.column_labels], name=names[0]) + return columns + + @columns.setter + def columns(self, columns) -> None: + if isinstance(columns, pd.MultiIndex): + column_labels = columns.tolist() + else: + column_labels = [ + col if is_name_like_tuple(col, allow_none=False) else (col,) for col in columns + ] + + if len(self._internal.column_labels) != len(column_labels): + raise ValueError( + "Length mismatch: Expected axis has {} elements, " + "new values have {} elements".format( + len(self._internal.column_labels), len(column_labels) + ) + ) + + if isinstance(columns, pd.Index): + column_label_names = [ + name if is_name_like_tuple(name) else (name,) for name in columns.names + ] # type: Optional[List] + else: + column_label_names = None + + ksers = [ + self._kser_for(label).rename(name) + for label, name in zip(self._internal.column_labels, column_labels) + ] + self._update_internal_frame( + self._internal.with_new_columns(ksers, column_label_names=column_label_names) + ) + + @property + def dtypes(self) -> pd.Series: + """Return the dtypes in the DataFrame. + + This returns a Series with the data type of each column. The result's index is the original + DataFrame's columns. Columns with mixed types are stored with the object dtype. + + Returns + ------- + pd.Series + The data type of each column. + + Examples + -------- + >>> df = pp.DataFrame({'a': list('abc'), + ... 'b': list(range(1, 4)), + ... 'c': np.arange(3, 6).astype('i1'), + ... 'd': np.arange(4.0, 7.0, dtype='float64'), + ... 'e': [True, False, True], + ... 'f': pd.date_range('20130101', periods=3)}, + ... columns=['a', 'b', 'c', 'd', 'e', 'f']) + >>> df.dtypes + a object + b int64 + c int8 + d float64 + e bool + f datetime64[ns] + dtype: object + """ + return pd.Series( + [self._kser_for(label).dtype for label in self._internal.column_labels], + index=pd.Index( + [label if len(label) > 1 else label[0] for label in self._internal.column_labels] + ), + ) + + def spark_schema(self, index_col: Optional[Union[str, List[str]]] = None) -> StructType: + warnings.warn( + "DataFrame.spark_schema is deprecated as of DataFrame.spark.schema. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.schema(index_col) + + spark_schema.__doc__ = SparkFrameMethods.schema.__doc__ + + def print_schema(self, index_col: Optional[Union[str, List[str]]] = None) -> None: + warnings.warn( + "DataFrame.print_schema is deprecated as of DataFrame.spark.print_schema. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.print_schema(index_col) + + print_schema.__doc__ = SparkFrameMethods.print_schema.__doc__ + + def select_dtypes(self, include=None, exclude=None) -> "DataFrame": + """ + Return a subset of the DataFrame's columns based on the column dtypes. + + Parameters + ---------- + include, exclude : scalar or list-like + A selection of dtypes or strings to be included/excluded. At least + one of these parameters must be supplied. It also takes Spark SQL + DDL type strings, for instance, 'string' and 'date'. + + Returns + ------- + DataFrame + The subset of the frame including the dtypes in ``include`` and + excluding the dtypes in ``exclude``. + + Raises + ------ + ValueError + * If both of ``include`` and ``exclude`` are empty + + >>> df = pp.DataFrame({'a': [1, 2] * 3, + ... 'b': [True, False] * 3, + ... 'c': [1.0, 2.0] * 3}) + >>> df.select_dtypes() + Traceback (most recent call last): + ... + ValueError: at least one of include or exclude must be nonempty + + * If ``include`` and ``exclude`` have overlapping elements + + >>> df = pp.DataFrame({'a': [1, 2] * 3, + ... 'b': [True, False] * 3, + ... 'c': [1.0, 2.0] * 3}) + >>> df.select_dtypes(include='a', exclude='a') + Traceback (most recent call last): + ... + ValueError: include and exclude overlap on {'a'} + + Notes + ----- + * To select datetimes, use ``np.datetime64``, ``'datetime'`` or + ``'datetime64'`` + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 2] * 3, + ... 'b': [True, False] * 3, + ... 'c': [1.0, 2.0] * 3, + ... 'd': ['a', 'b'] * 3}, columns=['a', 'b', 'c', 'd']) + >>> df + a b c d + 0 1 True 1.0 a + 1 2 False 2.0 b + 2 1 True 1.0 a + 3 2 False 2.0 b + 4 1 True 1.0 a + 5 2 False 2.0 b + + >>> df.select_dtypes(include='bool') + b + 0 True + 1 False + 2 True + 3 False + 4 True + 5 False + + >>> df.select_dtypes(include=['float64'], exclude=['int']) + c + 0 1.0 + 1 2.0 + 2 1.0 + 3 2.0 + 4 1.0 + 5 2.0 + + >>> df.select_dtypes(exclude=['int']) + b c d + 0 True 1.0 a + 1 False 2.0 b + 2 True 1.0 a + 3 False 2.0 b + 4 True 1.0 a + 5 False 2.0 b + + Spark SQL DDL type strings can be used as well. + + >>> df.select_dtypes(exclude=['string']) + a b c + 0 1 True 1.0 + 1 2 False 2.0 + 2 1 True 1.0 + 3 2 False 2.0 + 4 1 True 1.0 + 5 2 False 2.0 + """ + from pyspark.sql.types import _parse_datatype_string + + if not is_list_like(include): + include = (include,) if include is not None else () + if not is_list_like(exclude): + exclude = (exclude,) if exclude is not None else () + + if not any((include, exclude)): + raise ValueError("at least one of include or exclude must be " "nonempty") + + # can't both include AND exclude! + if set(include).intersection(set(exclude)): + raise ValueError( + "include and exclude overlap on {inc_ex}".format( + inc_ex=set(include).intersection(set(exclude)) + ) + ) + + # Handle Spark types + include_spark_type = [] + for inc in include: + try: + include_spark_type.append(_parse_datatype_string(inc)) + except: + pass + + exclude_spark_type = [] + for exc in exclude: + try: + exclude_spark_type.append(_parse_datatype_string(exc)) + except: + pass + + # Handle pandas types + include_numpy_type = [] + for inc in include: + try: + include_numpy_type.append(infer_dtype_from_object(inc)) + except: + pass + + exclude_numpy_type = [] + for exc in exclude: + try: + exclude_numpy_type.append(infer_dtype_from_object(exc)) + except: + pass + + column_labels = [] + for label in self._internal.column_labels: + if len(include) > 0: + should_include = ( + infer_dtype_from_object(self._kser_for(label).dtype.name) in include_numpy_type + or self._internal.spark_type_for(label) in include_spark_type + ) + else: + should_include = not ( + infer_dtype_from_object(self._kser_for(label).dtype.name) in exclude_numpy_type + or self._internal.spark_type_for(label) in exclude_spark_type + ) + + if should_include: + column_labels.append(label) + + return DataFrame( + self._internal.with_new_columns([self._kser_for(label) for label in column_labels]) + ) + + def droplevel(self, level, axis=0) -> "DataFrame": + """ + Return DataFrame with requested index / column level(s) removed. + + Parameters + ---------- + level: int, str, or list-like + If a string is given, must be the name of a level If list-like, elements must + be names or positional indexes of levels. + + axis: {0 or ‘index’, 1 or ‘columns’}, default 0 + + Returns + ------- + DataFrame with requested index / column level(s) removed. + + Examples + -------- + >>> df = pp.DataFrame( + ... [[3, 4], [7, 8], [11, 12]], + ... index=pd.MultiIndex.from_tuples([(1, 2), (5, 6), (9, 10)], names=["a", "b"]), + ... ) + + >>> df.columns = pd.MultiIndex.from_tuples([ + ... ('c', 'e'), ('d', 'f') + ... ], names=['level_1', 'level_2']) + + >>> df # doctest: +NORMALIZE_WHITESPACE + level_1 c d + level_2 e f + a b + 1 2 3 4 + 5 6 7 8 + 9 10 11 12 + + >>> df.droplevel('a') # doctest: +NORMALIZE_WHITESPACE + level_1 c d + level_2 e f + b + 2 3 4 + 6 7 8 + 10 11 12 + + >>> df.droplevel('level_2', axis=1) # doctest: +NORMALIZE_WHITESPACE + level_1 c d + a b + 1 2 3 4 + 5 6 7 8 + 9 10 11 12 + """ + axis = validate_axis(axis) + if axis == 0: + if not isinstance(level, (tuple, list)): # huh? + level = [level] + + index_names = self.index.names + nlevels = self._internal.index_level + + int_level = set() + for n in level: + if isinstance(n, int): + if n < 0: + n = n + nlevels + if n < 0: + raise IndexError( + "Too many levels: Index has only {} levels, " + "{} is not a valid level number".format(nlevels, (n - nlevels)) + ) + if n >= nlevels: + raise IndexError( + "Too many levels: Index has only {} levels, not {}".format( + nlevels, (n + 1) + ) + ) + else: + if n not in index_names: + raise KeyError("Level {} not found".format(n)) + n = index_names.index(n) + int_level.add(n) + + if len(level) >= nlevels: + raise ValueError( + "Cannot remove {} levels from an index with {} levels: " + "at least one level must be left.".format(len(level), nlevels) + ) + + index_spark_columns, index_names, index_dtypes = zip( + *[ + item + for i, item in enumerate( + zip( + self._internal.index_spark_columns, + self._internal.index_names, + self._internal.index_dtypes, + ) + ) + if i not in int_level + ] + ) + + internal = self._internal.copy( + index_spark_columns=list(index_spark_columns), + index_names=list(index_names), + index_dtypes=list(index_dtypes), + ) + return DataFrame(internal) + else: + kdf = self.copy() + kdf.columns = kdf.columns.droplevel(level) + return kdf + + def drop( + self, labels=None, axis=1, columns: Union[Any, Tuple, List[Any], List[Tuple]] = None + ) -> "DataFrame": + """ + Drop specified labels from columns. + + Remove columns by specifying label names and axis=1 or columns. + When specifying both labels and columns, only labels will be dropped. + Removing rows is yet to be implemented. + + Parameters + ---------- + labels : single label or list-like + Column labels to drop. + axis : {1 or 'columns'}, default 1 + .. dropna currently only works for axis=1 'columns' + axis=0 is yet to be implemented. + columns : single label or list-like + Alternative to specifying axis (``labels, axis=1`` + is equivalent to ``columns=labels``). + + Returns + ------- + dropped : DataFrame + + See Also + -------- + Series.dropna + + Examples + -------- + >>> df = pp.DataFrame({'x': [1, 2], 'y': [3, 4], 'z': [5, 6], 'w': [7, 8]}, + ... columns=['x', 'y', 'z', 'w']) + >>> df + x y z w + 0 1 3 5 7 + 1 2 4 6 8 + + >>> df.drop('x', axis=1) + y z w + 0 3 5 7 + 1 4 6 8 + + >>> df.drop(['y', 'z'], axis=1) + x w + 0 1 7 + 1 2 8 + + >>> df.drop(columns=['y', 'z']) + x w + 0 1 7 + 1 2 8 + + Also support for MultiIndex + + >>> df = pp.DataFrame({'x': [1, 2], 'y': [3, 4], 'z': [5, 6], 'w': [7, 8]}, + ... columns=['x', 'y', 'z', 'w']) + >>> columns = [('a', 'x'), ('a', 'y'), ('b', 'z'), ('b', 'w')] + >>> df.columns = pd.MultiIndex.from_tuples(columns) + >>> df # doctest: +NORMALIZE_WHITESPACE + a b + x y z w + 0 1 3 5 7 + 1 2 4 6 8 + >>> df.drop('a') # doctest: +NORMALIZE_WHITESPACE + b + z w + 0 5 7 + 1 6 8 + + Notes + ----- + Currently only axis = 1 is supported in this function, + axis = 0 is yet to be implemented. + """ + if labels is not None: + axis = validate_axis(axis) + if axis == 1: + return self.drop(columns=labels) + raise NotImplementedError("Drop currently only works for axis=1") + elif columns is not None: + if is_name_like_tuple(columns): + columns = [columns] + elif is_name_like_value(columns): + columns = [(columns,)] + else: + columns = [col if is_name_like_tuple(col) else (col,) for col in columns] + drop_column_labels = set( + label + for label in self._internal.column_labels + for col in columns + if label[: len(col)] == col + ) + if len(drop_column_labels) == 0: + raise KeyError(columns) + cols, labels = zip( + *( + (column, label) + for column, label in zip( + self._internal.data_spark_column_names, self._internal.column_labels + ) + if label not in drop_column_labels + ) + ) + internal = self._internal.with_new_columns([self._kser_for(label) for label in labels]) + return DataFrame(internal) + else: + raise ValueError("Need to specify at least one of 'labels' or 'columns'") + + def _sort( + self, by: List[Column], ascending: Union[bool, List[bool]], inplace: bool, na_position: str + ): + if isinstance(ascending, bool): + ascending = [ascending] * len(by) + if len(ascending) != len(by): + raise ValueError( + "Length of ascending ({}) != length of by ({})".format(len(ascending), len(by)) + ) + if na_position not in ("first", "last"): + raise ValueError("invalid na_position: '{}'".format(na_position)) + + # Mapper: Get a spark column function for (ascending, na_position) combination + # Note that 'asc_nulls_first' and friends were added as of Spark 2.4, see SPARK-23847. + mapper = { + (True, "first"): lambda x: Column(getattr(x._jc, "asc_nulls_first")()), + (True, "last"): lambda x: Column(getattr(x._jc, "asc_nulls_last")()), + (False, "first"): lambda x: Column(getattr(x._jc, "desc_nulls_first")()), + (False, "last"): lambda x: Column(getattr(x._jc, "desc_nulls_last")()), + } + by = [mapper[(asc, na_position)](scol) for scol, asc in zip(by, ascending)] + sdf = self._internal.resolved_copy.spark_frame.sort(*(by + [NATURAL_ORDER_COLUMN_NAME])) + kdf = DataFrame(self._internal.with_new_sdf(sdf)) # type: DataFrame + if inplace: + self._update_internal_frame(kdf._internal) + return None + else: + return kdf + + def sort_values( + self, + by: Union[Any, List[Any], Tuple, List[Tuple]], + ascending: Union[bool, List[bool]] = True, + inplace: bool = False, + na_position: str = "last", + ) -> Optional["DataFrame"]: + """ + Sort by the values along either axis. + + Parameters + ---------- + by : str or list of str + ascending : bool or list of bool, default True + Sort ascending vs. descending. Specify list for multiple sort + orders. If this is a list of bools, must match the length of + the by. + inplace : bool, default False + if True, perform operation in-place + na_position : {'first', 'last'}, default 'last' + `first` puts NaNs at the beginning, `last` puts NaNs at the end + + Returns + ------- + sorted_obj : DataFrame + + Examples + -------- + >>> df = pp.DataFrame({ + ... 'col1': ['A', 'B', None, 'D', 'C'], + ... 'col2': [2, 9, 8, 7, 4], + ... 'col3': [0, 9, 4, 2, 3], + ... }, + ... columns=['col1', 'col2', 'col3']) + >>> df + col1 col2 col3 + 0 A 2 0 + 1 B 9 9 + 2 None 8 4 + 3 D 7 2 + 4 C 4 3 + + Sort by col1 + + >>> df.sort_values(by=['col1']) + col1 col2 col3 + 0 A 2 0 + 1 B 9 9 + 4 C 4 3 + 3 D 7 2 + 2 None 8 4 + + Sort Descending + + >>> df.sort_values(by='col1', ascending=False) + col1 col2 col3 + 3 D 7 2 + 4 C 4 3 + 1 B 9 9 + 0 A 2 0 + 2 None 8 4 + + Sort by multiple columns + + >>> df = pp.DataFrame({ + ... 'col1': ['A', 'A', 'B', None, 'D', 'C'], + ... 'col2': [2, 1, 9, 8, 7, 4], + ... 'col3': [0, 1, 9, 4, 2, 3], + ... }, + ... columns=['col1', 'col2', 'col3']) + >>> df.sort_values(by=['col1', 'col2']) + col1 col2 col3 + 1 A 1 1 + 0 A 2 0 + 2 B 9 9 + 5 C 4 3 + 4 D 7 2 + 3 None 8 4 + """ + inplace = validate_bool_kwarg(inplace, "inplace") + if is_name_like_value(by): + by = [by] + else: + assert is_list_like(by), type(by) + + new_by = [] + for colname in by: + ser = self[colname] + if not isinstance(ser, pp.Series): + raise ValueError( + "The column %s is not unique. For a multi-index, the label must be a tuple " + "with elements corresponding to each level." % name_like_string(colname) + ) + new_by.append(ser.spark.column) + + return self._sort(by=new_by, ascending=ascending, inplace=inplace, na_position=na_position) + + def sort_index( + self, + axis: int = 0, + level: Optional[Union[int, List[int]]] = None, + ascending: bool = True, + inplace: bool = False, + kind: str = None, + na_position: str = "last", + ) -> Optional["DataFrame"]: + """ + Sort object by labels (along an axis) + + Parameters + ---------- + axis : index, columns to direct sorting. Currently, only axis = 0 is supported. + level : int or level name or list of ints or list of level names + if not None, sort on values in specified index level(s) + ascending : boolean, default True + Sort ascending vs. descending + inplace : bool, default False + if True, perform operation in-place + kind : str, default None + Koalas does not allow specifying the sorting algorithm at the moment, default None + na_position : {‘first’, ‘last’}, default ‘last’ + first puts NaNs at the beginning, last puts NaNs at the end. Not implemented for + MultiIndex. + + Returns + ------- + sorted_obj : DataFrame + + Examples + -------- + >>> df = pp.DataFrame({'A': [2, 1, np.nan]}, index=['b', 'a', np.nan]) + + >>> df.sort_index() + A + a 1.0 + b 2.0 + NaN NaN + + >>> df.sort_index(ascending=False) + A + b 2.0 + a 1.0 + NaN NaN + + >>> df.sort_index(na_position='first') + A + NaN NaN + a 1.0 + b 2.0 + + >>> df.sort_index(inplace=True) + >>> df + A + a 1.0 + b 2.0 + NaN NaN + + >>> df = pp.DataFrame({'A': range(4), 'B': range(4)[::-1]}, + ... index=[['b', 'b', 'a', 'a'], [1, 0, 1, 0]], + ... columns=['A', 'B']) + + >>> df.sort_index() + A B + a 0 3 0 + 1 2 1 + b 0 1 2 + 1 0 3 + + >>> df.sort_index(level=1) # doctest: +SKIP + A B + a 0 3 0 + b 0 1 2 + a 1 2 1 + b 1 0 3 + + >>> df.sort_index(level=[1, 0]) + A B + a 0 3 0 + b 0 1 2 + a 1 2 1 + b 1 0 3 + """ + inplace = validate_bool_kwarg(inplace, "inplace") + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError("No other axis than 0 are supported at the moment") + if kind is not None: + raise NotImplementedError( + "Specifying the sorting algorithm is not supported at the moment." + ) + + if level is None or (is_list_like(level) and len(level) == 0): # type: ignore + by = self._internal.index_spark_columns + elif is_list_like(level): + by = [self._internal.index_spark_columns[l] for l in level] # type: ignore + else: + by = [self._internal.index_spark_columns[level]] # type: ignore + + return self._sort(by=by, ascending=ascending, inplace=inplace, na_position=na_position) + + def swaplevel(self, i=-2, j=-1, axis=0) -> "DataFrame": + """ + Swap levels i and j in a MultiIndex on a particular axis. + + Parameters + ---------- + i, j : int or str + Levels of the indices to be swapped. Can pass level name as string. + axis : {0 or 'index', 1 or 'columns'}, default 0 + The axis to swap levels on. 0 or 'index' for row-wise, 1 or + 'columns' for column-wise. + + Returns + ------- + DataFrame + DataFrame with levels swapped in MultiIndex. + + Examples + -------- + >>> midx = pd.MultiIndex.from_arrays( + ... [['red', 'blue'], [1, 2], ['s', 'm']], names = ['color', 'number', 'size']) + >>> midx # doctest: +SKIP + MultiIndex([( 'red', 1, 's'), + ('blue', 2, 'm')], + names=['color', 'number', 'size']) + + Swap levels in a MultiIndex on index. + + >>> kdf = pp.DataFrame({'x': [5, 6], 'y':[5, 6]}, index=midx) + >>> kdf # doctest: +NORMALIZE_WHITESPACE + x y + color number size + red 1 s 5 5 + blue 2 m 6 6 + + >>> kdf.swaplevel() # doctest: +NORMALIZE_WHITESPACE + x y + color size number + red s 1 5 5 + blue m 2 6 6 + + >>> kdf.swaplevel(0, 1) # doctest: +NORMALIZE_WHITESPACE + x y + number color size + 1 red s 5 5 + 2 blue m 6 6 + + >>> kdf.swaplevel('number', 'size') # doctest: +NORMALIZE_WHITESPACE + x y + color size number + red s 1 5 5 + blue m 2 6 6 + + Swap levels in a MultiIndex on columns. + + >>> kdf = pp.DataFrame({'x': [5, 6], 'y':[5, 6]}) + >>> kdf.columns = midx + >>> kdf + color red blue + number 1 2 + size s m + 0 5 5 + 1 6 6 + + >>> kdf.swaplevel(axis=1) + color red blue + size s m + number 1 2 + 0 5 5 + 1 6 6 + + >>> kdf.swaplevel(axis=1) + color red blue + size s m + number 1 2 + 0 5 5 + 1 6 6 + + >>> kdf.swaplevel(0, 1, axis=1) + number 1 2 + color red blue + size s m + 0 5 5 + 1 6 6 + + >>> kdf.swaplevel('number', 'color', axis=1) + number 1 2 + color red blue + size s m + 0 5 5 + 1 6 6 + """ + axis = validate_axis(axis) + if axis == 0: + internal = self._swaplevel_index(i, j) + else: + assert axis == 1 + internal = self._swaplevel_columns(i, j) + + return DataFrame(internal) + + def swapaxes(self, i: Union[str, int], j: Union[str, int], copy: bool = True) -> "DataFrame": + """ + Interchange axes and swap values axes appropriately. + + .. note:: This method is based on an expensive operation due to the nature + of big data. Internally it needs to generate each row for each value, and + then group twice - it is a huge operation. To prevent misusage, this method + has the 'compute.max_rows' default limit of input length, and raises a ValueError. + + >>> from pyspark.pandas.config import option_context + >>> with option_context('compute.max_rows', 1000): # doctest: +NORMALIZE_WHITESPACE + ... pp.DataFrame({'a': range(1001)}).swapaxes(i=0, j=1) + Traceback (most recent call last): + ... + ValueError: Current DataFrame has more then the given limit 1000 rows. + Please set 'compute.max_rows' by using 'pyspark.pandas.config.set_option' + to retrieve to retrieve more than 1000 rows. Note that, before changing the + 'compute.max_rows', this operation is considerably expensive. + + Parameters + ---------- + i: {0 or 'index', 1 or 'columns'}. The axis to swap. + j: {0 or 'index', 1 or 'columns'}. The axis to swap. + copy : bool, default True. + + Returns + ------- + DataFrame + + Examples + -------- + >>> kdf = pp.DataFrame( + ... [[1, 2, 3], [4, 5, 6], [7, 8, 9]], index=['x', 'y', 'z'], columns=['a', 'b', 'c'] + ... ) + >>> kdf + a b c + x 1 2 3 + y 4 5 6 + z 7 8 9 + >>> kdf.swapaxes(i=1, j=0) + x y z + a 1 4 7 + b 2 5 8 + c 3 6 9 + >>> kdf.swapaxes(i=1, j=1) + a b c + x 1 2 3 + y 4 5 6 + z 7 8 9 + """ + assert copy is True + + i = validate_axis(i) + j = validate_axis(j) + + return self.copy() if i == j else self.transpose() + + def _swaplevel_columns(self, i, j) -> InternalFrame: + assert isinstance(self.columns, pd.MultiIndex) + for index in (i, j): + if not isinstance(index, int) and index not in self.columns.names: + raise KeyError("Level %s not found" % index) + + i = i if isinstance(i, int) else self.columns.names.index(i) + j = j if isinstance(j, int) else self.columns.names.index(j) + for index in (i, j): + if index >= len(self.columns) or index < -len(self.columns): + raise IndexError( + "Too many levels: Columns have only %s levels, " + "%s is not a valid level number" % (self._internal.index_level, index) + ) + + column_label_names = self._internal.column_label_names.copy() + column_label_names[i], column_label_names[j], = ( + column_label_names[j], + column_label_names[i], + ) + column_labels = self._internal._column_labels + column_label_list = [list(label) for label in column_labels] + for label_list in column_label_list: + label_list[i], label_list[j] = label_list[j], label_list[i] + column_labels = [tuple(x) for x in column_label_list] + internal = self._internal.copy( + column_label_names=list(column_label_names), column_labels=list(column_labels) + ) + return internal + + def _swaplevel_index(self, i, j) -> InternalFrame: + assert isinstance(self.index, pp.MultiIndex) + for index in (i, j): + if not isinstance(index, int) and index not in self.index.names: + raise KeyError("Level %s not found" % index) + + i = i if isinstance(i, int) else self.index.names.index(i) + j = j if isinstance(j, int) else self.index.names.index(j) + for index in (i, j): + if index >= self._internal.index_level or index < -self._internal.index_level: + raise IndexError( + "Too many levels: Index has only %s levels, " + "%s is not a valid level number" % (self._internal.index_level, index) + ) + + index_map = list( + zip( + self._internal.index_spark_columns, + self._internal.index_names, + self._internal.index_dtypes, + ) + ) + index_map[i], index_map[j], = index_map[j], index_map[i] + index_spark_columns, index_names, index_dtypes = zip(*index_map) + internal = self._internal.copy( + index_spark_columns=list(index_spark_columns), + index_names=list(index_names), + index_dtypes=list(index_dtypes), + ) + return internal + + # TODO: add keep = First + def nlargest(self, n: int, columns: "Any") -> "DataFrame": + """ + Return the first `n` rows ordered by `columns` in descending order. + + Return the first `n` rows with the largest values in `columns`, in + descending order. The columns that are not specified are returned as + well, but not used for ordering. + + This method is equivalent to + ``df.sort_values(columns, ascending=False).head(n)``, but more + performant in pandas. + In Koalas, thanks to Spark's lazy execution and query optimizer, + the two would have same performance. + + Parameters + ---------- + n : int + Number of rows to return. + columns : label or list of labels + Column label(s) to order by. + + Returns + ------- + DataFrame + The first `n` rows ordered by the given columns in descending + order. + + See Also + -------- + DataFrame.nsmallest : Return the first `n` rows ordered by `columns` in + ascending order. + DataFrame.sort_values : Sort DataFrame by the values. + DataFrame.head : Return the first `n` rows without re-ordering. + + Notes + ----- + + This function cannot be used with all column types. For example, when + specifying columns with `object` or `category` dtypes, ``TypeError`` is + raised. + + Examples + -------- + >>> df = pp.DataFrame({'X': [1, 2, 3, 5, 6, 7, np.nan], + ... 'Y': [6, 7, 8, 9, 10, 11, 12]}) + >>> df + X Y + 0 1.0 6 + 1 2.0 7 + 2 3.0 8 + 3 5.0 9 + 4 6.0 10 + 5 7.0 11 + 6 NaN 12 + + In the following example, we will use ``nlargest`` to select the three + rows having the largest values in column "population". + + >>> df.nlargest(n=3, columns='X') + X Y + 5 7.0 11 + 4 6.0 10 + 3 5.0 9 + + >>> df.nlargest(n=3, columns=['Y', 'X']) + X Y + 6 NaN 12 + 5 7.0 11 + 4 6.0 10 + + """ + return self.sort_values(by=columns, ascending=False).head(n=n) + + # TODO: add keep = First + def nsmallest(self, n: int, columns: "Any") -> "DataFrame": + """ + Return the first `n` rows ordered by `columns` in ascending order. + + Return the first `n` rows with the smallest values in `columns`, in + ascending order. The columns that are not specified are returned as + well, but not used for ordering. + + This method is equivalent to ``df.sort_values(columns, ascending=True).head(n)``, + but more performant. In Koalas, thanks to Spark's lazy execution and query optimizer, + the two would have same performance. + + Parameters + ---------- + n : int + Number of items to retrieve. + columns : list or str + Column name or names to order by. + + Returns + ------- + DataFrame + + See Also + -------- + DataFrame.nlargest : Return the first `n` rows ordered by `columns` in + descending order. + DataFrame.sort_values : Sort DataFrame by the values. + DataFrame.head : Return the first `n` rows without re-ordering. + + Examples + -------- + >>> df = pp.DataFrame({'X': [1, 2, 3, 5, 6, 7, np.nan], + ... 'Y': [6, 7, 8, 9, 10, 11, 12]}) + >>> df + X Y + 0 1.0 6 + 1 2.0 7 + 2 3.0 8 + 3 5.0 9 + 4 6.0 10 + 5 7.0 11 + 6 NaN 12 + + In the following example, we will use ``nsmallest`` to select the + three rows having the smallest values in column "a". + + >>> df.nsmallest(n=3, columns='X') # doctest: +NORMALIZE_WHITESPACE + X Y + 0 1.0 6 + 1 2.0 7 + 2 3.0 8 + + To order by the largest values in column "a" and then "c", we can + specify multiple columns like in the next example. + + >>> df.nsmallest(n=3, columns=['Y', 'X']) # doctest: +NORMALIZE_WHITESPACE + X Y + 0 1.0 6 + 1 2.0 7 + 2 3.0 8 + """ + return self.sort_values(by=columns, ascending=True).head(n=n) + + def isin(self, values) -> "DataFrame": + """ + Whether each element in the DataFrame is contained in values. + + Parameters + ---------- + values : iterable or dict + The sequence of values to test. If values is a dict, + the keys must be the column names, which must match. + Series and DataFrame are not supported. + + Returns + ------- + DataFrame + DataFrame of booleans showing whether each element in the DataFrame + is contained in values. + + Examples + -------- + >>> df = pp.DataFrame({'num_legs': [2, 4], 'num_wings': [2, 0]}, + ... index=['falcon', 'dog'], + ... columns=['num_legs', 'num_wings']) + >>> df + num_legs num_wings + falcon 2 2 + dog 4 0 + + When ``values`` is a list check whether every value in the DataFrame + is present in the list (which animals have 0 or 2 legs or wings) + + >>> df.isin([0, 2]) + num_legs num_wings + falcon True True + dog False True + + When ``values`` is a dict, we can pass values to check for each + column separately: + + >>> df.isin({'num_wings': [0, 3]}) + num_legs num_wings + falcon False False + dog False True + """ + if isinstance(values, (pd.DataFrame, pd.Series)): + raise NotImplementedError("DataFrame and Series are not supported") + if isinstance(values, dict) and not set(values.keys()).issubset(self.columns): + raise AttributeError( + "'DataFrame' object has no attribute %s" + % (set(values.keys()).difference(self.columns)) + ) + + data_spark_columns = [] + if isinstance(values, dict): + for i, col in enumerate(self.columns): + if col in values: + item = values[col] + item = item.tolist() if isinstance(item, np.ndarray) else list(item) + data_spark_columns.append( + self._internal.spark_column_for(self._internal.column_labels[i]) + .isin(item) + .alias(self._internal.data_spark_column_names[i]) + ) + else: + data_spark_columns.append( + F.lit(False).alias(self._internal.data_spark_column_names[i]) + ) + elif is_list_like(values): + values = values.tolist() if isinstance(values, np.ndarray) else list(values) + data_spark_columns += [ + self._internal.spark_column_for(label) + .isin(values) + .alias(self._internal.spark_column_name_for(label)) + for label in self._internal.column_labels + ] + else: + raise TypeError("Values should be iterable, Series, DataFrame or dict.") + + return DataFrame(self._internal.with_new_columns(data_spark_columns)) + + @property + def shape(self) -> Tuple[int, int]: + """ + Return a tuple representing the dimensionality of the DataFrame. + + Examples + -------- + >>> df = pp.DataFrame({'col1': [1, 2], 'col2': [3, 4]}) + >>> df.shape + (2, 2) + + >>> df = pp.DataFrame({'col1': [1, 2], 'col2': [3, 4], + ... 'col3': [5, 6]}) + >>> df.shape + (2, 3) + """ + return len(self), len(self.columns) + + def merge( + self, + right: "DataFrame", + how: str = "inner", + on: Optional[Union[Any, List[Any], Tuple, List[Tuple]]] = None, + left_on: Optional[Union[Any, List[Any], Tuple, List[Tuple]]] = None, + right_on: Optional[Union[Any, List[Any], Tuple, List[Tuple]]] = None, + left_index: bool = False, + right_index: bool = False, + suffixes: Tuple[str, str] = ("_x", "_y"), + ) -> "DataFrame": + """ + Merge DataFrame objects with a database-style join. + + The index of the resulting DataFrame will be one of the following: + - 0...n if no index is used for merging + - Index of the left DataFrame if merged only on the index of the right DataFrame + - Index of the right DataFrame if merged only on the index of the left DataFrame + - All involved indices if merged using the indices of both DataFrames + e.g. if `left` with indices (a, x) and `right` with indices (b, x), the result will + be an index (x, a, b) + + Parameters + ---------- + right: Object to merge with. + how: Type of merge to be performed. + {'left', 'right', 'outer', 'inner'}, default 'inner' + + left: use only keys from left frame, similar to a SQL left outer join; not preserve + key order unlike pandas. + right: use only keys from right frame, similar to a SQL right outer join; not preserve + key order unlike pandas. + outer: use union of keys from both frames, similar to a SQL full outer join; sort keys + lexicographically. + inner: use intersection of keys from both frames, similar to a SQL inner join; + not preserve the order of the left keys unlike pandas. + on: Column or index level names to join on. These must be found in both DataFrames. If on + is None and not merging on indexes then this defaults to the intersection of the + columns in both DataFrames. + left_on: Column or index level names to join on in the left DataFrame. Can also + be an array or list of arrays of the length of the left DataFrame. + These arrays are treated as if they are columns. + right_on: Column or index level names to join on in the right DataFrame. Can also + be an array or list of arrays of the length of the right DataFrame. + These arrays are treated as if they are columns. + left_index: Use the index from the left DataFrame as the join key(s). If it is a + MultiIndex, the number of keys in the other DataFrame (either the index or a number of + columns) must match the number of levels. + right_index: Use the index from the right DataFrame as the join key. Same caveats as + left_index. + suffixes: Suffix to apply to overlapping column names in the left and right side, + respectively. + + Returns + ------- + DataFrame + A DataFrame of the two merged objects. + + See Also + -------- + DataFrame.join : Join columns of another DataFrame. + DataFrame.update : Modify in place using non-NA values from another DataFrame. + DataFrame.hint : Specifies some hint on the current DataFrame. + broadcast : Marks a DataFrame as small enough for use in broadcast joins. + + Examples + -------- + >>> df1 = pp.DataFrame({'lkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [1, 2, 3, 5]}, + ... columns=['lkey', 'value']) + >>> df2 = pp.DataFrame({'rkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [5, 6, 7, 8]}, + ... columns=['rkey', 'value']) + >>> df1 + lkey value + 0 foo 1 + 1 bar 2 + 2 baz 3 + 3 foo 5 + >>> df2 + rkey value + 0 foo 5 + 1 bar 6 + 2 baz 7 + 3 foo 8 + + Merge df1 and df2 on the lkey and rkey columns. The value columns have + the default suffixes, _x and _y, appended. + + >>> merged = df1.merge(df2, left_on='lkey', right_on='rkey') + >>> merged.sort_values(by=['lkey', 'value_x', 'rkey', 'value_y']) # doctest: +ELLIPSIS + lkey value_x rkey value_y + ...bar 2 bar 6 + ...baz 3 baz 7 + ...foo 1 foo 5 + ...foo 1 foo 8 + ...foo 5 foo 5 + ...foo 5 foo 8 + + >>> left_kdf = pp.DataFrame({'A': [1, 2]}) + >>> right_kdf = pp.DataFrame({'B': ['x', 'y']}, index=[1, 2]) + + >>> left_kdf.merge(right_kdf, left_index=True, right_index=True).sort_index() + A B + 1 2 x + + >>> left_kdf.merge(right_kdf, left_index=True, right_index=True, how='left').sort_index() + A B + 0 1 None + 1 2 x + + >>> left_kdf.merge(right_kdf, left_index=True, right_index=True, how='right').sort_index() + A B + 1 2.0 x + 2 NaN y + + >>> left_kdf.merge(right_kdf, left_index=True, right_index=True, how='outer').sort_index() + A B + 0 1.0 None + 1 2.0 x + 2 NaN y + + Notes + ----- + As described in #263, joining string columns currently returns None for missing values + instead of NaN. + """ + + def to_list(os: Optional[Union[Any, List[Any], Tuple, List[Tuple]]]) -> List[Tuple]: + if os is None: + return [] + elif is_name_like_tuple(os): + return [os] # type: ignore + elif is_name_like_value(os): + return [(os,)] + else: + return [o if is_name_like_tuple(o) else (o,) for o in os] + + if isinstance(right, pp.Series): + right = right.to_frame() + + if on: + if left_on or right_on: + raise ValueError( + 'Can only pass argument "on" OR "left_on" and "right_on", ' + "not a combination of both." + ) + left_key_names = list(map(self._internal.spark_column_name_for, to_list(on))) + right_key_names = list(map(right._internal.spark_column_name_for, to_list(on))) + else: + # TODO: need special handling for multi-index. + if left_index: + left_key_names = self._internal.index_spark_column_names + else: + left_key_names = list(map(self._internal.spark_column_name_for, to_list(left_on))) + if right_index: + right_key_names = right._internal.index_spark_column_names + else: + right_key_names = list( + map(right._internal.spark_column_name_for, to_list(right_on)) + ) + + if left_key_names and not right_key_names: + raise ValueError("Must pass right_on or right_index=True") + if right_key_names and not left_key_names: + raise ValueError("Must pass left_on or left_index=True") + if not left_key_names and not right_key_names: + common = list(self.columns.intersection(right.columns)) + if len(common) == 0: + raise ValueError( + "No common columns to perform merge on. Merge options: " + "left_on=None, right_on=None, left_index=False, right_index=False" + ) + left_key_names = list(map(self._internal.spark_column_name_for, to_list(common))) + right_key_names = list(map(right._internal.spark_column_name_for, to_list(common))) + if len(left_key_names) != len(right_key_names): + raise ValueError("len(left_keys) must equal len(right_keys)") + + # We should distinguish the name to avoid ambiguous column name after merging. + right_prefix = "__right_" + right_key_names = [right_prefix + right_key_name for right_key_name in right_key_names] + + how = validate_how(how) + + def resolve(internal, side): + rename = lambda col: "__{}_{}".format(side, col) + internal = internal.resolved_copy + sdf = internal.spark_frame + sdf = sdf.select( + [ + scol_for(sdf, col).alias(rename(col)) + for col in sdf.columns + if col not in HIDDEN_COLUMNS + ] + + list(HIDDEN_COLUMNS) + ) + return internal.copy( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, rename(col)) for col in internal.index_spark_column_names + ], + data_spark_columns=[ + scol_for(sdf, rename(col)) for col in internal.data_spark_column_names + ], + ) + + left_internal = self._internal.resolved_copy + right_internal = resolve(right._internal, "right") + + left_table = left_internal.spark_frame.alias("left_table") + right_table = right_internal.spark_frame.alias("right_table") + + left_key_columns = [scol_for(left_table, label) for label in left_key_names] + right_key_columns = [scol_for(right_table, label) for label in right_key_names] + + join_condition = reduce( + lambda x, y: x & y, + [lkey == rkey for lkey, rkey in zip(left_key_columns, right_key_columns)], + ) + + joined_table = left_table.join(right_table, join_condition, how=how) + + # Unpack suffixes tuple for convenience + left_suffix = suffixes[0] + right_suffix = suffixes[1] + + # Append suffixes to columns with the same name to avoid conflicts later + duplicate_columns = set(left_internal.column_labels) & set(right_internal.column_labels) + + exprs = [] + data_columns = [] + column_labels = [] + + left_scol_for = lambda label: scol_for( + left_table, left_internal.spark_column_name_for(label) + ) + right_scol_for = lambda label: scol_for( + right_table, right_internal.spark_column_name_for(label) + ) + + for label in left_internal.column_labels: + col = left_internal.spark_column_name_for(label) + scol = left_scol_for(label) + if label in duplicate_columns: + spark_column_name = left_internal.spark_column_name_for(label) + if ( + spark_column_name in left_key_names + and (right_prefix + spark_column_name) in right_key_names + ): + right_scol = right_scol_for(label) + if how == "right": + scol = right_scol.alias(col) + elif how == "full": + scol = F.when(scol.isNotNull(), scol).otherwise(right_scol).alias(col) + else: + pass + else: + col = col + left_suffix + scol = scol.alias(col) + label = tuple([str(label[0]) + left_suffix] + list(label[1:])) + exprs.append(scol) + data_columns.append(col) + column_labels.append(label) + for label in right_internal.column_labels: + # recover `right_prefix` here. + col = right_internal.spark_column_name_for(label)[len(right_prefix):] + scol = right_scol_for(label).alias(col) + if label in duplicate_columns: + spark_column_name = left_internal.spark_column_name_for(label) + if ( + spark_column_name in left_key_names + and (right_prefix + spark_column_name) in right_key_names + ): + continue + else: + col = col + right_suffix + scol = scol.alias(col) + label = tuple([str(label[0]) + right_suffix] + list(label[1:])) + exprs.append(scol) + data_columns.append(col) + column_labels.append(label) + + left_index_scols = left_internal.index_spark_columns + right_index_scols = right_internal.index_spark_columns + + # Retain indices if they are used for joining + if left_index: + if right_index: + if how in ("inner", "left"): + exprs.extend(left_index_scols) + index_spark_column_names = left_internal.index_spark_column_names + index_names = left_internal.index_names + elif how == "right": + exprs.extend(right_index_scols) + index_spark_column_names = right_internal.index_spark_column_names + index_names = right_internal.index_names + else: + index_spark_column_names = left_internal.index_spark_column_names + index_names = left_internal.index_names + for col, left_scol, right_scol in zip( + index_spark_column_names, left_index_scols, right_index_scols + ): + scol = F.when(left_scol.isNotNull(), left_scol).otherwise(right_scol) + exprs.append(scol.alias(col)) + else: + exprs.extend(right_index_scols) + index_spark_column_names = right_internal.index_spark_column_names + index_names = right_internal.index_names + elif right_index: + exprs.extend(left_index_scols) + index_spark_column_names = left_internal.index_spark_column_names + index_names = left_internal.index_names + else: + index_spark_column_names = [] + index_names = [] + + selected_columns = joined_table.select(*exprs) + + internal = InternalFrame( + spark_frame=selected_columns, + index_spark_columns=[ + scol_for(selected_columns, col) for col in index_spark_column_names + ], + index_names=index_names, + column_labels=column_labels, + data_spark_columns=[scol_for(selected_columns, col) for col in data_columns], + ) + return DataFrame(internal) + + def join( + self, + right: "DataFrame", + on: Optional[Union[Any, List[Any], Tuple, List[Tuple]]] = None, + how: str = "left", + lsuffix: str = "", + rsuffix: str = "", + ) -> "DataFrame": + """ + Join columns of another DataFrame. + + Join columns with `right` DataFrame either on index or on a key column. Efficiently join + multiple DataFrame objects by index at once by passing a list. + + Parameters + ---------- + right: DataFrame, Series + on: str, list of str, or array-like, optional + Column or index level name(s) in the caller to join on the index in `right`, otherwise + joins index-on-index. If multiple values given, the `right` DataFrame must have a + MultiIndex. Can pass an array as the join key if it is not already contained in the + calling DataFrame. Like an Excel VLOOKUP operation. + how: {'left', 'right', 'outer', 'inner'}, default 'left' + How to handle the operation of the two objects. + + * left: use `left` frame’s index (or column if on is specified). + * right: use `right`’s index. + * outer: form union of `left` frame’s index (or column if on is specified) with + right’s index, and sort it. lexicographically. + * inner: form intersection of `left` frame’s index (or column if on is specified) + with `right`’s index, preserving the order of the `left`’s one. + lsuffix : str, default '' + Suffix to use from left frame's overlapping columns. + rsuffix : str, default '' + Suffix to use from `right` frame's overlapping columns. + + Returns + ------- + DataFrame + A dataframe containing columns from both the `left` and `right`. + + See Also + -------- + DataFrame.merge: For column(s)-on-columns(s) operations. + DataFrame.update : Modify in place using non-NA values from another DataFrame. + DataFrame.hint : Specifies some hint on the current DataFrame. + broadcast : Marks a DataFrame as small enough for use in broadcast joins. + + Notes + ----- + Parameters on, lsuffix, and rsuffix are not supported when passing a list of DataFrame + objects. + + Examples + -------- + >>> kdf1 = pp.DataFrame({'key': ['K0', 'K1', 'K2', 'K3'], + ... 'A': ['A0', 'A1', 'A2', 'A3']}, + ... columns=['key', 'A']) + >>> kdf2 = pp.DataFrame({'key': ['K0', 'K1', 'K2'], + ... 'B': ['B0', 'B1', 'B2']}, + ... columns=['key', 'B']) + >>> kdf1 + key A + 0 K0 A0 + 1 K1 A1 + 2 K2 A2 + 3 K3 A3 + >>> kdf2 + key B + 0 K0 B0 + 1 K1 B1 + 2 K2 B2 + + Join DataFrames using their indexes. + + >>> join_kdf = kdf1.join(kdf2, lsuffix='_left', rsuffix='_right') + >>> join_kdf.sort_values(by=join_kdf.columns) + key_left A key_right B + 0 K0 A0 K0 B0 + 1 K1 A1 K1 B1 + 2 K2 A2 K2 B2 + 3 K3 A3 None None + + If we want to join using the key columns, we need to set key to be the index in both df and + right. The joined DataFrame will have key as its index. + + >>> join_kdf = kdf1.set_index('key').join(kdf2.set_index('key')) + >>> join_kdf.sort_values(by=join_kdf.columns) # doctest: +NORMALIZE_WHITESPACE + A B + key + K0 A0 B0 + K1 A1 B1 + K2 A2 B2 + K3 A3 None + + Another option to join using the key columns is to use the on parameter. DataFrame.join + always uses right’s index but we can use any column in df. This method not preserve the + original DataFrame’s index in the result unlike pandas. + + >>> join_kdf = kdf1.join(kdf2.set_index('key'), on='key') + >>> join_kdf.index + Int64Index([0, 1, 2, 3], dtype='int64') + """ + if isinstance(right, pp.Series): + common = list(self.columns.intersection([right.name])) + else: + common = list(self.columns.intersection(right.columns)) + if len(common) > 0 and not lsuffix and not rsuffix: + raise ValueError( + "columns overlap but no suffix specified: " "{rename}".format(rename=common) + ) + + need_set_index = False + if on: + if not is_list_like(on): + on = [on] # type: ignore + if len(on) != right._internal.index_level: + raise ValueError( + 'len(left_on) must equal the number of levels in the index of "right"' + ) + + need_set_index = len(set(on) & set(self.index.names)) == 0 + if need_set_index: + self = self.set_index(on) + join_kdf = self.merge( + right, left_index=True, right_index=True, how=how, suffixes=(lsuffix, rsuffix) + ) + return join_kdf.reset_index() if need_set_index else join_kdf + + def append( + self, + other: "DataFrame", + ignore_index: bool = False, + verify_integrity: bool = False, + sort: bool = False, + ) -> "DataFrame": + """ + Append rows of other to the end of caller, returning a new object. + + Columns in other that are not in the caller are added as new columns. + + Parameters + ---------- + other : DataFrame or Series/dict-like object, or list of these + The data to append. + + ignore_index : boolean, default False + If True, do not use the index labels. + + verify_integrity : boolean, default False + If True, raise ValueError on creating index with duplicates. + + sort : boolean, default False + Currently not supported. + + Returns + ------- + appended : DataFrame + + Examples + -------- + >>> df = pp.DataFrame([[1, 2], [3, 4]], columns=list('AB')) + + >>> df.append(df) + A B + 0 1 2 + 1 3 4 + 0 1 2 + 1 3 4 + + >>> df.append(df, ignore_index=True) + A B + 0 1 2 + 1 3 4 + 2 1 2 + 3 3 4 + """ + if isinstance(other, pp.Series): + raise ValueError("DataFrames.append() does not support appending Series to DataFrames") + if sort: + raise NotImplementedError("The 'sort' parameter is currently not supported") + + if not ignore_index: + index_scols = self._internal.index_spark_columns + if len(index_scols) != other._internal.index_level: + raise ValueError("Both DataFrames have to have the same number of index levels") + + if verify_integrity and len(index_scols) > 0: + if ( + self._internal.spark_frame.select(index_scols) + .intersect( + other._internal.spark_frame.select(other._internal.index_spark_columns) + ) + .count() + ) > 0: + raise ValueError("Indices have overlapping values") + + # Lazy import to avoid circular dependency issues + from pyspark.pandas.namespace import concat + + return cast(DataFrame, concat([self, other], ignore_index=ignore_index)) + + # TODO: add 'filter_func' and 'errors' parameter + def update(self, other: "DataFrame", join: str = "left", overwrite: bool = True) -> None: + """ + Modify in place using non-NA values from another DataFrame. + Aligns on indices. There is no return value. + + Parameters + ---------- + other : DataFrame, or Series + join : 'left', default 'left' + Only left join is implemented, keeping the index and columns of the original object. + overwrite : bool, default True + How to handle non-NA values for overlapping keys: + + * True: overwrite original DataFrame's values with values from `other`. + * False: only update values that are NA in the original DataFrame. + + Returns + ------- + None : method directly changes calling object + + See Also + -------- + DataFrame.merge : For column(s)-on-columns(s) operations. + DataFrame.join : Join columns of another DataFrame. + DataFrame.hint : Specifies some hint on the current DataFrame. + broadcast : Marks a DataFrame as small enough for use in broadcast joins. + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2, 3], 'B': [400, 500, 600]}, columns=['A', 'B']) + >>> new_df = pp.DataFrame({'B': [4, 5, 6], 'C': [7, 8, 9]}, columns=['B', 'C']) + >>> df.update(new_df) + >>> df.sort_index() + A B + 0 1 4 + 1 2 5 + 2 3 6 + + The DataFrame's length does not increase as a result of the update, + only values at matching index/column labels are updated. + + >>> df = pp.DataFrame({'A': ['a', 'b', 'c'], 'B': ['x', 'y', 'z']}, columns=['A', 'B']) + >>> new_df = pp.DataFrame({'B': ['d', 'e', 'f', 'g', 'h', 'i']}, columns=['B']) + >>> df.update(new_df) + >>> df.sort_index() + A B + 0 a d + 1 b e + 2 c f + + For Series, it's name attribute must be set. + + >>> df = pp.DataFrame({'A': ['a', 'b', 'c'], 'B': ['x', 'y', 'z']}, columns=['A', 'B']) + >>> new_column = pp.Series(['d', 'e'], name='B', index=[0, 2]) + >>> df.update(new_column) + >>> df.sort_index() + A B + 0 a d + 1 b y + 2 c e + + If `other` contains None the corresponding values are not updated in the original dataframe. + + >>> df = pp.DataFrame({'A': [1, 2, 3], 'B': [400, 500, 600]}, columns=['A', 'B']) + >>> new_df = pp.DataFrame({'B': [4, None, 6]}, columns=['B']) + >>> df.update(new_df) + >>> df.sort_index() + A B + 0 1 4.0 + 1 2 500.0 + 2 3 6.0 + """ + if join != "left": + raise NotImplementedError("Only left join is supported") + + if isinstance(other, pp.Series): + other = other.to_frame() + + update_columns = list( + set(self._internal.column_labels).intersection(set(other._internal.column_labels)) + ) + update_sdf = self.join( + other[update_columns], rsuffix="_new" + )._internal.resolved_copy.spark_frame + + data_dtypes = self._internal.data_dtypes.copy() + for column_labels in update_columns: + column_name = self._internal.spark_column_name_for(column_labels) + old_col = scol_for(update_sdf, column_name) + new_col = scol_for( + update_sdf, other._internal.spark_column_name_for(column_labels) + "_new" + ) + if overwrite: + update_sdf = update_sdf.withColumn( + column_name, F.when(new_col.isNull(), old_col).otherwise(new_col) + ) + else: + update_sdf = update_sdf.withColumn( + column_name, F.when(old_col.isNull(), new_col).otherwise(old_col) + ) + data_dtypes[self._internal.column_labels.index(column_labels)] = None # TODO: dtype? + sdf = update_sdf.select( + [scol_for(update_sdf, col) for col in self._internal.spark_column_names] + + list(HIDDEN_COLUMNS) + ) + internal = self._internal.with_new_sdf(sdf, data_dtypes=data_dtypes) + self._update_internal_frame(internal, requires_same_anchor=False) + + def sample( + self, + n: Optional[int] = None, + frac: Optional[float] = None, + replace: bool = False, + random_state: Optional[int] = None, + ) -> "DataFrame": + """ + Return a random sample of items from an axis of object. + + Please call this function using named argument by specifying the ``frac`` argument. + + You can use `random_state` for reproducibility. However, note that different from pandas, + specifying a seed in Koalas/Spark does not guarantee the sampled rows will be fixed. The + result set depends on not only the seed, but also how the data is distributed across + machines and to some extent network randomness when shuffle operations are involved. Even + in the simplest case, the result set will depend on the system's CPU core count. + + Parameters + ---------- + n : int, optional + Number of items to return. This is currently NOT supported. Use frac instead. + frac : float, optional + Fraction of axis items to return. + replace : bool, default False + Sample with or without replacement. + random_state : int, optional + Seed for the random number generator (if int). + + Returns + ------- + Series or DataFrame + A new object of same type as caller containing the sampled items. + + Examples + -------- + >>> df = pp.DataFrame({'num_legs': [2, 4, 8, 0], + ... 'num_wings': [2, 0, 0, 0], + ... 'num_specimen_seen': [10, 2, 1, 8]}, + ... index=['falcon', 'dog', 'spider', 'fish'], + ... columns=['num_legs', 'num_wings', 'num_specimen_seen']) + >>> df # doctest: +SKIP + num_legs num_wings num_specimen_seen + falcon 2 2 10 + dog 4 0 2 + spider 8 0 1 + fish 0 0 8 + + A random 25% sample of the ``DataFrame``. + Note that we use `random_state` to ensure the reproducibility of + the examples. + + >>> df.sample(frac=0.25, random_state=1) # doctest: +SKIP + num_legs num_wings num_specimen_seen + falcon 2 2 10 + fish 0 0 8 + + Extract 25% random elements from the ``Series`` ``df['num_legs']``, with replacement, + so the same items could appear more than once. + + >>> df['num_legs'].sample(frac=0.4, replace=True, random_state=1) # doctest: +SKIP + falcon 2 + spider 8 + spider 8 + Name: num_legs, dtype: int64 + + Specifying the exact number of items to return is not supported at the moment. + + >>> df.sample(n=5) # doctest: +ELLIPSIS + Traceback (most recent call last): + ... + NotImplementedError: Function sample currently does not support specifying ... + """ + # Note: we don't run any of the doctests because the result can change depending on the + # system's core count. + if n is not None: + raise NotImplementedError( + "Function sample currently does not support specifying " + "exact number of items to return. Use frac instead." + ) + + if frac is None: + raise ValueError("frac must be specified.") + + sdf = self._internal.resolved_copy.spark_frame.sample( + withReplacement=replace, fraction=frac, seed=random_state + ) + return DataFrame(self._internal.with_new_sdf(sdf)) + + def astype(self, dtype) -> "DataFrame": + """ + Cast a Koalas object to a specified dtype ``dtype``. + + Parameters + ---------- + dtype : data type, or dict of column name -> data type + Use a numpy.dtype or Python type to cast entire Koalas object to + the same type. Alternatively, use {col: dtype, ...}, where col is a + column label and dtype is a numpy.dtype or Python type to cast one + or more of the DataFrame's columns to column-specific types. + + Returns + ------- + casted : same type as caller + + See Also + -------- + to_datetime : Convert argument to datetime. + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 2, 3], 'b': [1, 2, 3]}, dtype='int64') + >>> df + a b + 0 1 1 + 1 2 2 + 2 3 3 + + Convert to float type: + + >>> df.astype('float') + a b + 0 1.0 1.0 + 1 2.0 2.0 + 2 3.0 3.0 + + Convert to int64 type back: + + >>> df.astype('int64') + a b + 0 1 1 + 1 2 2 + 2 3 3 + + Convert column a to float type: + + >>> df.astype({'a': float}) + a b + 0 1.0 1 + 1 2.0 2 + 2 3.0 3 + + """ + applied = [] + if is_dict_like(dtype): + for col_name in dtype.keys(): + if col_name not in self.columns: + raise KeyError( + "Only a column name can be used for the " + "key in a dtype mappings argument." + ) + for col_name, col in self.items(): + if col_name in dtype: + applied.append(col.astype(dtype=dtype[col_name])) + else: + applied.append(col) + else: + for col_name, col in self.items(): + applied.append(col.astype(dtype=dtype)) + return DataFrame(self._internal.with_new_columns(applied)) + + def add_prefix(self, prefix) -> "DataFrame": + """ + Prefix labels with string `prefix`. + + For Series, the row labels are prefixed. + For DataFrame, the column labels are prefixed. + + Parameters + ---------- + prefix : str + The string to add before each label. + + Returns + ------- + DataFrame + New DataFrame with updated labels. + + See Also + -------- + Series.add_prefix: Prefix row labels with string `prefix`. + Series.add_suffix: Suffix row labels with string `suffix`. + DataFrame.add_suffix: Suffix column labels with string `suffix`. + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2, 3, 4], 'B': [3, 4, 5, 6]}, columns=['A', 'B']) + >>> df + A B + 0 1 3 + 1 2 4 + 2 3 5 + 3 4 6 + + >>> df.add_prefix('col_') + col_A col_B + 0 1 3 + 1 2 4 + 2 3 5 + 3 4 6 + """ + assert isinstance(prefix, str) + return self._apply_series_op( + lambda kser: kser.rename(tuple([prefix + i for i in kser._column_label])) + ) + + def add_suffix(self, suffix) -> "DataFrame": + """ + Suffix labels with string `suffix`. + + For Series, the row labels are suffixed. + For DataFrame, the column labels are suffixed. + + Parameters + ---------- + suffix : str + The string to add before each label. + + Returns + ------- + DataFrame + New DataFrame with updated labels. + + See Also + -------- + Series.add_prefix: Prefix row labels with string `prefix`. + Series.add_suffix: Suffix row labels with string `suffix`. + DataFrame.add_prefix: Prefix column labels with string `prefix`. + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2, 3, 4], 'B': [3, 4, 5, 6]}, columns=['A', 'B']) + >>> df + A B + 0 1 3 + 1 2 4 + 2 3 5 + 3 4 6 + + >>> df.add_suffix('_col') + A_col B_col + 0 1 3 + 1 2 4 + 2 3 5 + 3 4 6 + """ + assert isinstance(suffix, str) + return self._apply_series_op( + lambda kser: kser.rename(tuple([i + suffix for i in kser._column_label])) + ) + + # TODO: include, and exclude should be implemented. + def describe(self, percentiles: Optional[List[float]] = None) -> "DataFrame": + """ + Generate descriptive statistics that summarize the central tendency, + dispersion and shape of a dataset's distribution, excluding + ``NaN`` values. + + Analyzes both numeric and object series, as well + as ``DataFrame`` column sets of mixed data types. The output + will vary depending on what is provided. Refer to the notes + below for more detail. + + Parameters + ---------- + percentiles : list of ``float`` in range [0.0, 1.0], default [0.25, 0.5, 0.75] + A list of percentiles to be computed. + + Returns + ------- + DataFrame + Summary statistics of the Dataframe provided. + + See Also + -------- + DataFrame.count: Count number of non-NA/null observations. + DataFrame.max: Maximum of the values in the object. + DataFrame.min: Minimum of the values in the object. + DataFrame.mean: Mean of the values. + DataFrame.std: Standard deviation of the observations. + + Notes + ----- + For numeric data, the result's index will include ``count``, + ``mean``, ``std``, ``min``, ``25%``, ``50%``, ``75%``, ``max``. + + Currently only numeric data is supported. + + Examples + -------- + Describing a numeric ``Series``. + + >>> s = pp.Series([1, 2, 3]) + >>> s.describe() + count 3.0 + mean 2.0 + std 1.0 + min 1.0 + 25% 1.0 + 50% 2.0 + 75% 3.0 + max 3.0 + dtype: float64 + + Describing a ``DataFrame``. Only numeric fields are returned. + + >>> df = pp.DataFrame({'numeric1': [1, 2, 3], + ... 'numeric2': [4.0, 5.0, 6.0], + ... 'object': ['a', 'b', 'c'] + ... }, + ... columns=['numeric1', 'numeric2', 'object']) + >>> df.describe() + numeric1 numeric2 + count 3.0 3.0 + mean 2.0 5.0 + std 1.0 1.0 + min 1.0 4.0 + 25% 1.0 4.0 + 50% 2.0 5.0 + 75% 3.0 6.0 + max 3.0 6.0 + + For multi-index columns: + + >>> df.columns = [('num', 'a'), ('num', 'b'), ('obj', 'c')] + >>> df.describe() # doctest: +NORMALIZE_WHITESPACE + num + a b + count 3.0 3.0 + mean 2.0 5.0 + std 1.0 1.0 + min 1.0 4.0 + 25% 1.0 4.0 + 50% 2.0 5.0 + 75% 3.0 6.0 + max 3.0 6.0 + + >>> df[('num', 'b')].describe() + count 3.0 + mean 5.0 + std 1.0 + min 4.0 + 25% 4.0 + 50% 5.0 + 75% 6.0 + max 6.0 + Name: (num, b), dtype: float64 + + Describing a ``DataFrame`` and selecting custom percentiles. + + >>> df = pp.DataFrame({'numeric1': [1, 2, 3], + ... 'numeric2': [4.0, 5.0, 6.0] + ... }, + ... columns=['numeric1', 'numeric2']) + >>> df.describe(percentiles = [0.85, 0.15]) + numeric1 numeric2 + count 3.0 3.0 + mean 2.0 5.0 + std 1.0 1.0 + min 1.0 4.0 + 15% 1.0 4.0 + 50% 2.0 5.0 + 85% 3.0 6.0 + max 3.0 6.0 + + Describing a column from a ``DataFrame`` by accessing it as + an attribute. + + >>> df.numeric1.describe() + count 3.0 + mean 2.0 + std 1.0 + min 1.0 + 25% 1.0 + 50% 2.0 + 75% 3.0 + max 3.0 + Name: numeric1, dtype: float64 + + Describing a column from a ``DataFrame`` by accessing it as + an attribute and selecting custom percentiles. + + >>> df.numeric1.describe(percentiles = [0.85, 0.15]) + count 3.0 + mean 2.0 + std 1.0 + min 1.0 + 15% 1.0 + 50% 2.0 + 85% 3.0 + max 3.0 + Name: numeric1, dtype: float64 + """ + exprs = [] + column_labels = [] + for label in self._internal.column_labels: + scol = self._internal.spark_column_for(label) + spark_type = self._internal.spark_type_for(label) + if isinstance(spark_type, DoubleType) or isinstance(spark_type, FloatType): + exprs.append( + F.nanvl(scol, F.lit(None)).alias(self._internal.spark_column_name_for(label)) + ) + column_labels.append(label) + elif isinstance(spark_type, NumericType): + exprs.append(scol) + column_labels.append(label) + + if len(exprs) == 0: + raise ValueError("Cannot describe a DataFrame without columns") + + if percentiles is not None: + if any((p < 0.0) or (p > 1.0) for p in percentiles): + raise ValueError("Percentiles should all be in the interval [0, 1]") + # appending 50% if not in percentiles already + percentiles = (percentiles + [0.5]) if 0.5 not in percentiles else percentiles + else: + percentiles = [0.25, 0.5, 0.75] + + formatted_perc = ["{:.0%}".format(p) for p in sorted(percentiles)] + stats = ["count", "mean", "stddev", "min", *formatted_perc, "max"] + + sdf = self._internal.spark_frame.select(*exprs).summary(stats) + sdf = sdf.replace("stddev", "std", subset="summary") + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, "summary")], + column_labels=column_labels, + data_spark_columns=[ + scol_for(sdf, self._internal.spark_column_name_for(label)) + for label in column_labels + ], + ) + return DataFrame(internal).astype("float64") + + def drop_duplicates(self, subset=None, keep="first", inplace=False) -> Optional["DataFrame"]: + """ + Return DataFrame with duplicate rows removed, optionally only + considering certain columns. + + Parameters + ---------- + subset : column label or sequence of labels, optional + Only consider certain columns for identifying duplicates, by + default use all of the columns. + keep : {'first', 'last', False}, default 'first' + Determines which duplicates (if any) to keep. + - ``first`` : Drop duplicates except for the first occurrence. + - ``last`` : Drop duplicates except for the last occurrence. + - False : Drop all duplicates. + inplace : boolean, default False + Whether to drop duplicates in place or to return a copy. + + Returns + ------- + DataFrame + DataFrame with duplicates removed or None if ``inplace=True``. + + >>> df = pp.DataFrame( + ... {'a': [1, 2, 2, 2, 3], 'b': ['a', 'a', 'a', 'c', 'd']}, columns = ['a', 'b']) + >>> df + a b + 0 1 a + 1 2 a + 2 2 a + 3 2 c + 4 3 d + + >>> df.drop_duplicates().sort_index() + a b + 0 1 a + 1 2 a + 3 2 c + 4 3 d + + >>> df.drop_duplicates('a').sort_index() + a b + 0 1 a + 1 2 a + 4 3 d + + >>> df.drop_duplicates(['a', 'b']).sort_index() + a b + 0 1 a + 1 2 a + 3 2 c + 4 3 d + + >>> df.drop_duplicates(keep='last').sort_index() + a b + 0 1 a + 2 2 a + 3 2 c + 4 3 d + + >>> df.drop_duplicates(keep=False).sort_index() + a b + 0 1 a + 3 2 c + 4 3 d + """ + inplace = validate_bool_kwarg(inplace, "inplace") + + sdf, column = self._mark_duplicates(subset, keep) + + sdf = sdf.where(~scol_for(sdf, column)).drop(column) + internal = self._internal.with_new_sdf(sdf) + if inplace: + self._update_internal_frame(internal) + return None + else: + return DataFrame(internal) + + def reindex( + self, + labels: Optional[Any] = None, + index: Optional[Any] = None, + columns: Optional[Any] = None, + axis: Optional[Union[int, str]] = None, + copy: Optional[bool] = True, + fill_value: Optional[Any] = None, + ) -> "DataFrame": + """ + Conform DataFrame to new index with optional filling logic, placing + NA/NaN in locations having no value in the previous index. A new object + is produced unless the new index is equivalent to the current one and + ``copy=False``. + + Parameters + ---------- + labels: array-like, optional + New labels / index to conform the axis specified by ‘axis’ to. + index, columns: array-like, optional + New labels / index to conform to, should be specified using keywords. + Preferably an Index object to avoid duplicating data + axis: int or str, optional + Axis to target. Can be either the axis name (‘index’, ‘columns’) or + number (0, 1). + copy : bool, default True + Return a new object, even if the passed indexes are the same. + fill_value : scalar, default np.NaN + Value to use for missing values. Defaults to NaN, but can be any + "compatible" value. + + Returns + ------- + DataFrame with changed index. + + See Also + -------- + DataFrame.set_index : Set row labels. + DataFrame.reset_index : Remove row labels or move them to new columns. + + Examples + -------- + + ``DataFrame.reindex`` supports two calling conventions + + * ``(index=index_labels, columns=column_labels, ...)`` + * ``(labels, axis={'index', 'columns'}, ...)`` + + We *highly* recommend using keyword arguments to clarify your + intent. + + Create a dataframe with some fictional data. + + >>> index = ['Firefox', 'Chrome', 'Safari', 'IE10', 'Konqueror'] + >>> df = pp.DataFrame({ + ... 'http_status': [200, 200, 404, 404, 301], + ... 'response_time': [0.04, 0.02, 0.07, 0.08, 1.0]}, + ... index=index, + ... columns=['http_status', 'response_time']) + >>> df + http_status response_time + Firefox 200 0.04 + Chrome 200 0.02 + Safari 404 0.07 + IE10 404 0.08 + Konqueror 301 1.00 + + Create a new index and reindex the dataframe. By default + values in the new index that do not have corresponding + records in the dataframe are assigned ``NaN``. + + >>> new_index= ['Safari', 'Iceweasel', 'Comodo Dragon', 'IE10', + ... 'Chrome'] + >>> df.reindex(new_index).sort_index() + http_status response_time + Chrome 200.0 0.02 + Comodo Dragon NaN NaN + IE10 404.0 0.08 + Iceweasel NaN NaN + Safari 404.0 0.07 + + We can fill in the missing values by passing a value to + the keyword ``fill_value``. + + >>> df.reindex(new_index, fill_value=0, copy=False).sort_index() + http_status response_time + Chrome 200 0.02 + Comodo Dragon 0 0.00 + IE10 404 0.08 + Iceweasel 0 0.00 + Safari 404 0.07 + + We can also reindex the columns. + + >>> df.reindex(columns=['http_status', 'user_agent']).sort_index() + http_status user_agent + Chrome 200 NaN + Firefox 200 NaN + IE10 404 NaN + Konqueror 301 NaN + Safari 404 NaN + + Or we can use "axis-style" keyword arguments + + >>> df.reindex(['http_status', 'user_agent'], axis="columns").sort_index() + http_status user_agent + Chrome 200 NaN + Firefox 200 NaN + IE10 404 NaN + Konqueror 301 NaN + Safari 404 NaN + + To further illustrate the filling functionality in + ``reindex``, we will create a dataframe with a + monotonically increasing index (for example, a sequence + of dates). + + >>> date_index = pd.date_range('1/1/2010', periods=6, freq='D') + >>> df2 = pp.DataFrame({"prices": [100, 101, np.nan, 100, 89, 88]}, + ... index=date_index) + >>> df2.sort_index() + prices + 2010-01-01 100.0 + 2010-01-02 101.0 + 2010-01-03 NaN + 2010-01-04 100.0 + 2010-01-05 89.0 + 2010-01-06 88.0 + + Suppose we decide to expand the dataframe to cover a wider + date range. + + >>> date_index2 = pd.date_range('12/29/2009', periods=10, freq='D') + >>> df2.reindex(date_index2).sort_index() + prices + 2009-12-29 NaN + 2009-12-30 NaN + 2009-12-31 NaN + 2010-01-01 100.0 + 2010-01-02 101.0 + 2010-01-03 NaN + 2010-01-04 100.0 + 2010-01-05 89.0 + 2010-01-06 88.0 + 2010-01-07 NaN + """ + if axis is not None and (index is not None or columns is not None): + raise TypeError("Cannot specify both 'axis' and any of 'index' or 'columns'.") + + if labels is not None: + axis = validate_axis(axis) + if axis == 0: + index = labels + elif axis == 1: + columns = labels + else: + raise ValueError( + "No axis named %s for object type %s." % (axis, type(axis).__name__) + ) + + if index is not None and not is_list_like(index): + raise TypeError( + "Index must be called with a collection of some kind, " + "%s was passed" % type(index) + ) + + if columns is not None and not is_list_like(columns): + raise TypeError( + "Columns must be called with a collection of some kind, " + "%s was passed" % type(columns) + ) + + df = self + + if index is not None: + df = df._reindex_index(index, fill_value) + + if columns is not None: + df = df._reindex_columns(columns, fill_value) + + # Copy + if copy and df is self: + return df.copy() + else: + return df + + def _reindex_index(self, index, fill_value): + # When axis is index, we can mimic pandas' by a right outer join. + nlevels = self._internal.index_level + assert nlevels <= 1 or ( + isinstance(index, pp.MultiIndex) and nlevels == index.nlevels + ), "MultiIndex DataFrame can only be reindexed with a similar Koalas MultiIndex." + + index_columns = self._internal.index_spark_column_names + frame = self._internal.resolved_copy.spark_frame.drop(NATURAL_ORDER_COLUMN_NAME) + + if isinstance(index, pp.Index): + if nlevels != index.nlevels: + return DataFrame(index._internal.with_new_columns([])).reindex( + columns=self.columns, fill_value=fill_value + ) + + index_names = index._internal.index_names + scols = index._internal.index_spark_columns + labels = index._internal.spark_frame.select( + [scol.alias(index_column) for scol, index_column in zip(scols, index_columns)] + ) + else: + kser = pp.Series(list(index)) + labels = kser._internal.spark_frame.select(kser.spark.column.alias(index_columns[0])) + index_names = self._internal.index_names + + if fill_value is not None: + frame_index_columns = [ + verify_temp_column_name(frame, "__frame_index_column_{}__".format(i)) + for i in range(nlevels) + ] + index_scols = [ + scol_for(frame, index_col).alias(frame_index_col) + for index_col, frame_index_col in zip(index_columns, frame_index_columns) + ] + scols = self._internal.resolved_copy.data_spark_columns + frame = frame.select(index_scols + scols) + + temp_fill_value = verify_temp_column_name(frame, "__fill_value__") + labels = labels.withColumn(temp_fill_value, F.lit(fill_value)) + + frame_index_scols = [scol_for(frame, col) for col in frame_index_columns] + labels_index_scols = [scol_for(labels, col) for col in index_columns] + + joined_df = frame.join( + labels, + on=[fcol == lcol for fcol, lcol in zip(frame_index_scols, labels_index_scols)], + how="right", + ) + + joined_df = joined_df.select( + *labels_index_scols, + *[ + F.when( + reduce( + lambda c1, c2: c1 & c2, + [ + fcol.isNull() & lcol.isNotNull() + for fcol, lcol in zip(frame_index_scols, labels_index_scols) + ], + ), + scol_for(joined_df, temp_fill_value), + ) + .otherwise(scol_for(joined_df, col)) + .alias(col) + for col in self._internal.data_spark_column_names + ], + ) + else: + joined_df = frame.join(labels, on=index_columns, how="right") + + sdf = joined_df.drop(NATURAL_ORDER_COLUMN_NAME) + internal = self._internal.copy( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=index_names, + index_dtypes=None, # TODO: dtypes? + data_spark_columns=[ + scol_for(sdf, col) for col in self._internal.data_spark_column_names + ], + ) + return DataFrame(internal) + + def _reindex_columns(self, columns, fill_value): + level = self._internal.column_labels_level + if level > 1: + label_columns = list(columns) + for col in label_columns: + if not isinstance(col, tuple): + raise TypeError("Expected tuple, got {}".format(type(col).__name__)) + else: + label_columns = [(col,) for col in columns] + for col in label_columns: + if len(col) != level: + raise ValueError( + "shape (1,{}) doesn't match the shape (1,{})".format(len(col), level) + ) + fill_value = np.nan if fill_value is None else fill_value + scols_or_ksers, labels = [], [] + for label in label_columns: + if label in self._internal.column_labels: + scols_or_ksers.append(self._kser_for(label)) + else: + scols_or_ksers.append(F.lit(fill_value).alias(name_like_string(label))) + labels.append(label) + + if isinstance(columns, pd.Index): + column_label_names = [ + name if is_name_like_tuple(name) else (name,) for name in columns.names + ] + internal = self._internal.with_new_columns( + scols_or_ksers, column_labels=labels, column_label_names=column_label_names + ) + else: + internal = self._internal.with_new_columns(scols_or_ksers, column_labels=labels) + + return DataFrame(internal) + + def reindex_like(self, other: "DataFrame", copy: bool = True) -> "DataFrame": + """ + Return a DataFrame with matching indices as other object. + + Conform the object to the same index on all axes. Places NA/NaN in locations + having no value in the previous index. A new object is produced unless the + new index is equivalent to the current one and copy=False. + + Parameters + ---------- + other : DataFrame + Its row and column indices are used to define the new indices + of this object. + copy : bool, default True + Return a new object, even if the passed indexes are the same. + + Returns + ------- + DataFrame + DataFrame with changed indices on each axis. + + See Also + -------- + DataFrame.set_index : Set row labels. + DataFrame.reset_index : Remove row labels or move them to new columns. + DataFrame.reindex : Change to new indices or expand indices. + + Notes + ----- + Same as calling + ``.reindex(index=other.index, columns=other.columns,...)``. + + Examples + -------- + + >>> df1 = pp.DataFrame([[24.3, 75.7, 'high'], + ... [31, 87.8, 'high'], + ... [22, 71.6, 'medium'], + ... [35, 95, 'medium']], + ... columns=['temp_celsius', 'temp_fahrenheit', + ... 'windspeed'], + ... index=pd.date_range(start='2014-02-12', + ... end='2014-02-15', freq='D')) + >>> df1 + temp_celsius temp_fahrenheit windspeed + 2014-02-12 24.3 75.7 high + 2014-02-13 31.0 87.8 high + 2014-02-14 22.0 71.6 medium + 2014-02-15 35.0 95.0 medium + + >>> df2 = pp.DataFrame([[28, 'low'], + ... [30, 'low'], + ... [35.1, 'medium']], + ... columns=['temp_celsius', 'windspeed'], + ... index=pd.DatetimeIndex(['2014-02-12', '2014-02-13', + ... '2014-02-15'])) + >>> df2 + temp_celsius windspeed + 2014-02-12 28.0 low + 2014-02-13 30.0 low + 2014-02-15 35.1 medium + + >>> df2.reindex_like(df1).sort_index() # doctest: +NORMALIZE_WHITESPACE + temp_celsius temp_fahrenheit windspeed + 2014-02-12 28.0 NaN low + 2014-02-13 30.0 NaN low + 2014-02-14 NaN NaN None + 2014-02-15 35.1 NaN medium + """ + + if isinstance(other, DataFrame): + return self.reindex(index=other.index, columns=other.columns, copy=copy) + else: + raise TypeError("other must be a Koalas DataFrame") + + def melt(self, id_vars=None, value_vars=None, var_name=None, value_name="value") -> "DataFrame": + """ + Unpivot a DataFrame from wide format to long format, optionally + leaving identifier variables set. + + This function is useful to massage a DataFrame into a format where one + or more columns are identifier variables (`id_vars`), while all other + columns, considered measured variables (`value_vars`), are "unpivoted" to + the row axis, leaving just two non-identifier columns, 'variable' and + 'value'. + + Parameters + ---------- + frame : DataFrame + id_vars : tuple, list, or ndarray, optional + Column(s) to use as identifier variables. + value_vars : tuple, list, or ndarray, optional + Column(s) to unpivot. If not specified, uses all columns that + are not set as `id_vars`. + var_name : scalar, default 'variable' + Name to use for the 'variable' column. If None it uses `frame.columns.name` or + ‘variable’. + value_name : scalar, default 'value' + Name to use for the 'value' column. + + Returns + ------- + DataFrame + Unpivoted DataFrame. + + Examples + -------- + >>> df = pp.DataFrame({'A': {0: 'a', 1: 'b', 2: 'c'}, + ... 'B': {0: 1, 1: 3, 2: 5}, + ... 'C': {0: 2, 1: 4, 2: 6}}, + ... columns=['A', 'B', 'C']) + >>> df + A B C + 0 a 1 2 + 1 b 3 4 + 2 c 5 6 + + >>> pp.melt(df) + variable value + 0 A a + 1 B 1 + 2 C 2 + 3 A b + 4 B 3 + 5 C 4 + 6 A c + 7 B 5 + 8 C 6 + + >>> df.melt(id_vars='A') + A variable value + 0 a B 1 + 1 a C 2 + 2 b B 3 + 3 b C 4 + 4 c B 5 + 5 c C 6 + + >>> df.melt(value_vars='A') + variable value + 0 A a + 1 A b + 2 A c + + >>> pp.melt(df, id_vars=['A', 'B']) + A B variable value + 0 a 1 C 2 + 1 b 3 C 4 + 2 c 5 C 6 + + >>> df.melt(id_vars=['A'], value_vars=['C']) + A variable value + 0 a C 2 + 1 b C 4 + 2 c C 6 + + The names of 'variable' and 'value' columns can be customized: + + >>> pp.melt(df, id_vars=['A'], value_vars=['B'], + ... var_name='myVarname', value_name='myValname') + A myVarname myValname + 0 a B 1 + 1 b B 3 + 2 c B 5 + """ + column_labels = self._internal.column_labels + + if id_vars is None: + id_vars = [] + else: + if isinstance(id_vars, tuple): + if self._internal.column_labels_level == 1: + id_vars = [idv if is_name_like_tuple(idv) else (idv,) for idv in id_vars] + else: + raise ValueError( + "id_vars must be a list of tuples" " when columns are a MultiIndex" + ) + elif is_name_like_value(id_vars): + id_vars = [(id_vars,)] + else: + id_vars = [idv if is_name_like_tuple(idv) else (idv,) for idv in id_vars] + + non_existence_col = [idv for idv in id_vars if idv not in column_labels] + if len(non_existence_col) != 0: + raveled_column_labels = np.ravel(column_labels) + missing = [ + nec for nec in np.ravel(non_existence_col) if nec not in raveled_column_labels + ] + if len(missing) != 0: + raise KeyError( + "The following 'id_vars' are not present" + " in the DataFrame: {}".format(missing) + ) + else: + raise KeyError( + "None of {} are in the {}".format(non_existence_col, column_labels) + ) + + if value_vars is None: + value_vars = [] + else: + if isinstance(value_vars, tuple): + if self._internal.column_labels_level == 1: + value_vars = [ + valv if is_name_like_tuple(valv) else (valv,) for valv in value_vars + ] + else: + raise ValueError( + "value_vars must be a list of tuples" " when columns are a MultiIndex" + ) + elif is_name_like_value(value_vars): + value_vars = [(value_vars,)] + else: + value_vars = [valv if is_name_like_tuple(valv) else (valv,) for valv in value_vars] + + non_existence_col = [valv for valv in value_vars if valv not in column_labels] + if len(non_existence_col) != 0: + raveled_column_labels = np.ravel(column_labels) + missing = [ + nec for nec in np.ravel(non_existence_col) if nec not in raveled_column_labels + ] + if len(missing) != 0: + raise KeyError( + "The following 'value_vars' are not present" + " in the DataFrame: {}".format(missing) + ) + else: + raise KeyError( + "None of {} are in the {}".format(non_existence_col, column_labels) + ) + + if len(value_vars) == 0: + value_vars = column_labels + + column_labels = [label for label in column_labels if label not in id_vars] + + sdf = self._internal.spark_frame + + if var_name is None: + if ( + self._internal.column_labels_level == 1 + and self._internal.column_label_names[0] is None + ): + var_name = ["variable"] + else: + var_name = [ + name_like_string(name) if name is not None else "variable_{}".format(i) + for i, name in enumerate(self._internal.column_label_names) + ] + elif isinstance(var_name, str): + var_name = [var_name] + + pairs = F.explode( + F.array( + *[ + F.struct( + *( + [F.lit(c).alias(name) for c, name in zip(label, var_name)] + + [self._internal.spark_column_for(label).alias(value_name)] + ) + ) + for label in column_labels + if label in value_vars + ] + ) + ) + + columns = ( + [ + self._internal.spark_column_for(label).alias(name_like_string(label)) + for label in id_vars + ] + + [F.col("pairs.`%s`" % name) for name in var_name] + + [F.col("pairs.`%s`" % value_name)] + ) + exploded_df = sdf.withColumn("pairs", pairs).select(columns) + + return DataFrame( + InternalFrame( + spark_frame=exploded_df, + index_spark_columns=None, + column_labels=( + [label if len(label) == 1 else (name_like_string(label),) for label in id_vars] + + [(name,) for name in var_name] + + [(value_name,)] + ), + ) + ) + + def stack(self) -> Union["DataFrame", "Series"]: + """ + Stack the prescribed level(s) from columns to index. + + Return a reshaped DataFrame or Series having a multi-level + index with one or more new inner-most levels compared to the current + DataFrame. The new inner-most levels are created by pivoting the + columns of the current dataframe: + + - if the columns have a single level, the output is a Series; + - if the columns have multiple levels, the new index + level(s) is (are) taken from the prescribed level(s) and + the output is a DataFrame. + + The new index levels are sorted. + + Returns + ------- + DataFrame or Series + Stacked dataframe or series. + + See Also + -------- + DataFrame.unstack : Unstack prescribed level(s) from index axis + onto column axis. + DataFrame.pivot : Reshape dataframe from long format to wide + format. + DataFrame.pivot_table : Create a spreadsheet-style pivot table + as a DataFrame. + + Notes + ----- + The function is named by analogy with a collection of books + being reorganized from being side by side on a horizontal + position (the columns of the dataframe) to being stacked + vertically on top of each other (in the index of the + dataframe). + + Examples + -------- + **Single level columns** + + >>> df_single_level_cols = pp.DataFrame([[0, 1], [2, 3]], + ... index=['cat', 'dog'], + ... columns=['weight', 'height']) + + Stacking a dataframe with a single level column axis returns a Series: + + >>> df_single_level_cols + weight height + cat 0 1 + dog 2 3 + >>> df_single_level_cols.stack().sort_index() + cat height 1 + weight 0 + dog height 3 + weight 2 + dtype: int64 + + **Multi level columns: simple case** + + >>> multicol1 = pd.MultiIndex.from_tuples([('weight', 'kg'), + ... ('weight', 'pounds')]) + >>> df_multi_level_cols1 = pp.DataFrame([[1, 2], [2, 4]], + ... index=['cat', 'dog'], + ... columns=multicol1) + + Stacking a dataframe with a multi-level column axis: + + >>> df_multi_level_cols1 # doctest: +NORMALIZE_WHITESPACE + weight + kg pounds + cat 1 2 + dog 2 4 + >>> df_multi_level_cols1.stack().sort_index() + weight + cat kg 1 + pounds 2 + dog kg 2 + pounds 4 + + **Missing values** + + >>> multicol2 = pd.MultiIndex.from_tuples([('weight', 'kg'), + ... ('height', 'm')]) + >>> df_multi_level_cols2 = pp.DataFrame([[1.0, 2.0], [3.0, 4.0]], + ... index=['cat', 'dog'], + ... columns=multicol2) + + It is common to have missing values when stacking a dataframe + with multi-level columns, as the stacked dataframe typically + has more values than the original dataframe. Missing values + are filled with NaNs: + + >>> df_multi_level_cols2 + weight height + kg m + cat 1.0 2.0 + dog 3.0 4.0 + >>> df_multi_level_cols2.stack().sort_index() # doctest: +SKIP + height weight + cat kg NaN 1.0 + m 2.0 NaN + dog kg NaN 3.0 + m 4.0 NaN + """ + from pyspark.pandas.series import first_series + + if len(self._internal.column_labels) == 0: + return DataFrame( + self._internal.copy( + column_label_names=self._internal.column_label_names[:-1] + ).with_filter(F.lit(False)) + ) + + column_labels = defaultdict(dict) # type: Union[defaultdict, OrderedDict] + index_values = set() + should_returns_series = False + for label in self._internal.column_labels: + new_label = label[:-1] + if len(new_label) == 0: + new_label = None + should_returns_series = True + value = label[-1] + + scol = self._internal.spark_column_for(label) + column_labels[new_label][value] = scol + + index_values.add(value) + + column_labels = OrderedDict(sorted(column_labels.items(), key=lambda x: x[0])) + + index_name = self._internal.column_label_names[-1] + column_label_names = self._internal.column_label_names[:-1] + if len(column_label_names) == 0: + column_label_names = [None] + + index_column = SPARK_INDEX_NAME_FORMAT(self._internal.index_level) + data_columns = [name_like_string(label) for label in column_labels] + + structs = [ + F.struct( + [F.lit(value).alias(index_column)] + + [ + ( + column_labels[label][value] + if value in column_labels[label] + else F.lit(None) + ).alias(name) + for label, name in zip(column_labels, data_columns) + ] + ).alias(value) + for value in index_values + ] + + pairs = F.explode(F.array(structs)) + + sdf = self._internal.spark_frame.withColumn("pairs", pairs) + sdf = sdf.select( + self._internal.index_spark_columns + + [sdf["pairs"][index_column].alias(index_column)] + + [sdf["pairs"][name].alias(name) for name in data_columns] + ) + + internal = InternalFrame( # TODO: dtypes? + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) + for col in (self._internal.index_spark_column_names + [index_column]) + ], + index_names=self._internal.index_names + [index_name], + index_dtypes=self._internal.index_dtypes + [None], + column_labels=list(column_labels), + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + column_label_names=column_label_names, # type: ignore + ) + kdf = DataFrame(internal) # type: "DataFrame" + + if should_returns_series: + return first_series(kdf) + else: + return kdf + + def unstack(self) -> Union["DataFrame", "Series"]: + """ + Pivot the (necessarily hierarchical) index labels. + + Returns a DataFrame having a new level of column labels whose inner-most level + consists of the pivoted index labels. + + If the index is not a MultiIndex, the output will be a Series. + + .. note:: If the index is a MultiIndex, the output DataFrame could be very wide, and + it could cause a serious performance degradation since Spark partitions it row based. + + Returns + ------- + Series or DataFrame + + See Also + -------- + DataFrame.pivot : Pivot a table based on column values. + DataFrame.stack : Pivot a level of the column labels (inverse operation from unstack). + + Examples + -------- + >>> df = pp.DataFrame({"A": {"0": "a", "1": "b", "2": "c"}, + ... "B": {"0": "1", "1": "3", "2": "5"}, + ... "C": {"0": "2", "1": "4", "2": "6"}}, + ... columns=["A", "B", "C"]) + >>> df + A B C + 0 a 1 2 + 1 b 3 4 + 2 c 5 6 + + >>> df.unstack().sort_index() + A 0 a + 1 b + 2 c + B 0 1 + 1 3 + 2 5 + C 0 2 + 1 4 + 2 6 + dtype: object + + >>> df.columns = pd.MultiIndex.from_tuples([('X', 'A'), ('X', 'B'), ('Y', 'C')]) + >>> df.unstack().sort_index() + X A 0 a + 1 b + 2 c + B 0 1 + 1 3 + 2 5 + Y C 0 2 + 1 4 + 2 6 + dtype: object + + For MultiIndex case: + + >>> df = pp.DataFrame({"A": ["a", "b", "c"], + ... "B": [1, 3, 5], + ... "C": [2, 4, 6]}, + ... columns=["A", "B", "C"]) + >>> df = df.set_index('A', append=True) + >>> df # doctest: +NORMALIZE_WHITESPACE + B C + A + 0 a 1 2 + 1 b 3 4 + 2 c 5 6 + >>> df.unstack().sort_index() # doctest: +NORMALIZE_WHITESPACE + B C + A a b c a b c + 0 1.0 NaN NaN 2.0 NaN NaN + 1 NaN 3.0 NaN NaN 4.0 NaN + 2 NaN NaN 5.0 NaN NaN 6.0 + """ + from pyspark.pandas.series import first_series + + if self._internal.index_level > 1: + # The index after `reset_index()` will never be used, so use "distributed" index + # as a dummy to avoid overhead. + with option_context("compute.default_index_type", "distributed"): + df = self.reset_index() + index = df._internal.column_labels[: self._internal.index_level - 1] + columns = df.columns[self._internal.index_level - 1] + df = df.pivot_table( + index=index, columns=columns, values=self._internal.column_labels, aggfunc="first" + ) + internal = df._internal.copy( + index_names=self._internal.index_names[:-1], + index_dtypes=self._internal.index_dtypes[:-1], + column_label_names=( + df._internal.column_label_names[:-1] + + [ + None + if self._internal.index_names[-1] is None + else df._internal.column_label_names[-1] + ] + ), + ) + return DataFrame(internal) + + # TODO: Codes here are similar with melt. Should we deduplicate? + column_labels = self._internal.column_labels + ser_name = SPARK_DEFAULT_SERIES_NAME + sdf = self._internal.spark_frame + new_index_columns = [ + SPARK_INDEX_NAME_FORMAT(i) for i in range(self._internal.column_labels_level) + ] + + new_index_map = list(zip(new_index_columns, self._internal.column_label_names)) + + pairs = F.explode( + F.array( + *[ + F.struct( + *( + [F.lit(c).alias(name) for c, name in zip(idx, new_index_columns)] + + [self._internal.spark_column_for(idx).alias(ser_name)] + ) + ) + for idx in column_labels + ] + ) + ) + + columns = [ + F.col("pairs.%s" % name) + for name in new_index_columns[: self._internal.column_labels_level] + ] + [F.col("pairs.%s" % ser_name)] + + new_index_len = len(new_index_columns) + existing_index_columns = [] + for i, index_name in enumerate(self._internal.index_names): + new_index_map.append((SPARK_INDEX_NAME_FORMAT(i + new_index_len), index_name)) + existing_index_columns.append( + self._internal.index_spark_columns[i].alias( + SPARK_INDEX_NAME_FORMAT(i + new_index_len) + ) + ) + + exploded_df = sdf.withColumn("pairs", pairs).select(existing_index_columns + columns) + + index_spark_column_names, index_names = zip(*new_index_map) + return first_series( + DataFrame( + InternalFrame( # TODO: dtypes? + exploded_df, + index_spark_columns=[ + scol_for(exploded_df, col) for col in index_spark_column_names + ], + index_names=list(index_names), + column_labels=[None], + ) + ) + ) + + # TODO: axis, skipna, and many arguments should be implemented. + def all(self, axis: Union[int, str] = 0) -> "Series": + """ + Return whether all elements are True. + + Returns True unless there is at least one element within a series that is + False or equivalent (e.g. zero or empty) + + Parameters + ---------- + axis : {0 or 'index'}, default 0 + Indicate which axis or axes should be reduced. + + * 0 / 'index' : reduce the index, return a Series whose index is the + original column labels. + + Returns + ------- + Series + + Examples + -------- + Create a dataframe from a dictionary. + + >>> df = pp.DataFrame({ + ... 'col1': [True, True, True], + ... 'col2': [True, False, False], + ... 'col3': [0, 0, 0], + ... 'col4': [1, 2, 3], + ... 'col5': [True, True, None], + ... 'col6': [True, False, None]}, + ... columns=['col1', 'col2', 'col3', 'col4', 'col5', 'col6']) + + Default behaviour checks if column-wise values all return a boolean. + + >>> df.all() + col1 True + col2 False + col3 False + col4 True + col5 True + col6 False + dtype: bool + """ + from pyspark.pandas.series import first_series + + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + applied = [] + column_labels = self._internal.column_labels + for label in column_labels: + scol = self._internal.spark_column_for(label) + all_col = F.min(F.coalesce(scol.cast("boolean"), F.lit(True))) + applied.append(F.when(all_col.isNull(), True).otherwise(all_col)) + + # TODO: there is a similar logic to transpose in, for instance, + # DataFrame.any, Series.quantile. Maybe we should deduplicate it. + value_column = "value" + cols = [] + for label, applied_col in zip(column_labels, applied): + cols.append( + F.struct( + [F.lit(col).alias(SPARK_INDEX_NAME_FORMAT(i)) for i, col in enumerate(label)] + + [applied_col.alias(value_column)] + ) + ) + + sdf = self._internal.spark_frame.select(F.array(*cols).alias("arrays")).select( + F.explode(F.col("arrays")) + ) + sdf = sdf.selectExpr("col.*") + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, SPARK_INDEX_NAME_FORMAT(i)) + for i in range(self._internal.column_labels_level) + ], + index_names=self._internal.column_label_names, + column_labels=[None], + data_spark_columns=[scol_for(sdf, value_column)], + ) + + return first_series(DataFrame(internal)) + + # TODO: axis, skipna, and many arguments should be implemented. + def any(self, axis: Union[int, str] = 0) -> "Series": + """ + Return whether any element is True. + + Returns False unless there is at least one element within a series that is + True or equivalent (e.g. non-zero or non-empty). + + Parameters + ---------- + axis : {0 or 'index'}, default 0 + Indicate which axis or axes should be reduced. + + * 0 / 'index' : reduce the index, return a Series whose index is the + original column labels. + + Returns + ------- + Series + + Examples + -------- + Create a dataframe from a dictionary. + + >>> df = pp.DataFrame({ + ... 'col1': [False, False, False], + ... 'col2': [True, False, False], + ... 'col3': [0, 0, 1], + ... 'col4': [0, 1, 2], + ... 'col5': [False, False, None], + ... 'col6': [True, False, None]}, + ... columns=['col1', 'col2', 'col3', 'col4', 'col5', 'col6']) + + Default behaviour checks if column-wise values all return a boolean. + + >>> df.any() + col1 False + col2 True + col3 True + col4 True + col5 False + col6 True + dtype: bool + """ + from pyspark.pandas.series import first_series + + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + applied = [] + column_labels = self._internal.column_labels + for label in column_labels: + scol = self._internal.spark_column_for(label) + all_col = F.max(F.coalesce(scol.cast("boolean"), F.lit(False))) + applied.append(F.when(all_col.isNull(), False).otherwise(all_col)) + + # TODO: there is a similar logic to transpose in, for instance, + # DataFrame.all, Series.quantile. Maybe we should deduplicate it. + value_column = "value" + cols = [] + for label, applied_col in zip(column_labels, applied): + cols.append( + F.struct( + [F.lit(col).alias(SPARK_INDEX_NAME_FORMAT(i)) for i, col in enumerate(label)] + + [applied_col.alias(value_column)] + ) + ) + + sdf = self._internal.spark_frame.select(F.array(*cols).alias("arrays")).select( + F.explode(F.col("arrays")) + ) + sdf = sdf.selectExpr("col.*") + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, SPARK_INDEX_NAME_FORMAT(i)) + for i in range(self._internal.column_labels_level) + ], + index_names=self._internal.column_label_names, + column_labels=[None], + data_spark_columns=[scol_for(sdf, value_column)], + ) + + return first_series(DataFrame(internal)) + + # TODO: add axis, numeric_only, pct, na_option parameter + def rank(self, method="average", ascending=True) -> "DataFrame": + """ + Compute numerical data ranks (1 through n) along axis. Equal values are + assigned a rank that is the average of the ranks of those values. + + .. note:: the current implementation of rank uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + method : {'average', 'min', 'max', 'first', 'dense'} + * average: average rank of group + * min: lowest rank in group + * max: highest rank in group + * first: ranks assigned in order they appear in the array + * dense: like 'min', but rank always increases by 1 between groups + ascending : boolean, default True + False for ranks by high (1) to low (N) + + Returns + ------- + ranks : same type as caller + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2, 2, 3], 'B': [4, 3, 2, 1]}, columns= ['A', 'B']) + >>> df + A B + 0 1 4 + 1 2 3 + 2 2 2 + 3 3 1 + + >>> df.rank().sort_index() + A B + 0 1.0 4.0 + 1 2.5 3.0 + 2 2.5 2.0 + 3 4.0 1.0 + + If method is set to 'min', it use lowest rank in group. + + >>> df.rank(method='min').sort_index() + A B + 0 1.0 4.0 + 1 2.0 3.0 + 2 2.0 2.0 + 3 4.0 1.0 + + If method is set to 'max', it use highest rank in group. + + >>> df.rank(method='max').sort_index() + A B + 0 1.0 4.0 + 1 3.0 3.0 + 2 3.0 2.0 + 3 4.0 1.0 + + If method is set to 'dense', it leaves no gaps in group. + + >>> df.rank(method='dense').sort_index() + A B + 0 1.0 4.0 + 1 2.0 3.0 + 2 2.0 2.0 + 3 3.0 1.0 + """ + return self._apply_series_op( + lambda kser: kser._rank(method=method, ascending=ascending), should_resolve=True + ) + + def filter(self, items=None, like=None, regex=None, axis=None) -> "DataFrame": + """ + Subset rows or columns of dataframe according to labels in + the specified index. + + Note that this routine does not filter a dataframe on its + contents. The filter is applied to the labels of the index. + + Parameters + ---------- + items : list-like + Keep labels from axis which are in items. + like : string + Keep labels from axis for which "like in label == True". + regex : string (regular expression) + Keep labels from axis for which re.search(regex, label) == True. + axis : int or string axis name + The axis to filter on. By default this is the info axis, + 'index' for Series, 'columns' for DataFrame. + + Returns + ------- + same type as input object + + See Also + -------- + DataFrame.loc + + Notes + ----- + The ``items``, ``like``, and ``regex`` parameters are + enforced to be mutually exclusive. + + ``axis`` defaults to the info axis that is used when indexing + with ``[]``. + + Examples + -------- + >>> df = pp.DataFrame(np.array(([1, 2, 3], [4, 5, 6])), + ... index=['mouse', 'rabbit'], + ... columns=['one', 'two', 'three']) + + >>> # select columns by name + >>> df.filter(items=['one', 'three']) + one three + mouse 1 3 + rabbit 4 6 + + >>> # select columns by regular expression + >>> df.filter(regex='e$', axis=1) + one three + mouse 1 3 + rabbit 4 6 + + >>> # select rows containing 'bbi' + >>> df.filter(like='bbi', axis=0) + one two three + rabbit 4 5 6 + + For a Series, + + >>> # select rows by name + >>> df.one.filter(items=['rabbit']) + rabbit 4 + Name: one, dtype: int64 + + >>> # select rows by regular expression + >>> df.one.filter(regex='e$') + mouse 1 + Name: one, dtype: int64 + + >>> # select rows containing 'bbi' + >>> df.one.filter(like='bbi') + rabbit 4 + Name: one, dtype: int64 + """ + if sum(x is not None for x in (items, like, regex)) > 1: + raise TypeError( + "Keyword arguments `items`, `like`, or `regex` " "are mutually exclusive" + ) + + axis = validate_axis(axis, none_axis=1) + + index_scols = self._internal.index_spark_columns + + if items is not None: + if is_list_like(items): + items = list(items) + else: + raise ValueError("items should be a list-like object.") + if axis == 0: + if len(index_scols) == 1: + col = None + for item in items: + if col is None: + col = index_scols[0] == F.lit(item) + else: + col = col | (index_scols[0] == F.lit(item)) + elif len(index_scols) > 1: + # for multi-index + col = None + for item in items: + if not isinstance(item, tuple): + raise TypeError("Unsupported type {}".format(type(item).__name__)) + if not item: + raise ValueError("The item should not be empty.") + midx_col = None + for i, element in enumerate(item): + if midx_col is None: + midx_col = index_scols[i] == F.lit(element) + else: + midx_col = midx_col & (index_scols[i] == F.lit(element)) + if col is None: + col = midx_col + else: + col = col | midx_col + else: + raise ValueError("Single or multi index must be specified.") + return DataFrame(self._internal.with_filter(col)) + else: + return self[items] + elif like is not None: + if axis == 0: + col = None + for index_scol in index_scols: + if col is None: + col = index_scol.contains(like) + else: + col = col | index_scol.contains(like) + return DataFrame(self._internal.with_filter(col)) + else: + column_labels = self._internal.column_labels + output_labels = [label for label in column_labels if any(like in i for i in label)] + return self[output_labels] + elif regex is not None: + if axis == 0: + col = None + for index_scol in index_scols: + if col is None: + col = index_scol.rlike(regex) + else: + col = col | index_scol.rlike(regex) + return DataFrame(self._internal.with_filter(col)) + else: + column_labels = self._internal.column_labels + matcher = re.compile(regex) + output_labels = [ + label + for label in column_labels + if any(matcher.search(i) is not None for i in label) + ] + return self[output_labels] + else: + raise TypeError("Must pass either `items`, `like`, or `regex`") + + def rename( + self, + mapper=None, + index=None, + columns=None, + axis="index", + inplace=False, + level=None, + errors="ignore", + ) -> Optional["DataFrame"]: + + """ + Alter axes labels. + Function / dict values must be unique (1-to-1). Labels not contained in a dict / Series + will be left as-is. Extra labels listed don’t throw an error. + + Parameters + ---------- + mapper : dict-like or function + Dict-like or functions transformations to apply to that axis’ values. + Use either `mapper` and `axis` to specify the axis to target with `mapper`, or `index` + and `columns`. + index : dict-like or function + Alternative to specifying axis ("mapper, axis=0" is equivalent to "index=mapper"). + columns : dict-like or function + Alternative to specifying axis ("mapper, axis=1" is equivalent to "columns=mapper"). + axis : int or str, default 'index' + Axis to target with mapper. Can be either the axis name ('index', 'columns') or + number (0, 1). + inplace : bool, default False + Whether to return a new DataFrame. + level : int or level name, default None + In case of a MultiIndex, only rename labels in the specified level. + errors : {'ignore', 'raise}, default 'ignore' + If 'raise', raise a `KeyError` when a dict-like `mapper`, `index`, or `columns` + contains labels that are not present in the Index being transformed. If 'ignore', + existing keys will be renamed and extra keys will be ignored. + + Returns + ------- + DataFrame with the renamed axis labels. + + Raises + ------ + `KeyError` + If any of the labels is not found in the selected axis and "errors='raise'". + + Examples + -------- + >>> kdf1 = pp.DataFrame({"A": [1, 2, 3], "B": [4, 5, 6]}) + >>> kdf1.rename(columns={"A": "a", "B": "c"}) # doctest: +NORMALIZE_WHITESPACE + a c + 0 1 4 + 1 2 5 + 2 3 6 + + >>> kdf1.rename(index={1: 10, 2: 20}) # doctest: +NORMALIZE_WHITESPACE + A B + 0 1 4 + 10 2 5 + 20 3 6 + + >>> def str_lower(s) -> str: + ... return str.lower(s) + >>> kdf1.rename(str_lower, axis='columns') # doctest: +NORMALIZE_WHITESPACE + a b + 0 1 4 + 1 2 5 + 2 3 6 + + >>> def mul10(x) -> int: + ... return x * 10 + >>> kdf1.rename(mul10, axis='index') # doctest: +NORMALIZE_WHITESPACE + A B + 0 1 4 + 10 2 5 + 20 3 6 + + >>> idx = pd.MultiIndex.from_tuples([('X', 'A'), ('X', 'B'), ('Y', 'C'), ('Y', 'D')]) + >>> kdf2 = pp.DataFrame([[1, 2, 3, 4], [5, 6, 7, 8]], columns=idx) + >>> kdf2.rename(columns=str_lower, level=0) # doctest: +NORMALIZE_WHITESPACE + x y + A B C D + 0 1 2 3 4 + 1 5 6 7 8 + + >>> kdf3 = pp.DataFrame([[1, 2], [3, 4], [5, 6], [7, 8]], index=idx, columns=list('ab')) + >>> kdf3.rename(index=str_lower) # doctest: +NORMALIZE_WHITESPACE + a b + x a 1 2 + b 3 4 + y c 5 6 + d 7 8 + """ + + def gen_mapper_fn(mapper): + if isinstance(mapper, dict): + if len(mapper) == 0: + if errors == "raise": + raise KeyError("Index include label which is not in the `mapper`.") + else: + return DataFrame(self._internal) + + type_set = set(map(lambda x: type(x), mapper.values())) + if len(type_set) > 1: + raise ValueError("Mapper dict should have the same value type.") + spark_return_type = as_spark_type(list(type_set)[0]) + + def mapper_fn(x): + if x in mapper: + return mapper[x] + else: + if errors == "raise": + raise KeyError("Index include value which is not in the `mapper`") + return x + + elif callable(mapper): + spark_return_type = cast(ScalarType, infer_return_type(mapper)).spark_type + + def mapper_fn(x): + return mapper(x) + + else: + raise ValueError( + "`mapper` or `index` or `columns` should be " + "either dict-like or function type." + ) + return mapper_fn, spark_return_type + + index_mapper_fn = None + index_mapper_ret_stype = None + columns_mapper_fn = None + + inplace = validate_bool_kwarg(inplace, "inplace") + if mapper: + axis = validate_axis(axis) + if axis == 0: + index_mapper_fn, index_mapper_ret_stype = gen_mapper_fn(mapper) + elif axis == 1: + columns_mapper_fn, columns_mapper_ret_stype = gen_mapper_fn(mapper) + else: + raise ValueError( + "argument axis should be either the axis name " + "(‘index’, ‘columns’) or number (0, 1)" + ) + else: + if index: + index_mapper_fn, index_mapper_ret_stype = gen_mapper_fn(index) + if columns: + columns_mapper_fn, _ = gen_mapper_fn(columns) + + if not index and not columns: + raise ValueError("Either `index` or `columns` should be provided.") + + kdf = self.copy() + if index_mapper_fn: + # rename index labels, if `level` is None, rename all index columns, otherwise only + # rename the corresponding level index. + # implement this by transform the underlying spark dataframe, + # Example: + # suppose the kdf index column in underlying spark dataframe is "index_0", "index_1", + # if rename level 0 index labels, will do: + # ``kdf._sdf.withColumn("index_0", mapper_fn_udf(col("index_0"))`` + # if rename all index labels (`level` is None), then will do: + # ``` + # kdf._sdf.withColumn("index_0", mapper_fn_udf(col("index_0")) + # .withColumn("index_1", mapper_fn_udf(col("index_1")) + # ``` + + index_columns = kdf._internal.index_spark_column_names + num_indices = len(index_columns) + if level: + if level < 0 or level >= num_indices: + raise ValueError("level should be an integer between [0, num_indices)") + + def gen_new_index_column(level): + index_col_name = index_columns[level] + + index_mapper_udf = pandas_udf( + lambda s: s.map(index_mapper_fn), returnType=index_mapper_ret_stype + ) + return index_mapper_udf(scol_for(kdf._internal.spark_frame, index_col_name)) + + sdf = kdf._internal.resolved_copy.spark_frame + index_dtypes = self._internal.index_dtypes.copy() + if level is None: + for i in range(num_indices): + sdf = sdf.withColumn(index_columns[i], gen_new_index_column(i)) + index_dtypes[i] = None # TODO: dtype? + else: + sdf = sdf.withColumn(index_columns[level], gen_new_index_column(level)) + index_dtypes[level] = None # TODO: dtype? + kdf = DataFrame(kdf._internal.with_new_sdf(sdf, index_dtypes=index_dtypes)) + if columns_mapper_fn: + # rename column name. + # Will modify the `_internal._column_labels` and transform underlying spark dataframe + # to the same column name with `_internal._column_labels`. + if level: + if level < 0 or level >= kdf._internal.column_labels_level: + raise ValueError("level should be an integer between [0, column_labels_level)") + + def gen_new_column_labels_entry(column_labels_entry): + if isinstance(column_labels_entry, tuple): + if level is None: + # rename all level columns + return tuple(map(columns_mapper_fn, column_labels_entry)) + else: + # only rename specified level column + entry_list = list(column_labels_entry) + entry_list[level] = columns_mapper_fn(entry_list[level]) + return tuple(entry_list) + else: + return columns_mapper_fn(column_labels_entry) + + new_column_labels = list(map(gen_new_column_labels_entry, kdf._internal.column_labels)) + + new_data_scols = [ + kdf._kser_for(old_label).rename(new_label) + for old_label, new_label in zip(kdf._internal.column_labels, new_column_labels) + ] + kdf = DataFrame(kdf._internal.with_new_columns(new_data_scols)) + if inplace: + self._update_internal_frame(kdf._internal) + return None + else: + return kdf + + def rename_axis( + self, + mapper: Optional[Any] = None, + index: Optional[Any] = None, + columns: Optional[Any] = None, + axis: Optional[Union[int, str]] = 0, + inplace: Optional[bool] = False, + ) -> Optional["DataFrame"]: + """ + Set the name of the axis for the index or columns. + + Parameters + ---------- + mapper : scalar, list-like, optional + A scalar, list-like, dict-like or functions transformations to + apply to the axis name attribute. + index, columns : scalar, list-like, dict-like or function, optional + A scalar, list-like, dict-like or functions transformations to + apply to that axis' values. + + Use either ``mapper`` and ``axis`` to + specify the axis to target with ``mapper``, or ``index`` + and/or ``columns``. + axis : {0 or 'index', 1 or 'columns'}, default 0 + The axis to rename. + inplace : bool, default False + Modifies the object directly, instead of creating a new DataFrame. + + Returns + ------- + DataFrame, or None if `inplace` is True. + + See Also + -------- + Series.rename : Alter Series index labels or name. + DataFrame.rename : Alter DataFrame index labels or name. + Index.rename : Set new names on index. + + Notes + ----- + ``DataFrame.rename_axis`` supports two calling conventions + + * ``(index=index_mapper, columns=columns_mapper, ...)`` + * ``(mapper, axis={'index', 'columns'}, ...)`` + + The first calling convention will only modify the names of + the index and/or the names of the Index object that is the columns. + + The second calling convention will modify the names of the + corresponding index specified by axis. + + We *highly* recommend using keyword arguments to clarify your + intent. + + Examples + -------- + >>> df = pp.DataFrame({"num_legs": [4, 4, 2], + ... "num_arms": [0, 0, 2]}, + ... index=["dog", "cat", "monkey"], + ... columns=["num_legs", "num_arms"]) + >>> df + num_legs num_arms + dog 4 0 + cat 4 0 + monkey 2 2 + + >>> df = df.rename_axis("animal").sort_index() + >>> df # doctest: +NORMALIZE_WHITESPACE + num_legs num_arms + animal + cat 4 0 + dog 4 0 + monkey 2 2 + + >>> df = df.rename_axis("limbs", axis="columns").sort_index() + >>> df # doctest: +NORMALIZE_WHITESPACE + limbs num_legs num_arms + animal + cat 4 0 + dog 4 0 + monkey 2 2 + + **MultiIndex** + + >>> index = pd.MultiIndex.from_product([['mammal'], + ... ['dog', 'cat', 'monkey']], + ... names=['type', 'name']) + >>> df = pp.DataFrame({"num_legs": [4, 4, 2], + ... "num_arms": [0, 0, 2]}, + ... index=index, + ... columns=["num_legs", "num_arms"]) + >>> df # doctest: +NORMALIZE_WHITESPACE + num_legs num_arms + type name + mammal dog 4 0 + cat 4 0 + monkey 2 2 + + >>> df.rename_axis(index={'type': 'class'}).sort_index() # doctest: +NORMALIZE_WHITESPACE + num_legs num_arms + class name + mammal cat 4 0 + dog 4 0 + monkey 2 2 + + >>> df.rename_axis(index=str.upper).sort_index() # doctest: +NORMALIZE_WHITESPACE + num_legs num_arms + TYPE NAME + mammal cat 4 0 + dog 4 0 + monkey 2 2 + """ + + def gen_names(v, curnames): + if is_scalar(v): + newnames = [v] + elif is_list_like(v) and not is_dict_like(v): + newnames = list(v) + elif is_dict_like(v): + newnames = [v[name] if name in v else name for name in curnames] + elif callable(v): + newnames = [v(name) for name in curnames] + else: + raise ValueError( + "`mapper` or `index` or `columns` should be " + "either dict-like or function type." + ) + + if len(newnames) != len(curnames): + raise ValueError( + "Length of new names must be {}, got {}".format(len(curnames), len(newnames)) + ) + + return [name if is_name_like_tuple(name) else (name,) for name in newnames] + + if mapper is not None and (index is not None or columns is not None): + raise TypeError("Cannot specify both 'mapper' and any of 'index' or 'columns'.") + + if mapper is not None: + axis = validate_axis(axis) + if axis == 0: + index = mapper + elif axis == 1: + columns = mapper + + column_label_names = ( + gen_names(columns, self.columns.names) + if columns is not None + else self._internal.column_label_names + ) + index_names = ( + gen_names(index, self.index.names) if index is not None else self._internal.index_names + ) + + internal = self._internal.copy( + index_names=index_names, column_label_names=column_label_names + ) + if inplace: + self._update_internal_frame(internal) + return None + else: + return DataFrame(internal) + + def keys(self) -> pd.Index: + """ + Return alias for columns. + + Returns + ------- + Index + Columns of the DataFrame. + + Examples + -------- + >>> df = pp.DataFrame([[1, 2], [4, 5], [7, 8]], + ... index=['cobra', 'viper', 'sidewinder'], + ... columns=['max_speed', 'shield']) + >>> df + max_speed shield + cobra 1 2 + viper 4 5 + sidewinder 7 8 + + >>> df.keys() + Index(['max_speed', 'shield'], dtype='object') + """ + return self.columns + + def pct_change(self, periods=1) -> "DataFrame": + """ + Percentage change between the current and a prior element. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + periods : int, default 1 + Periods to shift for forming percent change. + + Returns + ------- + DataFrame + + Examples + -------- + Percentage change in French franc, Deutsche Mark, and Italian lira + from 1980-01-01 to 1980-03-01. + + >>> df = pp.DataFrame({ + ... 'FR': [4.0405, 4.0963, 4.3149], + ... 'GR': [1.7246, 1.7482, 1.8519], + ... 'IT': [804.74, 810.01, 860.13]}, + ... index=['1980-01-01', '1980-02-01', '1980-03-01']) + >>> df + FR GR IT + 1980-01-01 4.0405 1.7246 804.74 + 1980-02-01 4.0963 1.7482 810.01 + 1980-03-01 4.3149 1.8519 860.13 + + >>> df.pct_change() + FR GR IT + 1980-01-01 NaN NaN NaN + 1980-02-01 0.013810 0.013684 0.006549 + 1980-03-01 0.053365 0.059318 0.061876 + + You can set periods to shift for forming percent change + + >>> df.pct_change(2) + FR GR IT + 1980-01-01 NaN NaN NaN + 1980-02-01 NaN NaN NaN + 1980-03-01 0.067912 0.073814 0.06883 + """ + window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween(-periods, -periods) + + def op(kser): + prev_row = F.lag(kser.spark.column, periods).over(window) + return ((kser.spark.column - prev_row) / prev_row).alias( + kser._internal.data_spark_column_names[0] + ) + + return self._apply_series_op(op, should_resolve=True) + + # TODO: axis = 1 + def idxmax(self, axis=0) -> "Series": + """ + Return index of first occurrence of maximum over requested axis. + NA/null values are excluded. + + .. note:: This API collect all rows with maximum value using `to_pandas()` + because we suppose the number of rows with max values are usually small in general. + + Parameters + ---------- + axis : 0 or 'index' + Can only be set to 0 at the moment. + + Returns + ------- + Series + + See Also + -------- + Series.idxmax + + Examples + -------- + >>> kdf = pp.DataFrame({'a': [1, 2, 3, 2], + ... 'b': [4.0, 2.0, 3.0, 1.0], + ... 'c': [300, 200, 400, 200]}) + >>> kdf + a b c + 0 1 4.0 300 + 1 2 2.0 200 + 2 3 3.0 400 + 3 2 1.0 200 + + >>> kdf.idxmax() + a 2 + b 0 + c 2 + dtype: int64 + + For Multi-column Index + + >>> kdf = pp.DataFrame({'a': [1, 2, 3, 2], + ... 'b': [4.0, 2.0, 3.0, 1.0], + ... 'c': [300, 200, 400, 200]}) + >>> kdf.columns = pd.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> kdf + a b c + x y z + 0 1 4.0 300 + 1 2 2.0 200 + 2 3 3.0 400 + 3 2 1.0 200 + + >>> kdf.idxmax() + a x 2 + b y 0 + c z 2 + dtype: int64 + """ + max_cols = map(lambda scol: F.max(scol), self._internal.data_spark_columns) + sdf_max = self._internal.spark_frame.select(*max_cols).head() + # `sdf_max` looks like below + # +------+------+------+ + # |(a, x)|(b, y)|(c, z)| + # +------+------+------+ + # | 3| 4.0| 400| + # +------+------+------+ + + conds = ( + scol == max_val for scol, max_val in zip(self._internal.data_spark_columns, sdf_max) + ) + cond = reduce(lambda x, y: x | y, conds) + + kdf = DataFrame(self._internal.with_filter(cond)) # type: "DataFrame" + + return cast(pp.Series, pp.from_pandas(kdf._to_internal_pandas().idxmax())) + + # TODO: axis = 1 + def idxmin(self, axis=0) -> "Series": + """ + Return index of first occurrence of minimum over requested axis. + NA/null values are excluded. + + .. note:: This API collect all rows with minimum value using `to_pandas()` + because we suppose the number of rows with min values are usually small in general. + + Parameters + ---------- + axis : 0 or 'index' + Can only be set to 0 at the moment. + + Returns + ------- + Series + + See Also + -------- + Series.idxmin + + Examples + -------- + >>> kdf = pp.DataFrame({'a': [1, 2, 3, 2], + ... 'b': [4.0, 2.0, 3.0, 1.0], + ... 'c': [300, 200, 400, 200]}) + >>> kdf + a b c + 0 1 4.0 300 + 1 2 2.0 200 + 2 3 3.0 400 + 3 2 1.0 200 + + >>> kdf.idxmin() + a 0 + b 3 + c 1 + dtype: int64 + + For Multi-column Index + + >>> kdf = pp.DataFrame({'a': [1, 2, 3, 2], + ... 'b': [4.0, 2.0, 3.0, 1.0], + ... 'c': [300, 200, 400, 200]}) + >>> kdf.columns = pd.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> kdf + a b c + x y z + 0 1 4.0 300 + 1 2 2.0 200 + 2 3 3.0 400 + 3 2 1.0 200 + + >>> kdf.idxmin() + a x 0 + b y 3 + c z 1 + dtype: int64 + """ + min_cols = map(lambda scol: F.min(scol), self._internal.data_spark_columns) + sdf_min = self._internal.spark_frame.select(*min_cols).head() + + conds = ( + scol == min_val for scol, min_val in zip(self._internal.data_spark_columns, sdf_min) + ) + cond = reduce(lambda x, y: x | y, conds) + + kdf = DataFrame(self._internal.with_filter(cond)) # type: "DataFrame" + + return cast(pp.Series, pp.from_pandas(kdf._to_internal_pandas().idxmin())) + + def info(self, verbose=None, buf=None, max_cols=None, null_counts=None) -> None: + """ + Print a concise summary of a DataFrame. + + This method prints information about a DataFrame including + the index dtype and column dtypes, non-null values and memory usage. + + Parameters + ---------- + verbose : bool, optional + Whether to print the full summary. + buf : writable buffer, defaults to sys.stdout + Where to send the output. By default, the output is printed to + sys.stdout. Pass a writable buffer if you need to further process + the output. + max_cols : int, optional + When to switch from the verbose to the truncated output. If the + DataFrame has more than `max_cols` columns, the truncated output + is used. + null_counts : bool, optional + Whether to show the non-null counts. + + Returns + ------- + None + This method prints a summary of a DataFrame and returns None. + + See Also + -------- + DataFrame.describe: Generate descriptive statistics of DataFrame + columns. + + Examples + -------- + >>> int_values = [1, 2, 3, 4, 5] + >>> text_values = ['alpha', 'beta', 'gamma', 'delta', 'epsilon'] + >>> float_values = [0.0, 0.25, 0.5, 0.75, 1.0] + >>> df = pp.DataFrame( + ... {"int_col": int_values, "text_col": text_values, "float_col": float_values}, + ... columns=['int_col', 'text_col', 'float_col']) + >>> df + int_col text_col float_col + 0 1 alpha 0.00 + 1 2 beta 0.25 + 2 3 gamma 0.50 + 3 4 delta 0.75 + 4 5 epsilon 1.00 + + Prints information of all columns: + + >>> df.info(verbose=True) # doctest: +SKIP + + Index: 5 entries, 0 to 4 + Data columns (total 3 columns): + # Column Non-Null Count Dtype + --- ------ -------------- ----- + 0 int_col 5 non-null int64 + 1 text_col 5 non-null object + 2 float_col 5 non-null float64 + dtypes: float64(1), int64(1), object(1) + + Prints a summary of columns count and its dtypes but not per column + information: + + >>> df.info(verbose=False) # doctest: +SKIP + + Index: 5 entries, 0 to 4 + Columns: 3 entries, int_col to float_col + dtypes: float64(1), int64(1), object(1) + + Pipe output of DataFrame.info to buffer instead of sys.stdout, get + buffer content and writes to a text file: + + >>> import io + >>> buffer = io.StringIO() + >>> df.info(buf=buffer) + >>> s = buffer.getvalue() + >>> with open('%s/info.txt' % path, "w", + ... encoding="utf-8") as f: + ... _ = f.write(s) + >>> with open('%s/info.txt' % path) as f: + ... f.readlines() # doctest: +SKIP + ["\\n", + 'Index: 5 entries, 0 to 4\\n', + 'Data columns (total 3 columns):\\n', + ' # Column Non-Null Count Dtype \\n', + '--- ------ -------------- ----- \\n', + ' 0 int_col 5 non-null int64 \\n', + ' 1 text_col 5 non-null object \\n', + ' 2 float_col 5 non-null float64\\n', + 'dtypes: float64(1), int64(1), object(1)'] + """ + # To avoid pandas' existing config affects Koalas. + # TODO: should we have corresponding Koalas configs? + with pd.option_context( + "display.max_info_columns", sys.maxsize, "display.max_info_rows", sys.maxsize + ): + try: + # hack to use pandas' info as is. + object.__setattr__(self, "_data", self) + count_func = self.count + self.count = lambda: count_func().to_pandas() # type: ignore + return pd.DataFrame.info( + self, + verbose=verbose, + buf=buf, + max_cols=max_cols, + memory_usage=False, + null_counts=null_counts, + ) + finally: + del self._data + self.count = count_func # type: ignore + + # TODO: fix parameter 'axis' and 'numeric_only' to work same as pandas' + def quantile( + self, + q: Union[float, Iterable[float]] = 0.5, + axis: Union[int, str] = 0, + numeric_only: bool = True, + accuracy: int = 10000, + ) -> Union["DataFrame", "Series"]: + """ + Return value at the given quantile. + + .. note:: Unlike pandas', the quantile in Koalas is an approximated quantile based upon + approximate percentile computation because computing quantile across a large dataset + is extremely expensive. + + Parameters + ---------- + q : float or array-like, default 0.5 (50% quantile) + 0 <= q <= 1, the quantile(s) to compute. + axis : int or str, default 0 or 'index' + Can only be set to 0 at the moment. + numeric_only : bool, default True + If False, the quantile of datetime and timedelta data will be computed as well. + Can only be set to True at the moment. + accuracy : int, optional + Default accuracy of approximation. Larger value means better accuracy. + The relative error can be deduced by 1.0 / accuracy. + + Returns + ------- + Series or DataFrame + If q is an array, a DataFrame will be returned where the + index is q, the columns are the columns of self, and the values are the quantiles. + If q is a float, a Series will be returned where the + index is the columns of self and the values are the quantiles. + + Examples + -------- + >>> kdf = pp.DataFrame({'a': [1, 2, 3, 4, 5], 'b': [6, 7, 8, 9, 0]}) + >>> kdf + a b + 0 1 6 + 1 2 7 + 2 3 8 + 3 4 9 + 4 5 0 + + >>> kdf.quantile(.5) + a 3.0 + b 7.0 + Name: 0.5, dtype: float64 + + >>> kdf.quantile([.25, .5, .75]) + a b + 0.25 2.0 6.0 + 0.50 3.0 7.0 + 0.75 4.0 8.0 + """ + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + if not isinstance(accuracy, int): + raise ValueError( + "accuracy must be an integer; however, got [%s]" % type(accuracy).__name__ + ) + + if isinstance(q, Iterable): + q = list(q) + + for v in q if isinstance(q, list) else [q]: + if not isinstance(v, float): + raise ValueError( + "q must be a float or an array of floats; however, [%s] found." % type(v) + ) + if v < 0.0 or v > 1.0: + raise ValueError("percentiles should all be in the interval [0, 1].") + + def quantile(spark_column, spark_type): + if isinstance(spark_type, (BooleanType, NumericType)): + return SF.percentile_approx(spark_column.cast(DoubleType()), q, accuracy) + else: + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + + if isinstance(q, list): + # First calculate the percentiles from all columns and map it to each `quantiles` + # by creating each entry as a struct. So, it becomes an array of structs as below: + # + # +-----------------------------------------+ + # | arrays| + # +-----------------------------------------+ + # |[[0.25, 2, 6], [0.5, 3, 7], [0.75, 4, 8]]| + # +-----------------------------------------+ + + percentile_cols = [] + percentile_col_names = [] + column_labels = [] + for label, column in zip( + self._internal.column_labels, self._internal.data_spark_column_names + ): + spark_type = self._internal.spark_type_for(label) + + is_numeric_or_boolean = isinstance(spark_type, (NumericType, BooleanType)) + keep_column = not numeric_only or is_numeric_or_boolean + + if keep_column: + percentile_col = quantile(self._internal.spark_column_for(label), spark_type) + percentile_cols.append(percentile_col.alias(column)) + percentile_col_names.append(column) + column_labels.append(label) + + if len(percentile_cols) == 0: + return DataFrame(index=q) + + sdf = self._internal.spark_frame.select(percentile_cols) + # Here, after select percentile cols, a spark_frame looks like below: + # +---------+---------+ + # | a| b| + # +---------+---------+ + # |[2, 3, 4]|[6, 7, 8]| + # +---------+---------+ + + cols_dict = OrderedDict() # type: OrderedDict + for column in percentile_col_names: + cols_dict[column] = list() + for i in range(len(q)): + cols_dict[column].append(scol_for(sdf, column).getItem(i).alias(column)) + + internal_index_column = SPARK_DEFAULT_INDEX_NAME + cols = [] + for i, col in enumerate(zip(*cols_dict.values())): + cols.append(F.struct(F.lit(q[i]).alias(internal_index_column), *col)) + sdf = sdf.select(F.array(*cols).alias("arrays")) + + # And then, explode it and manually set the index. + # +-----------------+---+---+ + # |__index_level_0__| a| b| + # +-----------------+---+---+ + # | 0.25| 2| 6| + # | 0.5| 3| 7| + # | 0.75| 4| 8| + # +-----------------+---+---+ + sdf = sdf.select(F.explode(F.col("arrays"))).selectExpr("col.*") + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, internal_index_column)], + column_labels=column_labels, + data_spark_columns=[scol_for(sdf, col) for col in percentile_col_names], + ) + return DataFrame(internal) + else: + return self._reduce_for_stat_function( + quantile, name="quantile", numeric_only=numeric_only + ).rename(q) + + def query(self, expr, inplace=False) -> Optional["DataFrame"]: + """ + Query the columns of a DataFrame with a boolean expression. + + .. note:: Internal columns that starting with a '__' prefix are able to access, however, + they are not supposed to be accessed. + + .. note:: This API delegates to Spark SQL so the syntax follows Spark SQL. Therefore, the + pandas specific syntax such as `@` is not supported. If you want the pandas syntax, + you can work around with :meth:`DataFrame.koalas.apply_batch`, but you should + be aware that `query_func` will be executed at different nodes in a distributed manner. + So, for example, to use `@` syntax, make sure the variable is serialized by, for + example, putting it within the closure as below. + + >>> df = pp.DataFrame({'A': range(2000), 'B': range(2000)}) + >>> def query_func(pdf): + ... num = 1995 + ... return pdf.query('A > @num') + >>> df.koalas.apply_batch(query_func) + A B + 1996 1996 1996 + 1997 1997 1997 + 1998 1998 1998 + 1999 1999 1999 + + Parameters + ---------- + expr : str + The query string to evaluate. + + You can refer to column names that contain spaces by surrounding + them in backticpp. + + For example, if one of your columns is called ``a a`` and you want + to sum it with ``b``, your query should be ```a a` + b``. + + inplace : bool + Whether the query should modify the data in place or return + a modified copy. + + Returns + ------- + DataFrame + DataFrame resulting from the provided query expression. + + Examples + -------- + >>> df = pp.DataFrame({'A': range(1, 6), + ... 'B': range(10, 0, -2), + ... 'C C': range(10, 5, -1)}) + >>> df + A B C C + 0 1 10 10 + 1 2 8 9 + 2 3 6 8 + 3 4 4 7 + 4 5 2 6 + + >>> df.query('A > B') + A B C C + 4 5 2 6 + + The previous expression is equivalent to + + >>> df[df.A > df.B] + A B C C + 4 5 2 6 + + For columns with spaces in their name, you can use backtick quoting. + + >>> df.query('B == `C C`') + A B C C + 0 1 10 10 + + The previous expression is equivalent to + + >>> df[df.B == df['C C']] + A B C C + 0 1 10 10 + """ + if isinstance(self.columns, pd.MultiIndex): + raise ValueError("Doesn't support for MultiIndex columns") + if not isinstance(expr, str): + raise ValueError( + "expr must be a string to be evaluated, {} given".format(type(expr).__name__) + ) + inplace = validate_bool_kwarg(inplace, "inplace") + + data_columns = [label[0] for label in self._internal.column_labels] + sdf = self._internal.spark_frame.select( + self._internal.index_spark_columns + + [ + scol.alias(col) + for scol, col in zip(self._internal.data_spark_columns, data_columns) + ] + ).filter(expr) + internal = self._internal.with_new_sdf(sdf, data_columns=data_columns) + + if inplace: + self._update_internal_frame(internal) + return None + else: + return DataFrame(internal) + + def explain(self, extended: Optional[bool] = None, mode: Optional[str] = None) -> None: + warnings.warn( + "DataFrame.explain is deprecated as of DataFrame.spark.explain. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.explain(extended, mode) + + explain.__doc__ = SparkFrameMethods.explain.__doc__ + + def take(self, indices, axis=0, **kwargs) -> "DataFrame": + """ + Return the elements in the given *positional* indices along an axis. + + This means that we are not indexing according to actual values in + the index attribute of the object. We are indexing according to the + actual position of the element in the object. + + Parameters + ---------- + indices : array-like + An array of ints indicating which positions to take. + axis : {0 or 'index', 1 or 'columns', None}, default 0 + The axis on which to select elements. ``0`` means that we are + selecting rows, ``1`` means that we are selecting columns. + **kwargs + For compatibility with :meth:`numpy.take`. Has no effect on the + output. + + Returns + ------- + taken : same type as caller + An array-like containing the elements taken from the object. + + See Also + -------- + DataFrame.loc : Select a subset of a DataFrame by labels. + DataFrame.iloc : Select a subset of a DataFrame by positions. + numpy.take : Take elements from an array along an axis. + + Examples + -------- + >>> df = pp.DataFrame([('falcon', 'bird', 389.0), + ... ('parrot', 'bird', 24.0), + ... ('lion', 'mammal', 80.5), + ... ('monkey', 'mammal', np.nan)], + ... columns=['name', 'class', 'max_speed'], + ... index=[0, 2, 3, 1]) + >>> df + name class max_speed + 0 falcon bird 389.0 + 2 parrot bird 24.0 + 3 lion mammal 80.5 + 1 monkey mammal NaN + + Take elements at positions 0 and 3 along the axis 0 (default). + + Note how the actual indices selected (0 and 1) do not correspond to + our selected indices 0 and 3. That's because we are selecting the 0th + and 3rd rows, not rows whose indices equal 0 and 3. + + >>> df.take([0, 3]).sort_index() + name class max_speed + 0 falcon bird 389.0 + 1 monkey mammal NaN + + Take elements at indices 1 and 2 along the axis 1 (column selection). + + >>> df.take([1, 2], axis=1) + class max_speed + 0 bird 389.0 + 2 bird 24.0 + 3 mammal 80.5 + 1 mammal NaN + + We may take elements using negative integers for positive indices, + starting from the end of the object, just like with Python lists. + + >>> df.take([-1, -2]).sort_index() + name class max_speed + 1 monkey mammal NaN + 3 lion mammal 80.5 + """ + axis = validate_axis(axis) + if not is_list_like(indices) or isinstance(indices, (dict, set)): + raise ValueError("`indices` must be a list-like except dict or set") + if axis == 0: + return cast(DataFrame, self.iloc[indices, :]) + else: + return cast(DataFrame, self.iloc[:, indices]) + + def eval(self, expr, inplace=False) -> Optional[Union["DataFrame", "Series"]]: + """ + Evaluate a string describing operations on DataFrame columns. + + Operates on columns only, not specific rows or elements. This allows + `eval` to run arbitrary code, which can make you vulnerable to code + injection if you pass user input to this function. + + Parameters + ---------- + expr : str + The expression string to evaluate. + inplace : bool, default False + If the expression contains an assignment, whether to perform the + operation inplace and mutate the existing DataFrame. Otherwise, + a new DataFrame is returned. + + Returns + ------- + The result of the evaluation. + + See Also + -------- + DataFrame.query : Evaluates a boolean expression to query the columns + of a frame. + DataFrame.assign : Can evaluate an expression or function to create new + values for a column. + eval : Evaluate a Python expression as a string using various + backends. + + Examples + -------- + >>> df = pp.DataFrame({'A': range(1, 6), 'B': range(10, 0, -2)}) + >>> df + A B + 0 1 10 + 1 2 8 + 2 3 6 + 3 4 4 + 4 5 2 + >>> df.eval('A + B') + 0 11 + 1 10 + 2 9 + 3 8 + 4 7 + dtype: int64 + + Assignment is allowed though by default the original DataFrame is not + modified. + + >>> df.eval('C = A + B') + A B C + 0 1 10 11 + 1 2 8 10 + 2 3 6 9 + 3 4 4 8 + 4 5 2 7 + >>> df + A B + 0 1 10 + 1 2 8 + 2 3 6 + 3 4 4 + 4 5 2 + + Use ``inplace=True`` to modify the original DataFrame. + + >>> df.eval('C = A + B', inplace=True) + >>> df + A B C + 0 1 10 11 + 1 2 8 10 + 2 3 6 9 + 3 4 4 8 + 4 5 2 7 + """ + from pyspark.pandas.series import first_series + + if isinstance(self.columns, pd.MultiIndex): + raise ValueError("`eval` is not supported for multi-index columns") + inplace = validate_bool_kwarg(inplace, "inplace") + should_return_series = False + series_name = None + should_return_scalar = False + + # Since `eval_func` doesn't have a type hint, inferring the schema is always preformed + # in the `apply_batch`. Hence, the variables `should_return_series`, `series_name`, + # and `should_return_scalar` can be updated. + def eval_func(pdf): + nonlocal should_return_series + nonlocal series_name + nonlocal should_return_scalar + result_inner = pdf.eval(expr, inplace=inplace) + if inplace: + result_inner = pdf + if isinstance(result_inner, pd.Series): + should_return_series = True + series_name = result_inner.name + result_inner = result_inner.to_frame() + elif is_scalar(result_inner): + should_return_scalar = True + result_inner = pd.Series(result_inner).to_frame() + return result_inner + + result = self.koalas.apply_batch(eval_func) + if inplace: + # Here, the result is always a frame because the error is thrown during schema inference + # from pandas. + self._update_internal_frame(result._internal, requires_same_anchor=False) + return None + elif should_return_series: + return first_series(result).rename(series_name) + elif should_return_scalar: + return first_series(result)[0] + else: + # Returns a frame + return result + + def explode(self, column) -> "DataFrame": + """ + Transform each element of a list-like to a row, replicating index values. + + Parameters + ---------- + column : str or tuple + Column to explode. + + Returns + ------- + DataFrame + Exploded lists to rows of the subset columns; + index will be duplicated for these rows. + + See Also + -------- + DataFrame.unstack : Pivot a level of the (necessarily hierarchical) + index labels. + DataFrame.melt : Unpivot a DataFrame from wide format to long format. + + Examples + -------- + >>> df = pp.DataFrame({'A': [[1, 2, 3], [], [3, 4]], 'B': 1}) + >>> df + A B + 0 [1, 2, 3] 1 + 1 [] 1 + 2 [3, 4] 1 + + >>> df.explode('A') + A B + 0 1.0 1 + 0 2.0 1 + 0 3.0 1 + 1 NaN 1 + 2 3.0 1 + 2 4.0 1 + """ + from pyspark.pandas.series import Series + + if not is_name_like_value(column): + raise ValueError("column must be a scalar") + + kdf = DataFrame(self._internal.resolved_copy) # type: "DataFrame" + kser = kdf[column] + if not isinstance(kser, Series): + raise ValueError( + "The column %s is not unique. For a multi-index, the label must be a tuple " + "with elements corresponding to each level." % name_like_string(column) + ) + if not isinstance(kser.spark.data_type, ArrayType): + return self.copy() + + sdf = kdf._internal.spark_frame.withColumn( + kser._internal.data_spark_column_names[0], F.explode_outer(kser.spark.column) + ) + + data_dtypes = kdf._internal.data_dtypes.copy() + data_dtypes[kdf._internal.column_labels.index(kser._column_label)] = None # TODO: dtype? + + internal = kdf._internal.with_new_sdf(sdf, data_dtypes=data_dtypes) + return DataFrame(internal) + + def mad(self, axis=0) -> "Series": + """ + Return the mean absolute deviation of values. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + >>> df.mad() + a 0.666667 + b 0.066667 + dtype: float64 + + >>> df.mad(axis=1) + 0 0.45 + 1 0.90 + 2 1.35 + 3 NaN + dtype: float64 + """ + from pyspark.pandas.series import first_series + + axis = validate_axis(axis) + + if axis == 0: + + def get_spark_column(kdf, label): + scol = kdf._internal.spark_column_for(label) + col_type = kdf._internal.spark_type_for(label) + + if isinstance(col_type, BooleanType): + scol = scol.cast("integer") + + return scol + + new_column_labels = [] + for label in self._internal.column_labels: + # Filtering out only columns of numeric and boolean type column. + dtype = self._kser_for(label).spark.data_type + if isinstance(dtype, (NumericType, BooleanType)): + new_column_labels.append(label) + + new_columns = [ + F.avg(get_spark_column(self, label)).alias(name_like_string(label)) + for label in new_column_labels + ] + + mean_data = self._internal.spark_frame.select(new_columns).first() + + new_columns = [ + F.avg( + F.abs(get_spark_column(self, label) - mean_data[name_like_string(label)]) + ).alias(name_like_string(label)) + for label in new_column_labels + ] + + sdf = self._internal.spark_frame.select( + [F.lit(None).cast(StringType()).alias(SPARK_DEFAULT_INDEX_NAME)] + new_columns + ) + + # The data is expected to be small so it's fine to transpose/use default index. + with pp.option_context("compute.max_rows", 1): + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, SPARK_DEFAULT_INDEX_NAME)], + column_labels=new_column_labels, + column_label_names=self._internal.column_label_names, + ) + return first_series(DataFrame(internal).transpose()) + + else: + + @pandas_udf(returnType=DoubleType()) + def calculate_columns_axis(*cols): + return pd.concat(cols, axis=1).mad(axis=1) + + internal = self._internal.copy( + column_labels=[None], + data_spark_columns=[ + calculate_columns_axis(*self._internal.data_spark_columns).alias( + SPARK_DEFAULT_SERIES_NAME + ) + ], + data_dtypes=[None], + column_label_names=None, + ) + return first_series(DataFrame(internal)) + + def tail(self, n=5) -> "DataFrame": + """ + Return the last `n` rows. + + This function returns last `n` rows from the object based on + position. It is useful for quickly verifying data, for example, + after sorting or appending rows. + + For negative values of `n`, this function returns all rows except + the first `n` rows, equivalent to ``df[n:]``. + + Parameters + ---------- + n : int, default 5 + Number of rows to select. + + Returns + ------- + type of caller + The last `n` rows of the caller object. + + See Also + -------- + DataFrame.head : The first `n` rows of the caller object. + + Examples + -------- + >>> df = pp.DataFrame({'animal': ['alligator', 'bee', 'falcon', 'lion', + ... 'monkey', 'parrot', 'shark', 'whale', 'zebra']}) + >>> df + animal + 0 alligator + 1 bee + 2 falcon + 3 lion + 4 monkey + 5 parrot + 6 shark + 7 whale + 8 zebra + + Viewing the last 5 lines + + >>> df.tail() # doctest: +SKIP + animal + 4 monkey + 5 parrot + 6 shark + 7 whale + 8 zebra + + Viewing the last `n` lines (three in this case) + + >>> df.tail(3) # doctest: +SKIP + animal + 6 shark + 7 whale + 8 zebra + + For negative values of `n` + + >>> df.tail(-3) # doctest: +SKIP + animal + 3 lion + 4 monkey + 5 parrot + 6 shark + 7 whale + 8 zebra + """ + if LooseVersion(pyspark.__version__) < LooseVersion("3.0"): + raise RuntimeError("tail can be used in PySpark >= 3.0") + if not isinstance(n, int): + raise TypeError("bad operand type for unary -: '{}'".format(type(n).__name__)) + if n < 0: + n = len(self) + n + if n <= 0: + return pp.DataFrame(self._internal.with_filter(F.lit(False))) + # Should use `resolved_copy` here for the case like `(kdf + 1).tail()` + sdf = self._internal.resolved_copy.spark_frame + rows = sdf.tail(n) + new_sdf = default_session().createDataFrame(rows, sdf.schema) + + return DataFrame(self._internal.with_new_sdf(new_sdf)) + + def align( + self, + other: Union["DataFrame", "Series"], + join: str = "outer", + axis: Optional[Union[int, str]] = None, + copy: bool = True, + ) -> Tuple["DataFrame", Union["DataFrame", "Series"]]: + """ + Align two objects on their axes with the specified join method. + + Join method is specified for each axis Index. + + Parameters + ---------- + other : DataFrame or Series + join : {{'outer', 'inner', 'left', 'right'}}, default 'outer' + axis : allowed axis of the other object, default None + Align on index (0), columns (1), or both (None). + copy : bool, default True + Always returns new objects. If copy=False and no reindexing is + required then original objects are returned. + + Returns + ------- + (left, right) : (DataFrame, type of other) + Aligned objects. + + Examples + -------- + >>> pp.set_option("compute.ops_on_diff_frames", True) + >>> df1 = pp.DataFrame({"a": [1, 2, 3], "b": ["a", "b", "c"]}, index=[10, 20, 30]) + >>> df2 = pp.DataFrame({"a": [4, 5, 6], "c": ["d", "e", "f"]}, index=[10, 11, 12]) + + Align both axis: + + >>> aligned_l, aligned_r = df1.align(df2) + >>> aligned_l.sort_index() + a b c + 10 1.0 a NaN + 11 NaN None NaN + 12 NaN None NaN + 20 2.0 b NaN + 30 3.0 c NaN + >>> aligned_r.sort_index() + a b c + 10 4.0 NaN d + 11 5.0 NaN e + 12 6.0 NaN f + 20 NaN NaN None + 30 NaN NaN None + + Align only axis=0 (index): + + >>> aligned_l, aligned_r = df1.align(df2, axis=0) + >>> aligned_l.sort_index() + a b + 10 1.0 a + 11 NaN None + 12 NaN None + 20 2.0 b + 30 3.0 c + >>> aligned_r.sort_index() + a c + 10 4.0 d + 11 5.0 e + 12 6.0 f + 20 NaN None + 30 NaN None + + Align only axis=1 (column): + + >>> aligned_l, aligned_r = df1.align(df2, axis=1) + >>> aligned_l.sort_index() + a b c + 10 1 a NaN + 20 2 b NaN + 30 3 c NaN + >>> aligned_r.sort_index() + a b c + 10 4 NaN d + 11 5 NaN e + 12 6 NaN f + + Align with the join type "inner": + + >>> aligned_l, aligned_r = df1.align(df2, join="inner") + >>> aligned_l.sort_index() + a + 10 1 + >>> aligned_r.sort_index() + a + 10 4 + + Align with a Series: + + >>> s = pp.Series([7, 8, 9], index=[10, 11, 12]) + >>> aligned_l, aligned_r = df1.align(s, axis=0) + >>> aligned_l.sort_index() + a b + 10 1.0 a + 11 NaN None + 12 NaN None + 20 2.0 b + 30 3.0 c + >>> aligned_r.sort_index() + 10 7.0 + 11 8.0 + 12 9.0 + 20 NaN + 30 NaN + dtype: float64 + + >>> pp.reset_option("compute.ops_on_diff_frames") + """ + from pyspark.pandas.series import Series, first_series + + if not isinstance(other, (DataFrame, Series)): + raise TypeError("unsupported type: {}".format(type(other).__name__)) + + how = validate_how(join) + axis = validate_axis(axis, None) + + right_is_series = isinstance(other, Series) + if right_is_series: + if axis is None: + raise ValueError("Must specify axis=0 or 1") + elif axis != 0: + raise NotImplementedError( + "align currently only works for axis=0 when right is Series" + ) + + left = self + right = other + + if (axis is None or axis == 0) and not same_anchor(left, right): + combined = combine_frames(left, right, how=how) + left = combined["this"] + right = combined["that"] + + if right_is_series: + right = first_series(right).rename(other.name) + + if ( + axis is None or axis == 1 + ) and left._internal.column_labels != right._internal.column_labels: + + if left._internal.column_labels_level != right._internal.column_labels_level: + raise ValueError("cannot join with no overlapping index names") + + left = left.copy() + right = right.copy() + + if how == "full": + column_labels = sorted( + list(set(left._internal.column_labels) | set(right._internal.column_labels)) + ) + elif how == "inner": + column_labels = sorted( + list(set(left._internal.column_labels) & set(right._internal.column_labels)) + ) + elif how == "left": + column_labels = left._internal.column_labels + else: + column_labels = right._internal.column_labels + + for label in column_labels: + if label not in left._internal.column_labels: + left[label] = F.lit(None).cast(DoubleType()) + left = left[column_labels] + for label in column_labels: + if label not in right._internal.column_labels: + right[label] = F.lit(None).cast(DoubleType()) + right = right[column_labels] + + return (left.copy(), right.copy()) if copy else (left, right) + + @staticmethod + def from_dict(data, orient="columns", dtype=None, columns=None) -> "DataFrame": + """ + Construct DataFrame from dict of array-like or dicts. + + Creates DataFrame object from dictionary by columns or by index + allowing dtype specification. + + Parameters + ---------- + data : dict + Of the form {field : array-like} or {field : dict}. + orient : {'columns', 'index'}, default 'columns' + The "orientation" of the data. If the keys of the passed dict + should be the columns of the resulting DataFrame, pass 'columns' + (default). Otherwise if the keys should be rows, pass 'index'. + dtype : dtype, default None + Data type to force, otherwise infer. + columns : list, default None + Column labels to use when ``orient='index'``. Raises a ValueError + if used with ``orient='columns'``. + + Returns + ------- + DataFrame + + See Also + -------- + DataFrame.from_records : DataFrame from structured ndarray, sequence + of tuples or dicts, or DataFrame. + DataFrame : DataFrame object creation using constructor. + + Examples + -------- + By default the keys of the dict become the DataFrame columns: + + >>> data = {'col_1': [3, 2, 1, 0], 'col_2': [10, 20, 30, 40]} + >>> pp.DataFrame.from_dict(data) + col_1 col_2 + 0 3 10 + 1 2 20 + 2 1 30 + 3 0 40 + + Specify ``orient='index'`` to create the DataFrame using dictionary + keys as rows: + + >>> data = {'row_1': [3, 2, 1, 0], 'row_2': [10, 20, 30, 40]} + >>> pp.DataFrame.from_dict(data, orient='index').sort_index() + 0 1 2 3 + row_1 3 2 1 0 + row_2 10 20 30 40 + + When using the 'index' orientation, the column names can be + specified manually: + + >>> pp.DataFrame.from_dict(data, orient='index', + ... columns=['A', 'B', 'C', 'D']).sort_index() + A B C D + row_1 3 2 1 0 + row_2 10 20 30 40 + """ + return DataFrame(pd.DataFrame.from_dict(data, orient=orient, dtype=dtype, columns=columns)) + + def _to_internal_pandas(self): + """ + Return a pandas DataFrame directly from _internal to avoid overhead of copy. + + This method is for internal use only. + """ + return self._internal.to_pandas_frame + + def _get_or_create_repr_pandas_cache(self, n): + if not hasattr(self, "_repr_pandas_cache") or n not in self._repr_pandas_cache: + object.__setattr__( + self, "_repr_pandas_cache", {n: self.head(n + 1)._to_internal_pandas()} + ) + return self._repr_pandas_cache[n] + + def __repr__(self): + max_display_count = get_option("display.max_rows") + if max_display_count is None: + return self._to_internal_pandas().to_string() + + pdf = self._get_or_create_repr_pandas_cache(max_display_count) + pdf_length = len(pdf) + pdf = pdf.iloc[:max_display_count] + if pdf_length > max_display_count: + repr_string = pdf.to_string(show_dimensions=True) + match = REPR_PATTERN.search(repr_string) + if match is not None: + nrows = match.group("rows") + ncols = match.group("columns") + footer = "\n\n[Showing only the first {nrows} rows x {ncols} columns]".format( + nrows=nrows, ncols=ncols + ) + return REPR_PATTERN.sub(footer, repr_string) + return pdf.to_string() + + def _repr_html_(self): + max_display_count = get_option("display.max_rows") + # pandas 0.25.1 has a regression about HTML representation so 'bold_rows' + # has to be set as False explicitly. See https://github.com/pandas-dev/pandas/issues/28204 + bold_rows = not (LooseVersion("0.25.1") == LooseVersion(pd.__version__)) + if max_display_count is None: + return self._to_internal_pandas().to_html(notebook=True, bold_rows=bold_rows) + + pdf = self._get_or_create_repr_pandas_cache(max_display_count) + pdf_length = len(pdf) + pdf = pdf.iloc[:max_display_count] + if pdf_length > max_display_count: + repr_html = pdf.to_html(show_dimensions=True, notebook=True, bold_rows=bold_rows) + match = REPR_HTML_PATTERN.search(repr_html) + if match is not None: + nrows = match.group("rows") + ncols = match.group("columns") + by = chr(215) + footer = ( + "\n

Showing only the first {rows} rows " + "{by} {cols} columns

\n".format(rows=nrows, by=by, cols=ncols) + ) + return REPR_HTML_PATTERN.sub(footer, repr_html) + return pdf.to_html(notebook=True, bold_rows=bold_rows) + + def __getitem__(self, key): + from pyspark.pandas.series import Series + + if key is None: + raise KeyError("none key") + elif isinstance(key, Series): + return self.loc[key.astype(bool)] + elif isinstance(key, slice): + if any(type(n) == int or None for n in [key.start, key.stop]): + # Seems like pandas Frame always uses int as positional search when slicing + # with ints. + return self.iloc[key] + return self.loc[key] + elif is_name_like_value(key): + return self.loc[:, key] + elif is_list_like(key): + return self.loc[:, list(key)] + raise NotImplementedError(key) + + def __setitem__(self, key, value): + from pyspark.pandas.series import Series + + if isinstance(value, (DataFrame, Series)) and not same_anchor(value, self): + # Different Series or DataFrames + level = self._internal.column_labels_level + key = DataFrame._index_normalized_label(level, key) + value = DataFrame._index_normalized_frame(level, value) + + def assign_columns(kdf, this_column_labels, that_column_labels): + assert len(key) == len(that_column_labels) + # Note that here intentionally uses `zip_longest` that combine + # that_columns. + for k, this_label, that_label in zip_longest( + key, this_column_labels, that_column_labels + ): + yield (kdf._kser_for(that_label), tuple(["that", *k])) + if this_label is not None and this_label[1:] != k: + yield (kdf._kser_for(this_label), this_label) + + kdf = align_diff_frames(assign_columns, self, value, fillna=False, how="left") + elif isinstance(value, list): + if len(self) != len(value): + raise ValueError("Length of values does not match length of index") + + # TODO: avoid using default index? + with option_context( + "compute.default_index_type", + "distributed-sequence", + "compute.ops_on_diff_frames", + True, + ): + kdf = self.reset_index() + kdf[key] = pp.DataFrame(value) + kdf = kdf.set_index(kdf.columns[: self._internal.index_level]) + kdf.index.names = self.index.names + + elif isinstance(key, list): + assert isinstance(value, DataFrame) + # Same DataFrames. + field_names = value.columns + kdf = self._assign({k: value[c] for k, c in zip(key, field_names)}) + else: + # Same Series. + kdf = self._assign({key: value}) + + self._update_internal_frame(kdf._internal) + + @staticmethod + def _index_normalized_label(level, labels): + """ + Returns a label that is normalized against the current column index level. + For example, the key "abc" can be ("abc", "", "") if the current Frame has + a multi-index for its column + """ + if is_name_like_tuple(labels): + labels = [labels] + elif is_name_like_value(labels): + labels = [(labels,)] + else: + labels = [k if is_name_like_tuple(k) else (k,) for k in labels] + + if any(len(label) > level for label in labels): + raise KeyError( + "Key length ({}) exceeds index depth ({})".format( + max(len(label) for label in labels), level + ) + ) + return [tuple(list(label) + ([""] * (level - len(label)))) for label in labels] + + @staticmethod + def _index_normalized_frame(level, kser_or_kdf): + """ + Returns a frame that is normalized against the current column index level. + For example, the name in `pd.Series([...], name="abc")` can be can be + ("abc", "", "") if the current DataFrame has a multi-index for its column + """ + from pyspark.pandas.series import Series + + if isinstance(kser_or_kdf, Series): + kdf = kser_or_kdf.to_frame() + else: + assert isinstance(kser_or_kdf, DataFrame), type(kser_or_kdf) + kdf = kser_or_kdf.copy() + + kdf.columns = pd.MultiIndex.from_tuples( + [ + tuple([name_like_string(label)] + ([""] * (level - 1))) + for label in kdf._internal.column_labels + ], + ) + + return kdf + + def __getattr__(self, key: str) -> Any: + if key.startswith("__"): + raise AttributeError(key) + if hasattr(_MissingPandasLikeDataFrame, key): + property_or_func = getattr(_MissingPandasLikeDataFrame, key) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + + try: + return self.loc[:, key] + except KeyError: + raise AttributeError( + "'%s' object has no attribute '%s'" % (self.__class__.__name__, key) + ) + + def __setattr__(self, key: str, value) -> None: + try: + object.__getattribute__(self, key) + return object.__setattr__(self, key, value) + except AttributeError: + pass + + if (key,) in self._internal.column_labels: + self[key] = value + else: + msg = "Koalas doesn't allow columns to be created via a new attribute name" + if is_testing(): + raise AssertionError(msg) + else: + warnings.warn(msg, UserWarning) + + def __len__(self): + return self._internal.resolved_copy.spark_frame.count() + + def __dir__(self): + fields = [ + f for f in self._internal.resolved_copy.spark_frame.schema.fieldNames() if " " not in f + ] + return super().__dir__() + fields + + def __iter__(self): + return iter(self.columns) + + # NDArray Compat + def __array_ufunc__(self, ufunc: Callable, method: str, *inputs: Any, **kwargs: Any): + # TODO: is it possible to deduplicate it with '_map_series_op'? + if all(isinstance(inp, DataFrame) for inp in inputs) and any( + not same_anchor(inp, inputs[0]) for inp in inputs + ): + # binary only + assert len(inputs) == 2 + this = inputs[0] + that = inputs[1] + if this._internal.column_labels_level != that._internal.column_labels_level: + raise ValueError("cannot join with no overlapping index names") + + # Different DataFrames + def apply_op(kdf, this_column_labels, that_column_labels): + for this_label, that_label in zip(this_column_labels, that_column_labels): + yield ( + ufunc( + kdf._kser_for(this_label), kdf._kser_for(that_label), **kwargs + ).rename(this_label), + this_label, + ) + + return align_diff_frames(apply_op, this, that, fillna=True, how="full") + else: + # DataFrame and Series + applied = [] + this = inputs[0] + assert all(inp is this for inp in inputs if isinstance(inp, DataFrame)) + + for label in this._internal.column_labels: + arguments = [] + for inp in inputs: + arguments.append(inp[label] if isinstance(inp, DataFrame) else inp) + # both binary and unary. + applied.append(ufunc(*arguments, **kwargs).rename(label)) + + internal = this._internal.with_new_columns(applied) + return DataFrame(internal) + + if sys.version_info >= (3, 7): + + def __class_getitem__(cls, params): + # This is a workaround to support variadic generic in DataFrame in Python 3.7. + # See https://github.com/python/typing/issues/193 + # we always wraps the given type hints by a tuple to mimic the variadic generic. + return _create_tuple_for_frame_type(params) + + elif (3, 5) <= sys.version_info < (3, 7): + # This is a workaround to support variadic generic in DataFrame in Python 3.5+ + # The implementation is in its metaclass so this flag is needed to distinguish + # Koalas DataFrame. + is_dataframe = None + + +def _reduce_spark_multi(sdf, aggs): + """ + Performs a reduction on a spark DataFrame, the functions being known sql aggregate functions. + """ + assert isinstance(sdf, spark.DataFrame) + sdf0 = sdf.agg(*aggs) + l = sdf0.limit(2).toPandas() + assert len(l) == 1, (sdf, l) + row = l.iloc[0] + l2 = list(row) + assert len(l2) == len(aggs), (row, l2) + return l2 + + +class CachedDataFrame(DataFrame): + """ + Cached Koalas DataFrame, which corresponds to pandas DataFrame logically, but internally + it caches the corresponding Spark DataFrame. + """ + + def __init__(self, internal, storage_level=None): + if storage_level is None: + object.__setattr__(self, "_cached", internal.spark_frame.cache()) + elif isinstance(storage_level, StorageLevel): + object.__setattr__(self, "_cached", internal.spark_frame.persist(storage_level)) + else: + raise TypeError( + "Only a valid pyspark.StorageLevel type is acceptable for the `storage_level`" + ) + super().__init__(internal) + + def __enter__(self): + return self + + def __exit__(self, exception_type, exception_value, traceback): + self.spark.unpersist() + + # create accessor for Spark related methods. + spark = CachedAccessor("spark", CachedSparkFrameMethods) + + @property + def storage_level(self) -> StorageLevel: + warnings.warn( + "DataFrame.storage_level is deprecated as of DataFrame.spark.storage_level. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.storage_level + + storage_level.__doc__ = CachedSparkFrameMethods.storage_level.__doc__ + + def unpersist(self) -> None: + warnings.warn( + "DataFrame.unpersist is deprecated as of DataFrame.spark.unpersist. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.unpersist() + + unpersist.__doc__ = CachedSparkFrameMethods.unpersist.__doc__ diff --git a/python/pyspark/pandas/generic.py b/python/pyspark/pandas/generic.py new file mode 100644 index 0000000000..46d6051b89 --- /dev/null +++ b/python/pyspark/pandas/generic.py @@ -0,0 +1,3102 @@ +# +# 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. +# + +""" +A base class of DataFrame/Column to behave similar to pandas DataFrame/Series. +""" +from abc import ABCMeta, abstractmethod +from collections import Counter +from collections.abc import Iterable +from distutils.version import LooseVersion +from functools import reduce +from typing import Any, List, Optional, Tuple, Union, TYPE_CHECKING, cast +import warnings + +import numpy as np # noqa: F401 +import pandas as pd +from pandas.api.types import is_list_like + +import pyspark +from pyspark.sql import functions as F +from pyspark.sql.types import ( + BooleanType, + DoubleType, + FloatType, + IntegralType, + LongType, + NumericType, +) + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas.indexing import AtIndexer, iAtIndexer, iLocIndexer, LocIndexer +from pyspark.pandas.internal import InternalFrame +from pyspark.pandas.spark import functions as SF +from pyspark.pandas.typedef import Scalar, spark_type_to_pandas_dtype +from pyspark.pandas.utils import ( + is_name_like_tuple, + is_name_like_value, + name_like_string, + scol_for, + sql_conf, + validate_arguments_and_invoke_function, + validate_axis, + SPARK_CONF_ARROW_ENABLED, +) +from pyspark.pandas.window import Rolling, Expanding + +if TYPE_CHECKING: + from pyspark.pandas.frame import DataFrame # noqa: F401 (SPARK-34943) + from pyspark.pandas.groupby import DataFrameGroupBy, SeriesGroupBy # noqa: F401 (SPARK-34943) + from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943) + + +class Frame(object, metaclass=ABCMeta): + """ + The base class for both DataFrame and Series. + """ + + @abstractmethod + def __getitem__(self, key): + pass + + @property + @abstractmethod + def _internal(self) -> InternalFrame: + pass + + @abstractmethod + def _apply_series_op(self, op, should_resolve: bool = False): + pass + + @abstractmethod + def _reduce_for_stat_function(self, sfun, name, axis=None, numeric_only=True, **kwargs): + pass + + @property + @abstractmethod + def dtypes(self): + pass + + @abstractmethod + def to_pandas(self): + pass + + @property + @abstractmethod + def index(self): + pass + + @abstractmethod + def copy(self): + pass + + @abstractmethod + def _to_internal_pandas(self): + pass + + @abstractmethod + def head(self, n: int = 5): + pass + + # TODO: add 'axis' parameter + def cummin(self, skipna: bool = True) -> Union["Series", "DataFrame"]: + """ + Return cumulative minimum over a DataFrame or Series axis. + + Returns a DataFrame or Series of the same size containing the cumulative minimum. + + .. note:: the current implementation of cummin uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + skipna : boolean, default True + Exclude NA/null values. If an entire row/column is NA, the result will be NA. + + Returns + ------- + DataFrame or Series + + See Also + -------- + DataFrame.min : Return the minimum over DataFrame axis. + DataFrame.cummax : Return cumulative maximum over DataFrame axis. + DataFrame.cummin : Return cumulative minimum over DataFrame axis. + DataFrame.cumsum : Return cumulative sum over DataFrame axis. + Series.min : Return the minimum over Series axis. + Series.cummax : Return cumulative maximum over Series axis. + Series.cummin : Return cumulative minimum over Series axis. + Series.cumsum : Return cumulative sum over Series axis. + Series.cumprod : Return cumulative product over Series axis. + + Examples + -------- + >>> df = pp.DataFrame([[2.0, 1.0], [3.0, None], [1.0, 0.0]], columns=list('AB')) + >>> df + A B + 0 2.0 1.0 + 1 3.0 NaN + 2 1.0 0.0 + + By default, iterates over rows and finds the minimum in each column. + + >>> df.cummin() + A B + 0 2.0 1.0 + 1 2.0 NaN + 2 1.0 0.0 + + It works identically in Series. + + >>> df.A.cummin() + 0 2.0 + 1 2.0 + 2 1.0 + Name: A, dtype: float64 + """ + return self._apply_series_op(lambda kser: kser._cum(F.min, skipna), should_resolve=True) + + # TODO: add 'axis' parameter + def cummax(self, skipna: bool = True) -> Union["Series", "DataFrame"]: + """ + Return cumulative maximum over a DataFrame or Series axis. + + Returns a DataFrame or Series of the same size containing the cumulative maximum. + + .. note:: the current implementation of cummax uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + skipna : boolean, default True + Exclude NA/null values. If an entire row/column is NA, the result will be NA. + + Returns + ------- + DataFrame or Series + + See Also + -------- + DataFrame.max : Return the maximum over DataFrame axis. + DataFrame.cummax : Return cumulative maximum over DataFrame axis. + DataFrame.cummin : Return cumulative minimum over DataFrame axis. + DataFrame.cumsum : Return cumulative sum over DataFrame axis. + DataFrame.cumprod : Return cumulative product over DataFrame axis. + Series.max : Return the maximum over Series axis. + Series.cummax : Return cumulative maximum over Series axis. + Series.cummin : Return cumulative minimum over Series axis. + Series.cumsum : Return cumulative sum over Series axis. + Series.cumprod : Return cumulative product over Series axis. + + Examples + -------- + >>> df = pp.DataFrame([[2.0, 1.0], [3.0, None], [1.0, 0.0]], columns=list('AB')) + >>> df + A B + 0 2.0 1.0 + 1 3.0 NaN + 2 1.0 0.0 + + By default, iterates over rows and finds the maximum in each column. + + >>> df.cummax() + A B + 0 2.0 1.0 + 1 3.0 NaN + 2 3.0 1.0 + + It works identically in Series. + + >>> df.B.cummax() + 0 1.0 + 1 NaN + 2 1.0 + Name: B, dtype: float64 + """ + return self._apply_series_op(lambda kser: kser._cum(F.max, skipna), should_resolve=True) + + # TODO: add 'axis' parameter + def cumsum(self, skipna: bool = True) -> Union["Series", "DataFrame"]: + """ + Return cumulative sum over a DataFrame or Series axis. + + Returns a DataFrame or Series of the same size containing the cumulative sum. + + .. note:: the current implementation of cumsum uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + skipna : boolean, default True + Exclude NA/null values. If an entire row/column is NA, the result will be NA. + + Returns + ------- + DataFrame or Series + + See Also + -------- + DataFrame.sum : Return the sum over DataFrame axis. + DataFrame.cummax : Return cumulative maximum over DataFrame axis. + DataFrame.cummin : Return cumulative minimum over DataFrame axis. + DataFrame.cumsum : Return cumulative sum over DataFrame axis. + DataFrame.cumprod : Return cumulative product over DataFrame axis. + Series.sum : Return the sum over Series axis. + Series.cummax : Return cumulative maximum over Series axis. + Series.cummin : Return cumulative minimum over Series axis. + Series.cumsum : Return cumulative sum over Series axis. + Series.cumprod : Return cumulative product over Series axis. + + Examples + -------- + >>> df = pp.DataFrame([[2.0, 1.0], [3.0, None], [1.0, 0.0]], columns=list('AB')) + >>> df + A B + 0 2.0 1.0 + 1 3.0 NaN + 2 1.0 0.0 + + By default, iterates over rows and finds the sum in each column. + + >>> df.cumsum() + A B + 0 2.0 1.0 + 1 5.0 NaN + 2 6.0 1.0 + + It works identically in Series. + + >>> df.A.cumsum() + 0 2.0 + 1 5.0 + 2 6.0 + Name: A, dtype: float64 + """ + return self._apply_series_op(lambda kser: kser._cumsum(skipna), should_resolve=True) + + # TODO: add 'axis' parameter + # TODO: use pandas_udf to support negative values and other options later + # other window except unbounded ones is supported as of Spark 3.0. + def cumprod(self, skipna: bool = True) -> Union["Series", "DataFrame"]: + """ + Return cumulative product over a DataFrame or Series axis. + + Returns a DataFrame or Series of the same size containing the cumulative product. + + .. note:: the current implementation of cumprod uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + .. note:: unlike pandas', Koalas' emulates cumulative product by ``exp(sum(log(...)))`` + trick. Therefore, it only works for positive numbers. + + Parameters + ---------- + skipna : boolean, default True + Exclude NA/null values. If an entire row/column is NA, the result will be NA. + + Returns + ------- + DataFrame or Series + + See Also + -------- + DataFrame.cummax : Return cumulative maximum over DataFrame axis. + DataFrame.cummin : Return cumulative minimum over DataFrame axis. + DataFrame.cumsum : Return cumulative sum over DataFrame axis. + DataFrame.cumprod : Return cumulative product over DataFrame axis. + Series.cummax : Return cumulative maximum over Series axis. + Series.cummin : Return cumulative minimum over Series axis. + Series.cumsum : Return cumulative sum over Series axis. + Series.cumprod : Return cumulative product over Series axis. + + Raises + ------ + Exception : If the values is equal to or lower than 0. + + Examples + -------- + >>> df = pp.DataFrame([[2.0, 1.0], [3.0, None], [4.0, 10.0]], columns=list('AB')) + >>> df + A B + 0 2.0 1.0 + 1 3.0 NaN + 2 4.0 10.0 + + By default, iterates over rows and finds the sum in each column. + + >>> df.cumprod() + A B + 0 2.0 1.0 + 1 6.0 NaN + 2 24.0 10.0 + + It works identically in Series. + + >>> df.A.cumprod() + 0 2.0 + 1 6.0 + 2 24.0 + Name: A, dtype: float64 + """ + return self._apply_series_op(lambda kser: kser._cumprod(skipna), should_resolve=True) + + # TODO: Although this has removed pandas >= 1.0.0, but we're keeping this as deprecated + # since we're using this for `DataFrame.info` internally. + # We can drop it once our minimal pandas version becomes 1.0.0. + def get_dtype_counts(self) -> pd.Series: + """ + Return counts of unique dtypes in this object. + + .. deprecated:: 0.14.0 + + Returns + ------- + dtype : pd.Series + Series with the count of columns with each dtype. + + See Also + -------- + dtypes : Return the dtypes in this object. + + Examples + -------- + >>> a = [['a', 1, 1], ['b', 2, 2], ['c', 3, 3]] + >>> df = pp.DataFrame(a, columns=['str', 'int1', 'int2']) + >>> df + str int1 int2 + 0 a 1 1 + 1 b 2 2 + 2 c 3 3 + + >>> df.get_dtype_counts().sort_values() + object 1 + int64 2 + dtype: int64 + + >>> df.str.get_dtype_counts().sort_values() + object 1 + dtype: int64 + """ + warnings.warn( + "`get_dtype_counts` has been deprecated and will be " + "removed in a future version. For DataFrames use " + "`.dtypes.value_counts()", + FutureWarning, + ) + if not isinstance(self.dtypes, Iterable): + dtypes = [self.dtypes] + else: + dtypes = list(self.dtypes) + return pd.Series(dict(Counter([d.name for d in dtypes]))) + + def pipe(self, func, *args, **kwargs) -> Any: + r""" + Apply func(self, \*args, \*\*kwargs). + + Parameters + ---------- + func : function + function to apply to the DataFrame. + ``args``, and ``kwargs`` are passed into ``func``. + Alternatively a ``(callable, data_keyword)`` tuple where + ``data_keyword`` is a string indicating the keyword of + ``callable`` that expects the DataFrames. + args : iterable, optional + positional arguments passed into ``func``. + kwargs : mapping, optional + a dictionary of keyword arguments passed into ``func``. + + Returns + ------- + object : the return type of ``func``. + + Notes + ----- + Use ``.pipe`` when chaining together functions that expect + Series, DataFrames or GroupBy objects. For example, given + + >>> df = pp.DataFrame({'category': ['A', 'A', 'B'], + ... 'col1': [1, 2, 3], + ... 'col2': [4, 5, 6]}, + ... columns=['category', 'col1', 'col2']) + >>> def keep_category_a(df): + ... return df[df['category'] == 'A'] + >>> def add_one(df, column): + ... return df.assign(col3=df[column] + 1) + >>> def multiply(df, column1, column2): + ... return df.assign(col4=df[column1] * df[column2]) + + + instead of writing + + >>> multiply(add_one(keep_category_a(df), column="col1"), column1="col2", column2="col3") + category col1 col2 col3 col4 + 0 A 1 4 2 8 + 1 A 2 5 3 15 + + + You can write + + >>> (df.pipe(keep_category_a) + ... .pipe(add_one, column="col1") + ... .pipe(multiply, column1="col2", column2="col3") + ... ) + category col1 col2 col3 col4 + 0 A 1 4 2 8 + 1 A 2 5 3 15 + + + If you have a function that takes the data as (say) the second + argument, pass a tuple indicating which keyword expects the + data. For example, suppose ``f`` takes its data as ``df``: + + >>> def multiply_2(column1, df, column2): + ... return df.assign(col4=df[column1] * df[column2]) + + + Then you can write + + >>> (df.pipe(keep_category_a) + ... .pipe(add_one, column="col1") + ... .pipe((multiply_2, 'df'), column1="col2", column2="col3") + ... ) + category col1 col2 col3 col4 + 0 A 1 4 2 8 + 1 A 2 5 3 15 + + You can use lambda as wel + + >>> pp.Series([1, 2, 3]).pipe(lambda x: (x + 1).rename("value")) + 0 2 + 1 3 + 2 4 + Name: value, dtype: int64 + """ + + if isinstance(func, tuple): + func, target = func + if target in kwargs: + raise ValueError("%s is both the pipe target and a keyword " "argument" % target) + kwargs[target] = self + return func(*args, **kwargs) + else: + return func(self, *args, **kwargs) + + def to_numpy(self) -> np.ndarray: + """ + A NumPy ndarray representing the values in this DataFrame or Series. + + .. note:: This method should only be used if the resulting NumPy ndarray is expected + to be small, as all the data is loaded into the driver's memory. + + Returns + ------- + numpy.ndarray + + Examples + -------- + >>> pp.DataFrame({"A": [1, 2], "B": [3, 4]}).to_numpy() + array([[1, 3], + [2, 4]]) + + With heterogeneous data, the lowest common type will have to be used. + + >>> pp.DataFrame({"A": [1, 2], "B": [3.0, 4.5]}).to_numpy() + array([[1. , 3. ], + [2. , 4.5]]) + + For a mix of numeric and non-numeric types, the output array will have object dtype. + + >>> df = pp.DataFrame({"A": [1, 2], "B": [3.0, 4.5], "C": pd.date_range('2000', periods=2)}) + >>> df.to_numpy() + array([[1, 3.0, Timestamp('2000-01-01 00:00:00')], + [2, 4.5, Timestamp('2000-01-02 00:00:00')]], dtype=object) + + For Series, + + >>> pp.Series(['a', 'b', 'a']).to_numpy() + array(['a', 'b', 'a'], dtype=object) + """ + return self.to_pandas().values + + @property + def values(self) -> np.ndarray: + """ + Return a Numpy representation of the DataFrame or the Series. + + .. warning:: We recommend using `DataFrame.to_numpy()` or `Series.to_numpy()` instead. + + .. note:: This method should only be used if the resulting NumPy ndarray is expected + to be small, as all the data is loaded into the driver's memory. + + Returns + ------- + numpy.ndarray + + Examples + -------- + A DataFrame where all columns are the same type (e.g., int64) results in an array of + the same type. + + >>> df = pp.DataFrame({'age': [ 3, 29], + ... 'height': [94, 170], + ... 'weight': [31, 115]}) + >>> df + age height weight + 0 3 94 31 + 1 29 170 115 + >>> df.dtypes + age int64 + height int64 + weight int64 + dtype: object + >>> df.values + array([[ 3, 94, 31], + [ 29, 170, 115]]) + + A DataFrame with mixed type columns(e.g., str/object, int64, float32) results in an ndarray + of the broadest type that accommodates these mixed types (e.g., object). + + >>> df2 = pp.DataFrame([('parrot', 24.0, 'second'), + ... ('lion', 80.5, 'first'), + ... ('monkey', np.nan, None)], + ... columns=('name', 'max_speed', 'rank')) + >>> df2.dtypes + name object + max_speed float64 + rank object + dtype: object + >>> df2.values + array([['parrot', 24.0, 'second'], + ['lion', 80.5, 'first'], + ['monkey', nan, None]], dtype=object) + + For Series, + + >>> pp.Series([1, 2, 3]).values + array([1, 2, 3]) + + >>> pp.Series(list('aabc')).values + array(['a', 'a', 'b', 'c'], dtype=object) + """ + warnings.warn("We recommend using `{}.to_numpy()` instead.".format(type(self).__name__)) + return self.to_numpy() + + def to_csv( + self, + path=None, + sep=",", + na_rep="", + columns=None, + header=True, + quotechar='"', + date_format=None, + escapechar=None, + num_files=None, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> Optional[str]: + r""" + Write object to a comma-separated values (csv) file. + + .. note:: Koalas `to_csv` writes files to a path or URI. Unlike pandas', Koalas + respects HDFS's property such as 'fs.default.name'. + + .. note:: Koalas writes CSV files into the directory, `path`, and writes + multiple `part-...` files in the directory when `path` is specified. + This behaviour was inherited from Apache Spark. The number of files can + be controlled by `num_files`. + + Parameters + ---------- + path : str, default None + File path. If None is provided the result is returned as a string. + sep : str, default ',' + String of length 1. Field delimiter for the output file. + na_rep : str, default '' + Missing data representation. + columns : sequence, optional + Columns to write. + header : bool or list of str, default True + Write out the column names. If a list of strings is given it is + assumed to be aliases for the column names. + quotechar : str, default '\"' + String of length 1. Character used to quote fields. + date_format : str, default None + Format string for datetime objects. + escapechar : str, default None + String of length 1. Character used to escape `sep` and `quotechar` + when appropriate. + num_files : the number of files to be written in `path` directory when + this is a path. + mode : str {'append', 'overwrite', 'ignore', 'error', 'errorifexists'}, + default 'overwrite'. Specifies the behavior of the save operation when the + destination exists already. + + - 'append': Append the new data to existing data. + - 'overwrite': Overwrite existing data. + - 'ignore': Silently ignore this operation if data already exists. + - 'error' or 'errorifexists': Throw an exception if data already exists. + + partition_cols : str or list of str, optional, default None + Names of partitioning columns + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + options: keyword arguments for additional options specific to PySpark. + This kwargs are specific to PySpark's CSV options to pass. Check + the options in PySpark's API documentation for spark.write.csv(...). + It has higher priority and overwrites all other options. + This parameter only works when `path` is specified. + + Returns + ------- + str or None + + See Also + -------- + read_csv + DataFrame.to_delta + DataFrame.to_table + DataFrame.to_parquet + DataFrame.to_spark_io + + Examples + -------- + >>> df = pp.DataFrame(dict( + ... date=list(pd.date_range('2012-1-1 12:00:00', periods=3, freq='M')), + ... country=['KR', 'US', 'JP'], + ... code=[1, 2 ,3]), columns=['date', 'country', 'code']) + >>> df.sort_values(by="date") # doctest: +ELLIPSIS, +NORMALIZE_WHITESPACE + date country code + ... 2012-01-31 12:00:00 KR 1 + ... 2012-02-29 12:00:00 US 2 + ... 2012-03-31 12:00:00 JP 3 + + >>> print(df.to_csv()) # doctest: +NORMALIZE_WHITESPACE + date,country,code + 2012-01-31 12:00:00,KR,1 + 2012-02-29 12:00:00,US,2 + 2012-03-31 12:00:00,JP,3 + + >>> df.cummax().to_csv(path=r'%s/to_csv/foo.csv' % path, num_files=1) + >>> pp.read_csv( + ... path=r'%s/to_csv/foo.csv' % path + ... ).sort_values(by="date") # doctest: +ELLIPSIS, +NORMALIZE_WHITESPACE + date country code + ... 2012-01-31 12:00:00 KR 1 + ... 2012-02-29 12:00:00 US 2 + ... 2012-03-31 12:00:00 US 3 + + In case of Series, + + >>> print(df.date.to_csv()) # doctest: +NORMALIZE_WHITESPACE + date + 2012-01-31 12:00:00 + 2012-02-29 12:00:00 + 2012-03-31 12:00:00 + + >>> df.date.to_csv(path=r'%s/to_csv/foo.csv' % path, num_files=1) + >>> pp.read_csv( + ... path=r'%s/to_csv/foo.csv' % path + ... ).sort_values(by="date") # doctest: +ELLIPSIS, +NORMALIZE_WHITESPACE + date + ... 2012-01-31 12:00:00 + ... 2012-02-29 12:00:00 + ... 2012-03-31 12:00:00 + + You can preserve the index in the roundtrip as below. + + >>> df.set_index("country", append=True, inplace=True) + >>> df.date.to_csv( + ... path=r'%s/to_csv/bar.csv' % path, + ... num_files=1, + ... index_col=["index1", "index2"]) + >>> pp.read_csv( + ... path=r'%s/to_csv/bar.csv' % path, index_col=["index1", "index2"] + ... ).sort_values(by="date") # doctest: +ELLIPSIS, +NORMALIZE_WHITESPACE + date + index1 index2 + ... ... 2012-01-31 12:00:00 + ... ... 2012-02-29 12:00:00 + ... ... 2012-03-31 12:00:00 + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + if path is None: + # If path is none, just collect and use pandas's to_csv. + kdf_or_ser = self + if (LooseVersion("0.24") > LooseVersion(pd.__version__)) and isinstance( + self, pp.Series + ): + # 0.23 seems not having 'columns' parameter in Series' to_csv. + return kdf_or_ser.to_pandas().to_csv( # type: ignore + None, + sep=sep, + na_rep=na_rep, + header=header, + date_format=date_format, + index=False, + ) + else: + return kdf_or_ser.to_pandas().to_csv( # type: ignore + None, + sep=sep, + na_rep=na_rep, + columns=columns, + header=header, + quotechar=quotechar, + date_format=date_format, + escapechar=escapechar, + index=False, + ) + + kdf = self + if isinstance(self, pp.Series): + kdf = self.to_frame() + + if columns is None: + column_labels = kdf._internal.column_labels + else: + column_labels = [] + for label in columns: + if not is_name_like_tuple(label): + label = (label,) + if label not in kdf._internal.column_labels: + raise KeyError(name_like_string(label)) + column_labels.append(label) + + if isinstance(index_col, str): + index_cols = [index_col] + elif index_col is None: + index_cols = [] + else: + index_cols = index_col + + if header is True and kdf._internal.column_labels_level > 1: + raise ValueError("to_csv only support one-level index column now") + elif isinstance(header, list): + sdf = kdf.to_spark(index_col) # type: ignore + sdf = sdf.select( + [scol_for(sdf, name_like_string(label)) for label in index_cols] + + [ + scol_for(sdf, str(i) if label is None else name_like_string(label)).alias( + new_name + ) + for i, (label, new_name) in enumerate(zip(column_labels, header)) + ] + ) + header = True + else: + sdf = kdf.to_spark(index_col) # type: ignore + sdf = sdf.select( + [scol_for(sdf, name_like_string(label)) for label in index_cols] + + [ + scol_for(sdf, str(i) if label is None else name_like_string(label)) + for i, label in enumerate(column_labels) + ] + ) + + if num_files is not None: + sdf = sdf.repartition(num_files) + + builder = sdf.write.mode(mode) + if partition_cols is not None: + builder.partitionBy(partition_cols) + builder._set_opts( + sep=sep, + nullValue=na_rep, + header=header, + quote=quotechar, + dateFormat=date_format, + charToEscapeQuoteEscaping=escapechar, + ) + builder.options(**options).format("csv").save(path) + return None + + def to_json( + self, + path=None, + compression="uncompressed", + num_files=None, + mode: str = "overwrite", + orient="records", + lines=True, + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> Optional[str]: + """ + Convert the object to a JSON string. + + .. note:: Koalas `to_json` writes files to a path or URI. Unlike pandas', Koalas + respects HDFS's property such as 'fs.default.name'. + + .. note:: Koalas writes JSON files into the directory, `path`, and writes + multiple `part-...` files in the directory when `path` is specified. + This behaviour was inherited from Apache Spark. The number of files can + be controlled by `num_files`. + + .. note:: output JSON format is different from pandas'. It always use `orient='records'` + for its output. This behaviour might have to change in the near future. + + Note NaN's and None will be converted to null and datetime objects + will be converted to UNIX timestamps. + + Parameters + ---------- + path : string, optional + File path. If not specified, the result is returned as + a string. + lines : bool, default True + If ‘orient’ is ‘records’ write out line delimited json format. + Will throw ValueError if incorrect ‘orient’ since others are not + list like. It should be always True for now. + orient : str, default 'records' + It should be always 'records' for now. + compression : {'gzip', 'bz2', 'xz', None} + A string representing the compression to use in the output file, + only used when the first argument is a filename. By default, the + compression is inferred from the filename. + num_files : the number of files to be written in `path` directory when + this is a path. + mode : str {'append', 'overwrite', 'ignore', 'error', 'errorifexists'}, + default 'overwrite'. Specifies the behavior of the save operation when the + destination exists already. + + - 'append': Append the new data to existing data. + - 'overwrite': Overwrite existing data. + - 'ignore': Silently ignore this operation if data already exists. + - 'error' or 'errorifexists': Throw an exception if data already exists. + + partition_cols : str or list of str, optional, default None + Names of partitioning columns + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + options: keyword arguments for additional options specific to PySpark. + It is specific to PySpark's JSON options to pass. Check + the options in PySpark's API documentation for `spark.write.json(...)`. + It has a higher priority and overwrites all other options. + This parameter only works when `path` is specified. + + Returns + -------- + str or None + + Examples + -------- + >>> df = pp.DataFrame([['a', 'b'], ['c', 'd']], + ... columns=['col 1', 'col 2']) + >>> df.to_json() + '[{"col 1":"a","col 2":"b"},{"col 1":"c","col 2":"d"}]' + + >>> df['col 1'].to_json() + '[{"col 1":"a"},{"col 1":"c"}]' + + >>> df.to_json(path=r'%s/to_json/foo.json' % path, num_files=1) + >>> pp.read_json( + ... path=r'%s/to_json/foo.json' % path + ... ).sort_values(by="col 1") + col 1 col 2 + 0 a b + 1 c d + + >>> df['col 1'].to_json(path=r'%s/to_json/foo.json' % path, num_files=1, index_col="index") + >>> pp.read_json( + ... path=r'%s/to_json/foo.json' % path, index_col="index" + ... ).sort_values(by="col 1") # doctest: +NORMALIZE_WHITESPACE + col 1 + index + 0 a + 1 c + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + if not lines: + raise NotImplementedError("lines=False is not implemented yet.") + + if orient != "records": + raise NotImplementedError("orient='records' is supported only for now.") + + if path is None: + # If path is none, just collect and use pandas's to_json. + kdf_or_ser = self + pdf = kdf_or_ser.to_pandas() # type: ignore + if isinstance(self, pp.Series): + pdf = pdf.to_frame() + # To make the format consistent and readable by `read_json`, convert it to pandas' and + # use 'records' orient for now. + return pdf.to_json(orient="records") + + kdf = self + if isinstance(self, pp.Series): + kdf = self.to_frame() + sdf = kdf.to_spark(index_col=index_col) # type: ignore + + if num_files is not None: + sdf = sdf.repartition(num_files) + + builder = sdf.write.mode(mode) + if partition_cols is not None: + builder.partitionBy(partition_cols) + builder._set_opts(compression=compression) + builder.options(**options).format("json").save(path) + return None + + def to_excel( + self, + excel_writer, + sheet_name="Sheet1", + na_rep="", + float_format=None, + columns=None, + header=True, + index=True, + index_label=None, + startrow=0, + startcol=0, + engine=None, + merge_cells=True, + encoding=None, + inf_rep="inf", + verbose=True, + freeze_panes=None, + ) -> None: + """ + Write object to an Excel sheet. + + .. note:: This method should only be used if the resulting DataFrame is expected + to be small, as all the data is loaded into the driver's memory. + + To write a single object to an Excel .xlsx file it is only necessary to + specify a target file name. To write to multiple sheets it is necessary to + create an `ExcelWriter` object with a target file name, and specify a sheet + in the file to write to. + + Multiple sheets may be written to by specifying unique `sheet_name`. + With all data written to the file it is necessary to save the changes. + Note that creating an `ExcelWriter` object with a file name that already + exists will result in the contents of the existing file being erased. + + Parameters + ---------- + excel_writer : str or ExcelWriter object + File path or existing ExcelWriter. + sheet_name : str, default 'Sheet1' + Name of sheet which will contain DataFrame. + na_rep : str, default '' + Missing data representation. + float_format : str, optional + Format string for floating point numbers. For example + ``float_format="%%.2f"`` will format 0.1234 to 0.12. + columns : sequence or list of str, optional + Columns to write. + header : bool or list of str, default True + Write out the column names. If a list of string is given it is + assumed to be aliases for the column names. + index : bool, default True + Write row names (index). + index_label : str or sequence, optional + Column label for index column(s) if desired. If not specified, and + `header` and `index` are True, then the index names are used. A + sequence should be given if the DataFrame uses MultiIndex. + startrow : int, default 0 + Upper left cell row to dump data frame. + startcol : int, default 0 + Upper left cell column to dump data frame. + engine : str, optional + Write engine to use, 'openpyxl' or 'xlsxwriter'. You can also set this + via the options ``io.excel.xlsx.writer``, ``io.excel.xls.writer``, and + ``io.excel.xlsm.writer``. + merge_cells : bool, default True + Write MultiIndex and Hierarchical Rows as merged cells. + encoding : str, optional + Encoding of the resulting excel file. Only necessary for xlwt, + other writers support unicode natively. + inf_rep : str, default 'inf' + Representation for infinity (there is no native representation for + infinity in Excel). + verbose : bool, default True + Display more information in the error logs. + freeze_panes : tuple of int (length 2), optional + Specifies the one-based bottommost row and rightmost column that + is to be frozen. + + Notes + ----- + Once a workbook has been saved it is not possible write further data + without rewriting the whole workbook. + + See Also + -------- + read_excel : Read Excel file. + + Examples + -------- + Create, write to and save a workbook: + + >>> df1 = pp.DataFrame([['a', 'b'], ['c', 'd']], + ... index=['row 1', 'row 2'], + ... columns=['col 1', 'col 2']) + >>> df1.to_excel("output.xlsx") # doctest: +SKIP + + To specify the sheet name: + + >>> df1.to_excel("output.xlsx") # doctest: +SKIP + >>> df1.to_excel("output.xlsx", + ... sheet_name='Sheet_name_1') # doctest: +SKIP + + If you wish to write to more than one sheet in the workbook, it is + necessary to specify an ExcelWriter object: + + >>> with pd.ExcelWriter('output.xlsx') as writer: # doctest: +SKIP + ... df1.to_excel(writer, sheet_name='Sheet_name_1') + ... df2.to_excel(writer, sheet_name='Sheet_name_2') + + To set the library that is used to write the Excel file, + you can pass the `engine` keyword (the default engine is + automatically chosen depending on the file extension): + + >>> df1.to_excel('output1.xlsx', engine='xlsxwriter') # doctest: +SKIP + """ + # Make sure locals() call is at the top of the function so we don't capture local variables. + args = locals() + kdf = self + + if isinstance(self, pp.DataFrame): + f = pd.DataFrame.to_excel + elif isinstance(self, pp.Series): + f = pd.Series.to_excel + else: + raise TypeError( + "Constructor expects DataFrame or Series; however, " "got [%s]" % (self,) + ) + return validate_arguments_and_invoke_function( + kdf._to_internal_pandas(), self.to_excel, f, args + ) + + def mean( + self, axis: Union[int, str] = None, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return the mean of the values. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + + Returns + ------- + mean : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.mean() + a 2.0 + b 0.2 + dtype: float64 + + >>> df.mean(axis=1) + 0 0.55 + 1 1.10 + 2 1.65 + 3 NaN + dtype: float64 + + On a Series: + + >>> df['a'].mean() + 2.0 + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + + def mean(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + spark_column = spark_column.cast(LongType()) + elif not isinstance(spark_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + return F.mean(spark_column) + + return self._reduce_for_stat_function( + mean, name="mean", axis=axis, numeric_only=numeric_only + ) + + def sum( + self, axis: Union[int, str] = None, numeric_only: bool = None, min_count: int = 0 + ) -> Union[Scalar, "Series"]: + """ + Return the sum of the values. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + min_count : int, default 0 + The required number of valid values to perform the operation. If fewer than + ``min_count`` non-NA values are present the result will be NA. + + Returns + ------- + sum : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, np.nan, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.sum() + a 6.0 + b 0.4 + dtype: float64 + + >>> df.sum(axis=1) + 0 1.1 + 1 2.0 + 2 3.3 + 3 0.0 + dtype: float64 + + >>> df.sum(min_count=3) + a 6.0 + b NaN + dtype: float64 + + >>> df.sum(axis=1, min_count=1) + 0 1.1 + 1 2.0 + 2 3.3 + 3 NaN + dtype: float64 + + On a Series: + + >>> df['a'].sum() + 6.0 + + >>> df['a'].sum(min_count=3) + 6.0 + >>> df['b'].sum(min_count=3) + nan + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + elif numeric_only is True and axis == 1: + numeric_only = None + + def sum(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + spark_column = spark_column.cast(LongType()) + elif not isinstance(spark_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + return F.coalesce(F.sum(spark_column), F.lit(0)) + + return self._reduce_for_stat_function( + sum, name="sum", axis=axis, numeric_only=numeric_only, min_count=min_count + ) + + def product( + self, axis: Union[int, str] = None, numeric_only: bool = None, min_count: int = 0 + ) -> Union[Scalar, "Series"]: + """ + Return the product of the values. + + .. note:: unlike pandas', Koalas' emulates product by ``exp(sum(log(...)))`` + trick. Therefore, it only works for positive numbers. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + min_count : int, default 0 + The required number of valid values to perform the operation. If fewer than + ``min_count`` non-NA values are present the result will be NA. + + Examples + -------- + On a DataFrame: + + Non-numeric type column is not included to the result. + + >>> kdf = pp.DataFrame({'A': [1, 2, 3, 4, 5], + ... 'B': [10, 20, 30, 40, 50], + ... 'C': ['a', 'b', 'c', 'd', 'e']}) + >>> kdf + A B C + 0 1 10 a + 1 2 20 b + 2 3 30 c + 3 4 40 d + 4 5 50 e + + >>> kdf.prod() + A 120 + B 12000000 + dtype: int64 + + If there is no numeric type columns, returns empty Series. + + >>> pp.DataFrame({"key": ['a', 'b', 'c'], "val": ['x', 'y', 'z']}).prod() + Series([], dtype: float64) + + On a Series: + + >>> pp.Series([1, 2, 3, 4, 5]).prod() + 120 + + By default, the product of an empty or all-NA Series is ``1`` + + >>> pp.Series([]).prod() + 1.0 + + This can be controlled with the ``min_count`` parameter + + >>> pp.Series([]).prod(min_count=1) + nan + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + elif numeric_only is True and axis == 1: + numeric_only = None + + def prod(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + scol = F.min(F.coalesce(spark_column, F.lit(True))).cast(LongType()) + elif isinstance(spark_type, NumericType): + num_zeros = F.sum(F.when(spark_column == 0, 1).otherwise(0)) + sign = F.when( + F.sum(F.when(spark_column < 0, 1).otherwise(0)) % 2 == 0, 1 + ).otherwise(-1) + + scol = F.when(num_zeros > 0, 0).otherwise( + sign * F.exp(F.sum(F.log(F.abs(spark_column)))) + ) + + if isinstance(spark_type, IntegralType): + scol = F.round(scol).cast(LongType()) + else: + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + + return F.coalesce(scol, F.lit(1)) + + return self._reduce_for_stat_function( + prod, name="prod", axis=axis, numeric_only=numeric_only, min_count=min_count + ) + + prod = product + + def skew( + self, axis: Union[int, str] = None, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return unbiased skew normalized by N-1. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + + Returns + ------- + skew : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.skew() # doctest: +SKIP + a 0.000000e+00 + b -3.319678e-16 + dtype: float64 + + On a Series: + + >>> df['a'].skew() + 0.0 + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + + def skew(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + spark_column = spark_column.cast(LongType()) + elif not isinstance(spark_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + return F.skewness(spark_column) + + return self._reduce_for_stat_function( + skew, name="skew", axis=axis, numeric_only=numeric_only + ) + + def kurtosis( + self, axis: Union[int, str] = None, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return unbiased kurtosis using Fisher’s definition of kurtosis (kurtosis of normal == 0.0). + Normalized by N-1. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + + Returns + ------- + kurt : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.kurtosis() + a -1.5 + b -1.5 + dtype: float64 + + On a Series: + + >>> df['a'].kurtosis() + -1.5 + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + + def kurtosis(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + spark_column = spark_column.cast(LongType()) + elif not isinstance(spark_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + return F.kurtosis(spark_column) + + return self._reduce_for_stat_function( + kurtosis, name="kurtosis", axis=axis, numeric_only=numeric_only + ) + + kurt = kurtosis + + def min( + self, axis: Union[int, str] = None, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return the minimum of the values. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + If True, include only float, int, boolean columns. This parameter is mainly for + pandas compatibility. False is supported; however, the columns should + be all numeric or all non-numeric. + + Returns + ------- + min : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.min() + a 1.0 + b 0.1 + dtype: float64 + + >>> df.min(axis=1) + 0 0.1 + 1 0.2 + 2 0.3 + 3 NaN + dtype: float64 + + On a Series: + + >>> df['a'].min() + 1.0 + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + elif numeric_only is True and axis == 1: + numeric_only = None + + return self._reduce_for_stat_function( + F.min, name="min", axis=axis, numeric_only=numeric_only + ) + + def max( + self, axis: Union[int, str] = None, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return the maximum of the values. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + If True, include only float, int, boolean columns. This parameter is mainly for + pandas compatibility. False is supported; however, the columns should + be all numeric or all non-numeric. + + Returns + ------- + max : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.max() + a 3.0 + b 0.3 + dtype: float64 + + >>> df.max(axis=1) + 0 1.0 + 1 2.0 + 2 3.0 + 3 NaN + dtype: float64 + + On a Series: + + >>> df['a'].max() + 3.0 + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + elif numeric_only is True and axis == 1: + numeric_only = None + + return self._reduce_for_stat_function( + F.max, name="max", axis=axis, numeric_only=numeric_only + ) + + def count( + self, axis: Union[int, str] = None, numeric_only: bool = False + ) -> Union[Scalar, "Series"]: + """ + Count non-NA cells for each column. + + The values `None`, `NaN` are considered NA. + + Parameters + ---------- + axis : {0 or ‘index’, 1 or ‘columns’}, default 0 + If 0 or ‘index’ counts are generated for each column. If 1 or ‘columns’ counts are + generated for each row. + numeric_only : bool, default False + If True, include only float, int, boolean columns. This parameter is mainly for + pandas compatibility. + + Returns + ------- + max : scalar for a Series, and a Series for a DataFrame. + + See Also + -------- + DataFrame.shape: Number of DataFrame rows and columns (including NA + elements). + DataFrame.isna: Boolean same-sized DataFrame showing places of NA + elements. + + Examples + -------- + Constructing DataFrame from a dictionary: + + >>> df = pp.DataFrame({"Person": + ... ["John", "Myla", "Lewis", "John", "Myla"], + ... "Age": [24., np.nan, 21., 33, 26], + ... "Single": [False, True, True, True, False]}, + ... columns=["Person", "Age", "Single"]) + >>> df + Person Age Single + 0 John 24.0 False + 1 Myla NaN True + 2 Lewis 21.0 True + 3 John 33.0 True + 4 Myla 26.0 False + + Notice the uncounted NA values: + + >>> df.count() + Person 5 + Age 4 + Single 5 + dtype: int64 + + >>> df.count(axis=1) + 0 3 + 1 2 + 2 3 + 3 3 + 4 3 + dtype: int64 + + On a Series: + + >>> df['Person'].count() + 5 + + >>> df['Age'].count() + 4 + """ + + return self._reduce_for_stat_function( + Frame._count_expr, name="count", axis=axis, numeric_only=numeric_only + ) + + def std( + self, axis: Union[int, str] = None, ddof: int = 1, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return sample standard deviation. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + ddof : int, default 1 + Delta Degrees of Freedom. The divisor used in calculations is N - ddof, + where N represents the number of elements. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + + Returns + ------- + std : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.std() + a 1.0 + b 0.1 + dtype: float64 + + >>> df.std(axis=1) + 0 0.636396 + 1 1.272792 + 2 1.909188 + 3 NaN + dtype: float64 + + >>> df.std(ddof=0) + a 0.816497 + b 0.081650 + dtype: float64 + + On a Series: + + >>> df['a'].std() + 1.0 + + >>> df['a'].std(ddof=0) + 0.816496580927726 + """ + assert ddof in (0, 1) + + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + + def std(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + spark_column = spark_column.cast(LongType()) + elif not isinstance(spark_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + if ddof == 0: + return F.stddev_pop(spark_column) + else: + return F.stddev_samp(spark_column) + + return self._reduce_for_stat_function( + std, name="std", axis=axis, numeric_only=numeric_only, ddof=ddof + ) + + def var( + self, axis: Union[int, str] = None, ddof: int = 1, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return unbiased variance. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + ddof : int, default 1 + Delta Degrees of Freedom. The divisor used in calculations is N - ddof, + where N represents the number of elements. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + + Returns + ------- + var : scalar for a Series, and a Series for a DataFrame. + + Examples + -------- + + >>> df = pp.DataFrame({'a': [1, 2, 3, np.nan], 'b': [0.1, 0.2, 0.3, np.nan]}, + ... columns=['a', 'b']) + + On a DataFrame: + + >>> df.var() + a 1.00 + b 0.01 + dtype: float64 + + >>> df.var(axis=1) + 0 0.405 + 1 1.620 + 2 3.645 + 3 NaN + dtype: float64 + + >>> df.var(ddof=0) + a 0.666667 + b 0.006667 + dtype: float64 + + On a Series: + + >>> df['a'].var() + 1.0 + + >>> df['a'].var(ddof=0) + 0.6666666666666666 + """ + assert ddof in (0, 1) + + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + + def var(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + spark_column = spark_column.cast(LongType()) + elif not isinstance(spark_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + if ddof == 0: + return F.var_pop(spark_column) + else: + return F.var_samp(spark_column) + + return self._reduce_for_stat_function( + var, name="var", axis=axis, numeric_only=numeric_only, ddof=ddof + ) + + def median( + self, axis: Union[int, str] = None, numeric_only: bool = None, accuracy: int = 10000 + ) -> Union[Scalar, "Series"]: + """ + Return the median of the values for the requested axis. + + .. note:: Unlike pandas', the median in Koalas is an approximated median based upon + approximate percentile computation because computing median across a large dataset + is extremely expensive. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + accuracy : int, optional + Default accuracy of approximation. Larger value means better accuracy. + The relative error can be deduced by 1.0 / accuracy. + + Returns + ------- + median : scalar or Series + + Examples + -------- + >>> df = pp.DataFrame({ + ... 'a': [24., 21., 25., 33., 26.], 'b': [1, 2, 3, 4, 5]}, columns=['a', 'b']) + >>> df + a b + 0 24.0 1 + 1 21.0 2 + 2 25.0 3 + 3 33.0 4 + 4 26.0 5 + + On a DataFrame: + + >>> df.median() + a 25.0 + b 3.0 + dtype: float64 + + On a Series: + + >>> df['a'].median() + 25.0 + >>> (df['b'] + 100).median() + 103.0 + + For multi-index columns, + + >>> df.columns = pd.MultiIndex.from_tuples([('x', 'a'), ('y', 'b')]) + >>> df + x y + a b + 0 24.0 1 + 1 21.0 2 + 2 25.0 3 + 3 33.0 4 + 4 26.0 5 + + On a DataFrame: + + >>> df.median() + x a 25.0 + y b 3.0 + dtype: float64 + + >>> df.median(axis=1) + 0 12.5 + 1 11.5 + 2 14.0 + 3 18.5 + 4 15.5 + dtype: float64 + + On a Series: + + >>> df[('x', 'a')].median() + 25.0 + >>> (df[('y', 'b')] + 100).median() + 103.0 + """ + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + + if not isinstance(accuracy, int): + raise ValueError( + "accuracy must be an integer; however, got [%s]" % type(accuracy).__name__ + ) + + def median(spark_column, spark_type): + if isinstance(spark_type, (BooleanType, NumericType)): + return SF.percentile_approx(spark_column.cast(DoubleType()), 0.5, accuracy) + else: + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + + return self._reduce_for_stat_function( + median, name="median", numeric_only=numeric_only, axis=axis + ) + + def sem( + self, axis: Union[int, str] = None, ddof: int = 1, numeric_only: bool = None + ) -> Union[Scalar, "Series"]: + """ + Return unbiased standard error of the mean over requested axis. + + Parameters + ---------- + axis : {index (0), columns (1)} + Axis for the function to be applied on. + ddof : int, default 1 + Delta Degrees of Freedom. The divisor used in calculations is N - ddof, + where N represents the number of elements. + numeric_only : bool, default None + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + + Returns + ------- + scalar(for Series) or Series(for DataFrame) + + Examples + -------- + >>> kdf = pp.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}) + >>> kdf + a b + 0 1 4 + 1 2 5 + 2 3 6 + + >>> kdf.sem() + a 0.57735 + b 0.57735 + dtype: float64 + + >>> kdf.sem(ddof=0) + a 0.471405 + b 0.471405 + dtype: float64 + + >>> kdf.sem(axis=1) + 0 1.5 + 1 1.5 + 2 1.5 + dtype: float64 + + Support for Series + + >>> kser = kdf.a + >>> kser + 0 1 + 1 2 + 2 3 + Name: a, dtype: int64 + + >>> kser.sem() + 0.5773502691896258 + + >>> kser.sem(ddof=0) + 0.47140452079103173 + """ + assert ddof in (0, 1) + + axis = validate_axis(axis) + + if numeric_only is None and axis == 0: + numeric_only = True + + def std(spark_column, spark_type): + if isinstance(spark_type, BooleanType): + spark_column = spark_column.cast(LongType()) + elif not isinstance(spark_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + if ddof == 0: + return F.stddev_pop(spark_column) + else: + return F.stddev_samp(spark_column) + + def sem(spark_column, spark_type): + return std(spark_column, spark_type) / pow( + Frame._count_expr(spark_column, spark_type), 0.5 + ) + + return self._reduce_for_stat_function( + sem, name="sem", numeric_only=numeric_only, axis=axis, ddof=ddof + ) + + @property + def size(self) -> int: + """ + Return an int representing the number of elements in this object. + + Return the number of rows if Series. Otherwise return the number of + rows times number of columns if DataFrame. + + Examples + -------- + >>> s = pp.Series({'a': 1, 'b': 2, 'c': None}) + >>> s.size + 3 + + >>> df = pp.DataFrame({'col1': [1, 2, None], 'col2': [3, 4, None]}) + >>> df.size + 6 + + >>> df = pp.DataFrame(index=[1, 2, None]) + >>> df.size + 0 + """ + num_columns = len(self._internal.data_spark_columns) + if num_columns == 0: + return 0 + else: + return len(self) * num_columns # type: ignore + + def abs(self) -> Union["DataFrame", "Series"]: + """ + Return a Series/DataFrame with absolute numeric value of each element. + + Returns + ------- + abs : Series/DataFrame containing the absolute value of each element. + + Examples + -------- + + Absolute numeric values in a Series. + + >>> s = pp.Series([-1.10, 2, -3.33, 4]) + >>> s.abs() + 0 1.10 + 1 2.00 + 2 3.33 + 3 4.00 + dtype: float64 + + Absolute numeric values in a DataFrame. + + >>> df = pp.DataFrame({ + ... 'a': [4, 5, 6, 7], + ... 'b': [10, 20, 30, 40], + ... 'c': [100, 50, -30, -50] + ... }, + ... columns=['a', 'b', 'c']) + >>> df.abs() + a b c + 0 4 10 100 + 1 5 20 50 + 2 6 30 30 + 3 7 40 50 + """ + + def abs(kser): + if isinstance(kser.spark.data_type, BooleanType): + return kser + elif isinstance(kser.spark.data_type, NumericType): + return kser.spark.transform(F.abs) + else: + raise TypeError( + "bad operand type for abs(): {} ({})".format( + spark_type_to_pandas_dtype(kser.spark.data_type), + kser.spark.data_type.simpleString(), + ) + ) + + return self._apply_series_op(abs) + + # TODO: by argument only support the grouping name and as_index only for now. Documentation + # should be updated when it's supported. + def groupby( + self, by, axis=0, as_index: bool = True, dropna: bool = True + ) -> Union["DataFrameGroupBy", "SeriesGroupBy"]: + """ + Group DataFrame or Series using a Series of columns. + + A groupby operation involves some combination of splitting the + object, applying a function, and combining the results. This can be + used to group large amounts of data and compute operations on these + groups. + + Parameters + ---------- + by : Series, label, or list of labels + Used to determine the groups for the groupby. + If Series is passed, the Series or dict VALUES + will be used to determine the groups. A label or list of + labels may be passed to group by the columns in ``self``. + axis : int, default 0 or 'index' + Can only be set to 0 at the moment. + as_index : bool, default True + For aggregated output, return object with group labels as the + index. Only relevant for DataFrame input. as_index=False is + effectively "SQL-style" grouped output. + dropna : bool, default True + If True, and if group keys contain NA values, + NA values together with row/column will be dropped. + If False, NA values will also be treated as the key in groups. + + Returns + ------- + DataFrameGroupBy or SeriesGroupBy + Depends on the calling object and returns groupby object that + contains information about the groups. + + See Also + -------- + koalas.groupby.GroupBy + + Examples + -------- + >>> df = pp.DataFrame({'Animal': ['Falcon', 'Falcon', + ... 'Parrot', 'Parrot'], + ... 'Max Speed': [380., 370., 24., 26.]}, + ... columns=['Animal', 'Max Speed']) + >>> df + Animal Max Speed + 0 Falcon 380.0 + 1 Falcon 370.0 + 2 Parrot 24.0 + 3 Parrot 26.0 + + >>> df.groupby(['Animal']).mean().sort_index() # doctest: +NORMALIZE_WHITESPACE + Max Speed + Animal + Falcon 375.0 + Parrot 25.0 + + >>> df.groupby(['Animal'], as_index=False).mean().sort_values('Animal') + ... # doctest: +ELLIPSIS, +NORMALIZE_WHITESPACE + Animal Max Speed + ...Falcon 375.0 + ...Parrot 25.0 + + We can also choose to include NA in group keys or not by setting dropna parameter, + the default setting is True: + + >>> l = [[1, 2, 3], [1, None, 4], [2, 1, 3], [1, 2, 2]] + >>> df = pp.DataFrame(l, columns=["a", "b", "c"]) + >>> df.groupby(by=["b"]).sum().sort_index() # doctest: +NORMALIZE_WHITESPACE + a c + b + 1.0 2 3 + 2.0 2 5 + + >>> df.groupby(by=["b"], dropna=False).sum().sort_index() # doctest: +NORMALIZE_WHITESPACE + a c + b + 1.0 2 3 + 2.0 2 5 + NaN 1 4 + """ + from pyspark.pandas.groupby import DataFrameGroupBy, SeriesGroupBy + + if isinstance(by, pp.DataFrame): + raise ValueError("Grouper for '{}' not 1-dimensional".format(type(by).__name__)) + elif isinstance(by, pp.Series): + by = [by] + elif is_name_like_tuple(by): + if isinstance(self, pp.Series): + raise KeyError(by) + by = [by] + elif is_name_like_value(by): + if isinstance(self, pp.Series): + raise KeyError(by) + by = [(by,)] + elif is_list_like(by): + new_by = [] # type: List[Union[Tuple, pp.Series]] + for key in by: + if isinstance(key, pp.DataFrame): + raise ValueError( + "Grouper for '{}' not 1-dimensional".format(type(key).__name__) + ) + elif isinstance(key, pp.Series): + new_by.append(key) + elif is_name_like_tuple(key): + if isinstance(self, pp.Series): + raise KeyError(key) + new_by.append(key) + elif is_name_like_value(key): + if isinstance(self, pp.Series): + raise KeyError(key) + new_by.append((key,)) + else: + raise ValueError( + "Grouper for '{}' not 1-dimensional".format(type(key).__name__) + ) + by = new_by + else: + raise ValueError("Grouper for '{}' not 1-dimensional".format(type(by).__name__)) + if not len(by): + raise ValueError("No group keys passed!") + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + if isinstance(self, pp.DataFrame): + return DataFrameGroupBy._build(self, by, as_index=as_index, dropna=dropna) + elif isinstance(self, pp.Series): + return SeriesGroupBy._build(self, by, as_index=as_index, dropna=dropna) + else: + raise TypeError( + "Constructor expects DataFrame or Series; however, " "got [%s]" % (self,) + ) + + def bool(self) -> bool: + """ + Return the bool of a single element in the current object. + + This must be a boolean scalar value, either True or False. Raise a ValueError if + the object does not have exactly 1 element, or that element is not boolean + + Returns + -------- + bool + + Examples + -------- + >>> pp.DataFrame({'a': [True]}).bool() + True + + >>> pp.Series([False]).bool() + False + + If there are non-boolean or multiple values exist, it raises an exception in all + cases as below. + + >>> pp.DataFrame({'a': ['a']}).bool() + Traceback (most recent call last): + ... + ValueError: bool cannot act on a non-boolean single element DataFrame + + >>> pp.DataFrame({'a': [True], 'b': [False]}).bool() # doctest: +NORMALIZE_WHITESPACE + Traceback (most recent call last): + ... + ValueError: The truth value of a DataFrame is ambiguous. Use a.empty, a.bool(), + a.item(), a.any() or a.all(). + + >>> pp.Series([1]).bool() + Traceback (most recent call last): + ... + ValueError: bool cannot act on a non-boolean single element DataFrame + """ + if isinstance(self, pp.DataFrame): + df = self + elif isinstance(self, pp.Series): + df = self.to_dataframe() + else: + raise TypeError("bool() expects DataFrame or Series; however, " "got [%s]" % (self,)) + return df.head(2)._to_internal_pandas().bool() + + def first_valid_index(self) -> Optional[Union[Scalar, Tuple[Scalar, ...]]]: + """ + Retrieves the index of the first valid value. + + Returns + ------- + scalar, tuple, or None + + Examples + -------- + + Support for DataFrame + + >>> kdf = pp.DataFrame({'a': [None, 2, 3, 2], + ... 'b': [None, 2.0, 3.0, 1.0], + ... 'c': [None, 200, 400, 200]}, + ... index=['Q', 'W', 'E', 'R']) + >>> kdf + a b c + Q NaN NaN NaN + W 2.0 2.0 200.0 + E 3.0 3.0 400.0 + R 2.0 1.0 200.0 + + >>> kdf.first_valid_index() + 'W' + + Support for MultiIndex columns + + >>> kdf.columns = pd.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> kdf + a b c + x y z + Q NaN NaN NaN + W 2.0 2.0 200.0 + E 3.0 3.0 400.0 + R 2.0 1.0 200.0 + + >>> kdf.first_valid_index() + 'W' + + Support for Series. + + >>> s = pp.Series([None, None, 3, 4, 5], index=[100, 200, 300, 400, 500]) + >>> s + 100 NaN + 200 NaN + 300 3.0 + 400 4.0 + 500 5.0 + dtype: float64 + + >>> s.first_valid_index() + 300 + + Support for MultiIndex + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 1, 2]]) + >>> s = pp.Series([None, None, None, None, 250, 1.5, 320, 1, 0.3], index=midx) + >>> s + lama speed NaN + weight NaN + length NaN + cow speed NaN + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + >>> s.first_valid_index() + ('cow', 'weight') + """ + data_spark_columns = self._internal.data_spark_columns + + if len(data_spark_columns) == 0: + return None + + cond = reduce(lambda x, y: x & y, map(lambda x: x.isNotNull(), data_spark_columns)) + + with sql_conf({SPARK_CONF_ARROW_ENABLED: False}): + # Disable Arrow to keep row ordering. + first_valid_row = ( + self._internal.spark_frame.filter(cond) + .select(self._internal.index_spark_columns) + .limit(1) + .toPandas() + ) + + # For Empty Series or DataFrame, returns None. + if len(first_valid_row) == 0: + return None + + first_valid_row = first_valid_row.iloc[0] + if len(first_valid_row) == 1: + return first_valid_row.iloc[0] + else: + return tuple(first_valid_row) + + def last_valid_index(self) -> Optional[Union[Scalar, Tuple[Scalar, ...]]]: + """ + Return index for last non-NA/null value. + + Returns + ------- + scalar, tuple, or None + + Notes + ----- + This API only works with PySpark >= 3.0. + + Examples + -------- + + Support for DataFrame + + >>> kdf = pp.DataFrame({'a': [1, 2, 3, None], + ... 'b': [1.0, 2.0, 3.0, None], + ... 'c': [100, 200, 400, None]}, + ... index=['Q', 'W', 'E', 'R']) + >>> kdf + a b c + Q 1.0 1.0 100.0 + W 2.0 2.0 200.0 + E 3.0 3.0 400.0 + R NaN NaN NaN + + >>> kdf.last_valid_index() # doctest: +SKIP + 'E' + + Support for MultiIndex columns + + >>> kdf.columns = pd.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> kdf + a b c + x y z + Q 1.0 1.0 100.0 + W 2.0 2.0 200.0 + E 3.0 3.0 400.0 + R NaN NaN NaN + + >>> kdf.last_valid_index() # doctest: +SKIP + 'E' + + Support for Series. + + >>> s = pp.Series([1, 2, 3, None, None], index=[100, 200, 300, 400, 500]) + >>> s + 100 1.0 + 200 2.0 + 300 3.0 + 400 NaN + 500 NaN + dtype: float64 + + >>> s.last_valid_index() # doctest: +SKIP + 300 + + Support for MultiIndex + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 1, 2]]) + >>> s = pp.Series([250, 1.5, 320, 1, 0.3, None, None, None, None], index=midx) + >>> s + lama speed 250.0 + weight 1.5 + length 320.0 + cow speed 1.0 + weight 0.3 + length NaN + falcon speed NaN + weight NaN + length NaN + dtype: float64 + + >>> s.last_valid_index() # doctest: +SKIP + ('cow', 'weight') + """ + if LooseVersion(pyspark.__version__) < LooseVersion("3.0"): + raise RuntimeError("last_valid_index can be used in PySpark >= 3.0") + + data_spark_columns = self._internal.data_spark_columns + + if len(data_spark_columns) == 0: + return None + + cond = reduce(lambda x, y: x & y, map(lambda x: x.isNotNull(), data_spark_columns)) + + last_valid_rows = ( + self._internal.spark_frame.filter(cond) + .select(self._internal.index_spark_columns) + .tail(1) + ) + + # For Empty Series or DataFrame, returns None. + if len(last_valid_rows) == 0: + return None + + last_valid_row = last_valid_rows[0] + + if len(last_valid_row) == 1: + return last_valid_row[0] + else: + return tuple(last_valid_row) + + # TODO: 'center', 'win_type', 'on', 'axis' parameter should be implemented. + def rolling(self, window, min_periods=None) -> Rolling: + """ + Provide rolling transformations. + + .. note:: 'min_periods' in Koalas works as a fixed window size unlike pandas. + Unlike pandas, NA is also counted as the period. This might be changed + in the near future. + + Parameters + ---------- + window : int, or offset + Size of the moving window. + This is the number of observations used for calculating the statistic. + Each window will be a fixed size. + + min_periods : int, default None + Minimum number of observations in window required to have a value + (otherwise result is NA). + For a window that is specified by an offset, min_periods will default to 1. + Otherwise, min_periods will default to the size of the window. + + Returns + ------- + a Window sub-classed for the particular operation + """ + return Rolling(self, window=window, min_periods=min_periods) + + # TODO: 'center' and 'axis' parameter should be implemented. + # 'axis' implementation, refer https://github.com/pyspark.pandas/pull/607 + def expanding(self, min_periods=1) -> Expanding: + """ + Provide expanding transformations. + + .. note:: 'min_periods' in Koalas works as a fixed window size unlike pandas. + Unlike pandas, NA is also counted as the period. This might be changed + in the near future. + + Parameters + ---------- + min_periods : int, default 1 + Minimum number of observations in window required to have a value + (otherwise result is NA). + + Returns + ------- + a Window sub-classed for the particular operation + """ + return Expanding(self, min_periods=min_periods) + + def get(self, key, default=None) -> Any: + """ + Get item from object for given key (DataFrame column, Panel slice, + etc.). Returns default value if not found. + + Parameters + ---------- + key : object + + Returns + ------- + value : same type as items contained in object + + Examples + -------- + >>> df = pp.DataFrame({'x':range(3), 'y':['a','b','b'], 'z':['a','b','b']}, + ... columns=['x', 'y', 'z'], index=[10, 20, 20]) + >>> df + x y z + 10 0 a a + 20 1 b b + 20 2 b b + + >>> df.get('x') + 10 0 + 20 1 + 20 2 + Name: x, dtype: int64 + + >>> df.get(['x', 'y']) + x y + 10 0 a + 20 1 b + 20 2 b + + >>> df.x.get(10) + 0 + + >>> df.x.get(20) + 20 1 + 20 2 + Name: x, dtype: int64 + + >>> df.x.get(15, -1) + -1 + """ + try: + return self[key] + except (KeyError, ValueError, IndexError): + return default + + def squeeze(self, axis=None) -> Union[Scalar, "DataFrame", "Series"]: + """ + Squeeze 1 dimensional axis objects into scalars. + + Series or DataFrames with a single element are squeezed to a scalar. + DataFrames with a single column or a single row are squeezed to a + Series. Otherwise the object is unchanged. + + This method is most useful when you don't know if your + object is a Series or DataFrame, but you do know it has just a single + column. In that case you can safely call `squeeze` to ensure you have a + Series. + + Parameters + ---------- + axis : {0 or 'index', 1 or 'columns', None}, default None + A specific axis to squeeze. By default, all length-1 axes are + squeezed. + + Returns + ------- + DataFrame, Series, or scalar + The projection after squeezing `axis` or all the axes. + + See Also + -------- + Series.iloc : Integer-location based indexing for selecting scalars. + DataFrame.iloc : Integer-location based indexing for selecting Series. + Series.to_frame : Inverse of DataFrame.squeeze for a + single-column DataFrame. + + Examples + -------- + >>> primes = pp.Series([2, 3, 5, 7]) + + Slicing might produce a Series with a single value: + + >>> even_primes = primes[primes % 2 == 0] + >>> even_primes + 0 2 + dtype: int64 + + >>> even_primes.squeeze() + 2 + + Squeezing objects with more than one value in every axis does nothing: + + >>> odd_primes = primes[primes % 2 == 1] + >>> odd_primes + 1 3 + 2 5 + 3 7 + dtype: int64 + + >>> odd_primes.squeeze() + 1 3 + 2 5 + 3 7 + dtype: int64 + + Squeezing is even more effective when used with DataFrames. + + >>> df = pp.DataFrame([[1, 2], [3, 4]], columns=['a', 'b']) + >>> df + a b + 0 1 2 + 1 3 4 + + Slicing a single column will produce a DataFrame with the columns + having only one value: + + >>> df_a = df[['a']] + >>> df_a + a + 0 1 + 1 3 + + So the columns can be squeezed down, resulting in a Series: + + >>> df_a.squeeze('columns') + 0 1 + 1 3 + Name: a, dtype: int64 + + Slicing a single row from a single column will produce a single + scalar DataFrame: + + >>> df_1a = df.loc[[1], ['a']] + >>> df_1a + a + 1 3 + + Squeezing the rows produces a single scalar Series: + + >>> df_1a.squeeze('rows') + a 3 + Name: 1, dtype: int64 + + Squeezing all axes will project directly into a scalar: + + >>> df_1a.squeeze() + 3 + """ + if axis is not None: + axis = "index" if axis == "rows" else axis + axis = validate_axis(axis) + + if isinstance(self, pp.DataFrame): + from pyspark.pandas.series import first_series + + is_squeezable = len(self.columns[:2]) == 1 + # If DataFrame has multiple columns, there is no change. + if not is_squeezable: + return self + series_from_column = first_series(self) + has_single_value = len(series_from_column.head(2)) == 1 + # If DataFrame has only a single value, use pandas API directly. + if has_single_value: + result = self._to_internal_pandas().squeeze(axis) + return pp.Series(result) if isinstance(result, pd.Series) else result + elif axis == 0: + return self + else: + return series_from_column + else: + # The case of Series is simple. + # If Series has only a single value, just return it as a scalar. + # Otherwise, there is no change. + self_top_two = self.head(2) + has_single_value = len(self_top_two) == 1 + return cast(Union[Scalar, pp.Series], self_top_two[0] if has_single_value else self) + + def truncate( + self, before=None, after=None, axis=None, copy=True + ) -> Union["DataFrame", "Series"]: + """ + Truncate a Series or DataFrame before and after some index value. + + This is a useful shorthand for boolean indexing based on index + values above or below certain thresholds. + + .. note:: This API is dependent on :meth:`Index.is_monotonic_increasing` + which can be expensive. + + Parameters + ---------- + before : date, str, int + Truncate all rows before this index value. + after : date, str, int + Truncate all rows after this index value. + axis : {0 or 'index', 1 or 'columns'}, optional + Axis to truncate. Truncates the index (rows) by default. + copy : bool, default is True, + Return a copy of the truncated section. + + Returns + ------- + type of caller + The truncated Series or DataFrame. + + See Also + -------- + DataFrame.loc : Select a subset of a DataFrame by label. + DataFrame.iloc : Select a subset of a DataFrame by position. + + Examples + -------- + >>> df = pp.DataFrame({'A': ['a', 'b', 'c', 'd', 'e'], + ... 'B': ['f', 'g', 'h', 'i', 'j'], + ... 'C': ['k', 'l', 'm', 'n', 'o']}, + ... index=[1, 2, 3, 4, 5]) + >>> df + A B C + 1 a f k + 2 b g l + 3 c h m + 4 d i n + 5 e j o + + >>> df.truncate(before=2, after=4) + A B C + 2 b g l + 3 c h m + 4 d i n + + The columns of a DataFrame can be truncated. + + >>> df.truncate(before="A", after="B", axis="columns") + A B + 1 a f + 2 b g + 3 c h + 4 d i + 5 e j + + For Series, only rows can be truncated. + + >>> df['A'].truncate(before=2, after=4) + 2 b + 3 c + 4 d + Name: A, dtype: object + + A Series has index that sorted integers. + + >>> s = pp.Series([10, 20, 30, 40, 50, 60, 70], + ... index=[1, 2, 3, 4, 5, 6, 7]) + >>> s + 1 10 + 2 20 + 3 30 + 4 40 + 5 50 + 6 60 + 7 70 + dtype: int64 + + >>> s.truncate(2, 5) + 2 20 + 3 30 + 4 40 + 5 50 + dtype: int64 + + A Series has index that sorted strings. + + >>> s = pp.Series([10, 20, 30, 40, 50, 60, 70], + ... index=['a', 'b', 'c', 'd', 'e', 'f', 'g']) + >>> s + a 10 + b 20 + c 30 + d 40 + e 50 + f 60 + g 70 + dtype: int64 + + >>> s.truncate('b', 'e') + b 20 + c 30 + d 40 + e 50 + dtype: int64 + """ + from pyspark.pandas.series import first_series + + axis = validate_axis(axis) + indexes = self.index + indexes_increasing = indexes.is_monotonic_increasing + if not indexes_increasing and not indexes.is_monotonic_decreasing: + raise ValueError("truncate requires a sorted index") + if (before is None) and (after is None): + return cast(Union[pp.DataFrame, pp.Series], self.copy() if copy else self) + if (before is not None and after is not None) and before > after: + raise ValueError("Truncate: %s must be after %s" % (after, before)) + + if isinstance(self, pp.Series): + if indexes_increasing: + result = first_series(self.to_frame().loc[before:after]).rename(self.name) + else: + result = first_series(self.to_frame().loc[after:before]).rename(self.name) + elif isinstance(self, pp.DataFrame): + if axis == 0: + if indexes_increasing: + result = self.loc[before:after] + else: + result = self.loc[after:before] + elif axis == 1: + result = self.loc[:, before:after] + + return cast(Union[pp.DataFrame, pp.Series], result.copy() if copy else result) + + def to_markdown(self, buf=None, mode=None) -> str: + """ + Print Series or DataFrame in Markdown-friendly format. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. + + Parameters + ---------- + buf : writable buffer, defaults to sys.stdout + Where to send the output. By default, the output is printed to + sys.stdout. Pass a writable buffer if you need to further process + the output. + mode : str, optional + Mode in which file is opened. + **kwargs + These parameters will be passed to `tabulate`. + + Returns + ------- + str + Series or DataFrame in Markdown-friendly format. + + Examples + -------- + >>> kser = pp.Series(["elk", "pig", "dog", "quetzal"], name="animal") + >>> print(kser.to_markdown()) # doctest: +SKIP + | | animal | + |---:|:---------| + | 0 | elk | + | 1 | pig | + | 2 | dog | + | 3 | quetzal | + + >>> kdf = pp.DataFrame( + ... data={"animal_1": ["elk", "pig"], "animal_2": ["dog", "quetzal"]} + ... ) + >>> print(kdf.to_markdown()) # doctest: +SKIP + | | animal_1 | animal_2 | + |---:|:-----------|:-----------| + | 0 | elk | dog | + | 1 | pig | quetzal | + """ + # `to_markdown` is supported in pandas >= 1.0.0 since it's newly added in pandas 1.0.0. + if LooseVersion(pd.__version__) < LooseVersion("1.0.0"): + raise NotImplementedError( + "`to_markdown()` only supported in Koalas with pandas >= 1.0.0" + ) + # Make sure locals() call is at the top of the function so we don't capture local variables. + args = locals() + kser_or_kdf = self + internal_pandas = kser_or_kdf._to_internal_pandas() + return validate_arguments_and_invoke_function( + internal_pandas, self.to_markdown, type(internal_pandas).to_markdown, args + ) + + @abstractmethod + def fillna(self, value=None, method=None, axis=None, inplace=False, limit=None): + pass + + # TODO: add 'downcast' when value parameter exists + def bfill(self, axis=None, inplace=False, limit=None) -> Union["DataFrame", "Series"]: + """ + Synonym for `DataFrame.fillna()` or `Series.fillna()` with ``method=`bfill```. + + .. note:: the current implementation of 'bfill' uses Spark's Window + without specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + axis : {0 or `index`} + 1 and `columns` are not supported. + inplace : boolean, default False + Fill in place (do not create a new object) + limit : int, default None + If method is specified, this is the maximum number of consecutive NaN values to + forward/backward fill. In other words, if there is a gap with more than this number of + consecutive NaNs, it will only be partially filled. If method is not specified, + this is the maximum number of entries along the entire axis where NaNs will be filled. + Must be greater than 0 if not None + + Returns + ------- + DataFrame or Series + DataFrame or Series with NA entries filled. + + Examples + -------- + >>> kdf = pp.DataFrame({ + ... 'A': [None, 3, None, None], + ... 'B': [2, 4, None, 3], + ... 'C': [None, None, None, 1], + ... 'D': [0, 1, 5, 4] + ... }, + ... columns=['A', 'B', 'C', 'D']) + >>> kdf + A B C D + 0 NaN 2.0 NaN 0 + 1 3.0 4.0 NaN 1 + 2 NaN NaN NaN 5 + 3 NaN 3.0 1.0 4 + + Propagate non-null values backward. + + >>> kdf.bfill() + A B C D + 0 3.0 2.0 1.0 0 + 1 3.0 4.0 1.0 1 + 2 NaN 3.0 1.0 5 + 3 NaN 3.0 1.0 4 + + For Series + + >>> kser = pp.Series([None, None, None, 1]) + >>> kser + 0 NaN + 1 NaN + 2 NaN + 3 1.0 + dtype: float64 + + >>> kser.bfill() + 0 1.0 + 1 1.0 + 2 1.0 + 3 1.0 + dtype: float64 + """ + return self.fillna(method="bfill", axis=axis, inplace=inplace, limit=limit) + + backfill = bfill + + # TODO: add 'downcast' when value parameter exists + def ffill(self, axis=None, inplace=False, limit=None) -> Union["DataFrame", "Series"]: + """ + Synonym for `DataFrame.fillna()` or `Series.fillna()` with ``method=`ffill```. + + .. note:: the current implementation of 'ffill' uses Spark's Window + without specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + axis : {0 or `index`} + 1 and `columns` are not supported. + inplace : boolean, default False + Fill in place (do not create a new object) + limit : int, default None + If method is specified, this is the maximum number of consecutive NaN values to + forward/backward fill. In other words, if there is a gap with more than this number of + consecutive NaNs, it will only be partially filled. If method is not specified, + this is the maximum number of entries along the entire axis where NaNs will be filled. + Must be greater than 0 if not None + + Returns + ------- + DataFrame or Series + DataFrame or Series with NA entries filled. + + Examples + -------- + >>> kdf = pp.DataFrame({ + ... 'A': [None, 3, None, None], + ... 'B': [2, 4, None, 3], + ... 'C': [None, None, None, 1], + ... 'D': [0, 1, 5, 4] + ... }, + ... columns=['A', 'B', 'C', 'D']) + >>> kdf + A B C D + 0 NaN 2.0 NaN 0 + 1 3.0 4.0 NaN 1 + 2 NaN NaN NaN 5 + 3 NaN 3.0 1.0 4 + + Propagate non-null values forward. + + >>> kdf.ffill() + A B C D + 0 NaN 2.0 NaN 0 + 1 3.0 4.0 NaN 1 + 2 3.0 4.0 NaN 5 + 3 3.0 3.0 1.0 4 + + For Series + + >>> kser = pp.Series([2, 4, None, 3]) + >>> kser + 0 2.0 + 1 4.0 + 2 NaN + 3 3.0 + dtype: float64 + + >>> kser.ffill() + 0 2.0 + 1 4.0 + 2 4.0 + 3 3.0 + dtype: float64 + """ + return self.fillna(method="ffill", axis=axis, inplace=inplace, limit=limit) + + pad = ffill + + @property + def at(self) -> AtIndexer: + return AtIndexer(self) + + at.__doc__ = AtIndexer.__doc__ + + @property + def iat(self) -> iAtIndexer: + return iAtIndexer(self) + + iat.__doc__ = iAtIndexer.__doc__ + + @property + def iloc(self) -> iLocIndexer: + return iLocIndexer(self) + + iloc.__doc__ = iLocIndexer.__doc__ + + @property + def loc(self) -> LocIndexer: + return LocIndexer(self) + + loc.__doc__ = LocIndexer.__doc__ + + def __bool__(self): + raise ValueError( + "The truth value of a {0} is ambiguous. " + "Use a.empty, a.bool(), a.item(), a.any() or a.all().".format(self.__class__.__name__) + ) + + @staticmethod + def _count_expr(spark_column, spark_type): + # Special handle floating point types because Spark's count treats nan as a valid value, + # whereas pandas count doesn't include nan. + if isinstance(spark_type, (FloatType, DoubleType)): + return F.count(F.nanvl(spark_column, F.lit(None))) + else: + return F.count(spark_column) diff --git a/python/pyspark/pandas/groupby.py b/python/pyspark/pandas/groupby.py new file mode 100644 index 0000000000..c3fe2d8a65 --- /dev/null +++ b/python/pyspark/pandas/groupby.py @@ -0,0 +1,3186 @@ +# +# 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. +# + +""" +A wrapper for GroupedData to behave similar to pandas GroupBy. +""" + +from abc import ABCMeta, abstractmethod +import sys +import inspect +from collections import OrderedDict, namedtuple +from collections.abc import Callable +from distutils.version import LooseVersion +from functools import partial +from itertools import product +from typing import Any, List, Set, Tuple, Union, cast + +import pandas as pd +from pandas.api.types import is_hashable, is_list_like + +from pyspark.sql import Window, functions as F +from pyspark.sql.types import ( + FloatType, + DoubleType, + NumericType, + StructField, + StructType, + StringType, +) +from pyspark.sql.functions import PandasUDFType, pandas_udf, Column + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas.typedef import infer_return_type, DataFrameType, ScalarType, SeriesType +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.internal import ( + InternalFrame, + HIDDEN_COLUMNS, + NATURAL_ORDER_COLUMN_NAME, + SPARK_INDEX_NAME_FORMAT, + SPARK_DEFAULT_SERIES_NAME, +) +from pyspark.pandas.missing.groupby import ( + MissingPandasLikeDataFrameGroupBy, + MissingPandasLikeSeriesGroupBy, +) +from pyspark.pandas.series import Series, first_series +from pyspark.pandas.config import get_option +from pyspark.pandas.utils import ( + align_diff_frames, + is_name_like_tuple, + is_name_like_value, + name_like_string, + same_anchor, + scol_for, + verify_temp_column_name, +) +from pyspark.pandas.spark.utils import as_nullable_spark_type, force_decimal_precision_scale +from pyspark.pandas.window import RollingGroupby, ExpandingGroupby +from pyspark.pandas.exceptions import DataError +from pyspark.pandas.spark import functions as SF + +# to keep it the same as pandas +NamedAgg = namedtuple("NamedAgg", ["column", "aggfunc"]) + + +class GroupBy(object, metaclass=ABCMeta): + """ + :ivar _kdf: The parent dataframe that is used to perform the groupby + :type _kdf: DataFrame + :ivar _groupkeys: The list of keys that will be used to perform the grouping + :type _groupkeys: List[Series] + """ + + def __init__( + self, + kdf: DataFrame, + groupkeys: List[Series], + as_index: bool, + dropna: bool, + column_labels_to_exlcude: Set[Tuple], + agg_columns_selected: bool, + agg_columns: List[Series], + ): + self._kdf = kdf + self._groupkeys = groupkeys + self._as_index = as_index + self._dropna = dropna + self._column_labels_to_exlcude = column_labels_to_exlcude + self._agg_columns_selected = agg_columns_selected + self._agg_columns = agg_columns + + @property + def _groupkeys_scols(self): + return [s.spark.column for s in self._groupkeys] + + @property + def _agg_columns_scols(self): + return [s.spark.column for s in self._agg_columns] + + @abstractmethod + def _apply_series_op(self, op, should_resolve: bool = False, numeric_only: bool = False): + pass + + # TODO: Series support is not implemented yet. + # TODO: not all arguments are implemented comparing to pandas' for now. + def aggregate(self, func_or_funcs=None, *args, **kwargs) -> DataFrame: + """Aggregate using one or more operations over the specified axis. + + Parameters + ---------- + func_or_funcs : dict, str or list + a dict mapping from column name (string) to + aggregate functions (string or list of strings). + + Returns + ------- + Series or DataFrame + + The return can be: + + * Series : when DataFrame.agg is called with a single function + * DataFrame : when DataFrame.agg is called with several functions + + Return Series or DataFrame. + + Notes + ----- + `agg` is an alias for `aggregate`. Use the alias. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 1, 2, 2], + ... 'B': [1, 2, 3, 4], + ... 'C': [0.362, 0.227, 1.267, -0.562]}, + ... columns=['A', 'B', 'C']) + + >>> df + A B C + 0 1 1 0.362 + 1 1 2 0.227 + 2 2 3 1.267 + 3 2 4 -0.562 + + Different aggregations per column + + >>> aggregated = df.groupby('A').agg({'B': 'min', 'C': 'sum'}) + >>> aggregated[['B', 'C']].sort_index() # doctest: +NORMALIZE_WHITESPACE + B C + A + 1 1 0.589 + 2 3 0.705 + + >>> aggregated = df.groupby('A').agg({'B': ['min', 'max']}) + >>> aggregated.sort_index() # doctest: +NORMALIZE_WHITESPACE + B + min max + A + 1 1 2 + 2 3 4 + + >>> aggregated = df.groupby('A').agg('min') + >>> aggregated.sort_index() # doctest: +NORMALIZE_WHITESPACE + B C + A + 1 1 0.227 + 2 3 -0.562 + + >>> aggregated = df.groupby('A').agg(['min', 'max']) + >>> aggregated.sort_index() # doctest: +NORMALIZE_WHITESPACE + B C + min max min max + A + 1 1 2 0.227 0.362 + 2 3 4 -0.562 1.267 + + To control the output names with different aggregations per column, Koalas + also supports 'named aggregation' or nested renaming in .agg. It can also be + used when applying multiple aggregation functions to specific columns. + + >>> aggregated = df.groupby('A').agg(b_max=pp.NamedAgg(column='B', aggfunc='max')) + >>> aggregated.sort_index() # doctest: +NORMALIZE_WHITESPACE + b_max + A + 1 2 + 2 4 + + >>> aggregated = df.groupby('A').agg(b_max=('B', 'max'), b_min=('B', 'min')) + >>> aggregated.sort_index() # doctest: +NORMALIZE_WHITESPACE + b_max b_min + A + 1 2 1 + 2 4 3 + + >>> aggregated = df.groupby('A').agg(b_max=('B', 'max'), c_min=('C', 'min')) + >>> aggregated.sort_index() # doctest: +NORMALIZE_WHITESPACE + b_max c_min + A + 1 2 0.227 + 2 4 -0.562 + """ + # I think current implementation of func and arguments in Koalas for aggregate is different + # than pandas, later once arguments are added, this could be removed. + if func_or_funcs is None and kwargs is None: + raise ValueError("No aggregation argument or function specified.") + + relabeling = func_or_funcs is None and is_multi_agg_with_relabel(**kwargs) + if relabeling: + func_or_funcs, columns, order = normalize_keyword_aggregation(kwargs) + + if not isinstance(func_or_funcs, (str, list)): + if not isinstance(func_or_funcs, dict) or not all( + is_name_like_value(key) + and ( + isinstance(value, str) + or isinstance(value, list) + and all(isinstance(v, str) for v in value) + ) + for key, value in func_or_funcs.items() + ): + raise ValueError( + "aggs must be a dict mapping from column name " + "to aggregate functions (string or list of strings)." + ) + + else: + agg_cols = [col.name for col in self._agg_columns] + func_or_funcs = OrderedDict([(col, func_or_funcs) for col in agg_cols]) + + kdf = DataFrame( + GroupBy._spark_groupby(self._kdf, func_or_funcs, self._groupkeys) + ) # type: DataFrame + + if self._dropna: + kdf = DataFrame( + kdf._internal.with_new_sdf( + kdf._internal.spark_frame.dropna(subset=kdf._internal.index_spark_column_names) + ) + ) + + if not self._as_index: + should_drop_index = set( + i for i, gkey in enumerate(self._groupkeys) if gkey._kdf is not self._kdf + ) + if len(should_drop_index) > 0: + kdf = kdf.reset_index(level=should_drop_index, drop=True) + if len(should_drop_index) < len(self._groupkeys): + kdf = kdf.reset_index() + + if relabeling: + kdf = kdf[order] + kdf.columns = columns + return kdf + + agg = aggregate + + @staticmethod + def _spark_groupby(kdf, func, groupkeys=()): + groupkey_names = [SPARK_INDEX_NAME_FORMAT(i) for i in range(len(groupkeys))] + groupkey_scols = [s.spark.column.alias(name) for s, name in zip(groupkeys, groupkey_names)] + + multi_aggs = any(isinstance(v, list) for v in func.values()) + reordered = [] + data_columns = [] + column_labels = [] + for key, value in func.items(): + label = key if is_name_like_tuple(key) else (key,) + if len(label) != kdf._internal.column_labels_level: + raise TypeError("The length of the key must be the same as the column label level.") + for aggfunc in [value] if isinstance(value, str) else value: + column_label = tuple(list(label) + [aggfunc]) if multi_aggs else label + column_labels.append(column_label) + + data_col = name_like_string(column_label) + data_columns.append(data_col) + + col_name = kdf._internal.spark_column_name_for(label) + if aggfunc == "nunique": + reordered.append( + F.expr("count(DISTINCT `{0}`) as `{1}`".format(col_name, data_col)) + ) + + # Implement "quartiles" aggregate function for ``describe``. + elif aggfunc == "quartiles": + reordered.append( + F.expr( + "percentile_approx(`{0}`, array(0.25, 0.5, 0.75)) as `{1}`".format( + col_name, data_col + ) + ) + ) + + else: + reordered.append( + F.expr("{1}(`{0}`) as `{2}`".format(col_name, aggfunc, data_col)) + ) + + sdf = kdf._internal.spark_frame.select(groupkey_scols + kdf._internal.data_spark_columns) + sdf = sdf.groupby(*groupkey_names).agg(*reordered) + if len(groupkeys) > 0: + index_spark_column_names = groupkey_names + index_names = [kser._column_label for kser in groupkeys] + index_dtypes = [kser.dtype for kser in groupkeys] + else: + index_spark_column_names = [] + index_names = [] + index_dtypes = [] + + return InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_spark_column_names], + index_names=index_names, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + ) + + def count(self) -> Union[DataFrame, Series]: + """ + Compute count of group, excluding missing values. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 1, 2, 1, 2], + ... 'B': [np.nan, 2, 3, 4, 5], + ... 'C': [1, 2, 1, 1, 2]}, columns=['A', 'B', 'C']) + >>> df.groupby('A').count().sort_index() # doctest: +NORMALIZE_WHITESPACE + B C + A + 1 2 3 + 2 2 2 + """ + return self._reduce_for_stat_function(F.count, only_numeric=False) + + # TODO: We should fix See Also when Series implementation is finished. + def first(self) -> Union[DataFrame, Series]: + """ + Compute first of group values. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + """ + return self._reduce_for_stat_function(F.first, only_numeric=False) + + def last(self) -> Union[DataFrame, Series]: + """ + Compute last of group values. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + """ + return self._reduce_for_stat_function( + lambda col: F.last(col, ignorenulls=True), only_numeric=False + ) + + def max(self) -> Union[DataFrame, Series]: + """ + Compute max of group values. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + """ + return self._reduce_for_stat_function(F.max, only_numeric=False) + + # TODO: examples should be updated. + def mean(self) -> Union[DataFrame, Series]: + """ + Compute mean of groups, excluding missing values. + + Returns + ------- + koalas.Series or koalas.DataFrame + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 1, 2, 1, 2], + ... 'B': [np.nan, 2, 3, 4, 5], + ... 'C': [1, 2, 1, 1, 2]}, columns=['A', 'B', 'C']) + + Groupby one column and return the mean of the remaining columns in + each group. + + >>> df.groupby('A').mean().sort_index() # doctest: +NORMALIZE_WHITESPACE + B C + A + 1 3.0 1.333333 + 2 4.0 1.500000 + """ + + return self._reduce_for_stat_function(F.mean, only_numeric=True) + + def min(self) -> Union[DataFrame, Series]: + """ + Compute min of group values. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + """ + return self._reduce_for_stat_function(F.min, only_numeric=False) + + # TODO: sync the doc. + def std(self, ddof: int = 1) -> Union[DataFrame, Series]: + """ + Compute standard deviation of groups, excluding missing values. + + Parameters + ---------- + ddof : int, default 1 + Delta Degrees of Freedom. The divisor used in calculations is N - ddof, + where N represents the number of elements. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + """ + assert ddof in (0, 1) + + return self._reduce_for_stat_function( + F.stddev_pop if ddof == 0 else F.stddev_samp, only_numeric=True + ) + + def sum(self) -> Union[DataFrame, Series]: + """ + Compute sum of group values + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + """ + return self._reduce_for_stat_function(F.sum, only_numeric=True) + + # TODO: sync the doc. + def var(self, ddof: int = 1) -> Union[DataFrame, Series]: + """ + Compute variance of groups, excluding missing values. + + Parameters + ---------- + ddof : int, default 1 + Delta Degrees of Freedom. The divisor used in calculations is N - ddof, + where N represents the number of elements. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + """ + assert ddof in (0, 1) + + return self._reduce_for_stat_function( + F.var_pop if ddof == 0 else F.var_samp, only_numeric=True + ) + + # TODO: skipna should be implemented. + def all(self) -> Union[DataFrame, Series]: + """ + Returns True if all values in the group are truthful, else False. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 1, 2, 2, 3, 3, 4, 4, 5, 5], + ... 'B': [True, True, True, False, False, + ... False, None, True, None, False]}, + ... columns=['A', 'B']) + >>> df + A B + 0 1 True + 1 1 True + 2 2 True + 3 2 False + 4 3 False + 5 3 False + 6 4 None + 7 4 True + 8 5 None + 9 5 False + + >>> df.groupby('A').all().sort_index() # doctest: +NORMALIZE_WHITESPACE + B + A + 1 True + 2 False + 3 False + 4 True + 5 False + """ + return self._reduce_for_stat_function( + lambda col: F.min(F.coalesce(col.cast("boolean"), F.lit(True))), only_numeric=False + ) + + # TODO: skipna should be implemented. + def any(self) -> Union[DataFrame, Series]: + """ + Returns True if any value in the group is truthful, else False. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 1, 2, 2, 3, 3, 4, 4, 5, 5], + ... 'B': [True, True, True, False, False, + ... False, None, True, None, False]}, + ... columns=['A', 'B']) + >>> df + A B + 0 1 True + 1 1 True + 2 2 True + 3 2 False + 4 3 False + 5 3 False + 6 4 None + 7 4 True + 8 5 None + 9 5 False + + >>> df.groupby('A').any().sort_index() # doctest: +NORMALIZE_WHITESPACE + B + A + 1 True + 2 True + 3 False + 4 True + 5 False + """ + return self._reduce_for_stat_function( + lambda col: F.max(F.coalesce(col.cast("boolean"), F.lit(False))), only_numeric=False + ) + + # TODO: groupby multiply columns should be implemented. + def size(self) -> Series: + """ + Compute group sizes. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2, 2, 3, 3, 3], + ... 'B': [1, 1, 2, 3, 3, 3]}, + ... columns=['A', 'B']) + >>> df + A B + 0 1 1 + 1 2 1 + 2 2 2 + 3 3 3 + 4 3 3 + 5 3 3 + + >>> df.groupby('A').size().sort_index() + A + 1 1 + 2 2 + 3 3 + dtype: int64 + + >>> df.groupby(['A', 'B']).size().sort_index() + A B + 1 1 1 + 2 1 1 + 2 1 + 3 3 3 + dtype: int64 + + For Series, + + >>> df.B.groupby(df.A).size().sort_index() + A + 1 1 + 2 2 + 3 3 + Name: B, dtype: int64 + + >>> df.groupby(df.A).B.size().sort_index() + A + 1 1 + 2 2 + 3 3 + Name: B, dtype: int64 + """ + groupkeys = self._groupkeys + groupkey_names = [SPARK_INDEX_NAME_FORMAT(i) for i in range(len(groupkeys))] + groupkey_scols = [s.spark.column.alias(name) for s, name in zip(groupkeys, groupkey_names)] + sdf = self._kdf._internal.spark_frame.select( + groupkey_scols + self._kdf._internal.data_spark_columns + ) + sdf = sdf.groupby(*groupkey_names).count() + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in groupkey_names], + index_names=[kser._column_label for kser in groupkeys], + index_dtypes=[kser.dtype for kser in groupkeys], + column_labels=[None], + data_spark_columns=[scol_for(sdf, "count")], + ) + return first_series(DataFrame(internal)) + + def diff(self, periods=1) -> Union[DataFrame, Series]: + """ + First discrete difference of element. + + Calculates the difference of a DataFrame element compared with another element in the + DataFrame group (default is the element in the same column of the previous row). + + Parameters + ---------- + periods : int, default 1 + Periods to shift for calculating difference, accepts negative values. + + Returns + ------- + diffed : DataFrame or Series + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 2, 3, 4, 5, 6], + ... 'b': [1, 1, 2, 3, 5, 8], + ... 'c': [1, 4, 9, 16, 25, 36]}, columns=['a', 'b', 'c']) + >>> df + a b c + 0 1 1 1 + 1 2 1 4 + 2 3 2 9 + 3 4 3 16 + 4 5 5 25 + 5 6 8 36 + + >>> df.groupby(['b']).diff().sort_index() + a c + 0 NaN NaN + 1 1.0 3.0 + 2 NaN NaN + 3 NaN NaN + 4 NaN NaN + 5 NaN NaN + + Difference with previous column in a group. + + >>> df.groupby(['b'])['a'].diff().sort_index() + 0 NaN + 1 1.0 + 2 NaN + 3 NaN + 4 NaN + 5 NaN + Name: a, dtype: float64 + """ + return self._apply_series_op( + lambda sg: sg._kser._diff(periods, part_cols=sg._groupkeys_scols), should_resolve=True + ) + + def cumcount(self, ascending=True) -> Series: + """ + Number each item in each group from 0 to the length of that group - 1. + + Essentially this is equivalent to + + .. code-block:: python + + self.apply(lambda x: pd.Series(np.arange(len(x)), x.index)) + + Parameters + ---------- + ascending : bool, default True + If False, number in reverse, from length of group - 1 to 0. + + Returns + ------- + Series + Sequence number of each element within each group. + + Examples + -------- + + >>> df = pp.DataFrame([['a'], ['a'], ['a'], ['b'], ['b'], ['a']], + ... columns=['A']) + >>> df + A + 0 a + 1 a + 2 a + 3 b + 4 b + 5 a + >>> df.groupby('A').cumcount().sort_index() + 0 0 + 1 1 + 2 2 + 3 0 + 4 1 + 5 3 + dtype: int64 + >>> df.groupby('A').cumcount(ascending=False).sort_index() + 0 3 + 1 2 + 2 1 + 3 1 + 4 0 + 5 0 + dtype: int64 + """ + ret = ( + self._groupkeys[0] + .rename() + .spark.transform(lambda _: F.lit(0)) + ._cum(F.count, True, part_cols=self._groupkeys_scols, ascending=ascending) + - 1 + ) + internal = ret._internal.resolved_copy + return first_series(DataFrame(internal)) + + def cummax(self) -> Union[DataFrame, Series]: + """ + Cumulative max for each group. + + Returns + ------- + Series or DataFrame + + See Also + -------- + Series.cummax + DataFrame.cummax + + Examples + -------- + >>> df = pp.DataFrame( + ... [[1, None, 4], [1, 0.1, 3], [1, 20.0, 2], [4, 10.0, 1]], + ... columns=list('ABC')) + >>> df + A B C + 0 1 NaN 4 + 1 1 0.1 3 + 2 1 20.0 2 + 3 4 10.0 1 + + By default, iterates over rows and finds the sum in each column. + + >>> df.groupby("A").cummax().sort_index() + B C + 0 NaN 4 + 1 0.1 4 + 2 20.0 4 + 3 10.0 1 + + It works as below in Series. + + >>> df.C.groupby(df.A).cummax().sort_index() + 0 4 + 1 4 + 2 4 + 3 1 + Name: C, dtype: int64 + """ + return self._apply_series_op( + lambda sg: sg._kser._cum(F.max, True, part_cols=sg._groupkeys_scols), + should_resolve=True, + numeric_only=True, + ) + + def cummin(self) -> Union[DataFrame, Series]: + """ + Cumulative min for each group. + + Returns + ------- + Series or DataFrame + + See Also + -------- + Series.cummin + DataFrame.cummin + + Examples + -------- + >>> df = pp.DataFrame( + ... [[1, None, 4], [1, 0.1, 3], [1, 20.0, 2], [4, 10.0, 1]], + ... columns=list('ABC')) + >>> df + A B C + 0 1 NaN 4 + 1 1 0.1 3 + 2 1 20.0 2 + 3 4 10.0 1 + + By default, iterates over rows and finds the sum in each column. + + >>> df.groupby("A").cummin().sort_index() + B C + 0 NaN 4 + 1 0.1 3 + 2 0.1 2 + 3 10.0 1 + + It works as below in Series. + + >>> df.B.groupby(df.A).cummin().sort_index() + 0 NaN + 1 0.1 + 2 0.1 + 3 10.0 + Name: B, dtype: float64 + """ + return self._apply_series_op( + lambda sg: sg._kser._cum(F.min, True, part_cols=sg._groupkeys_scols), + should_resolve=True, + numeric_only=True, + ) + + def cumprod(self) -> Union[DataFrame, Series]: + """ + Cumulative product for each group. + + Returns + ------- + Series or DataFrame + + See Also + -------- + Series.cumprod + DataFrame.cumprod + + Examples + -------- + >>> df = pp.DataFrame( + ... [[1, None, 4], [1, 0.1, 3], [1, 20.0, 2], [4, 10.0, 1]], + ... columns=list('ABC')) + >>> df + A B C + 0 1 NaN 4 + 1 1 0.1 3 + 2 1 20.0 2 + 3 4 10.0 1 + + By default, iterates over rows and finds the sum in each column. + + >>> df.groupby("A").cumprod().sort_index() + B C + 0 NaN 4 + 1 0.1 12 + 2 2.0 24 + 3 10.0 1 + + It works as below in Series. + + >>> df.B.groupby(df.A).cumprod().sort_index() + 0 NaN + 1 0.1 + 2 2.0 + 3 10.0 + Name: B, dtype: float64 + """ + return self._apply_series_op( + lambda sg: sg._kser._cumprod(True, part_cols=sg._groupkeys_scols), + should_resolve=True, + numeric_only=True, + ) + + def cumsum(self) -> Union[DataFrame, Series]: + """ + Cumulative sum for each group. + + Returns + ------- + Series or DataFrame + + See Also + -------- + Series.cumsum + DataFrame.cumsum + + Examples + -------- + >>> df = pp.DataFrame( + ... [[1, None, 4], [1, 0.1, 3], [1, 20.0, 2], [4, 10.0, 1]], + ... columns=list('ABC')) + >>> df + A B C + 0 1 NaN 4 + 1 1 0.1 3 + 2 1 20.0 2 + 3 4 10.0 1 + + By default, iterates over rows and finds the sum in each column. + + >>> df.groupby("A").cumsum().sort_index() + B C + 0 NaN 4 + 1 0.1 7 + 2 20.1 9 + 3 10.0 1 + + It works as below in Series. + + >>> df.B.groupby(df.A).cumsum().sort_index() + 0 NaN + 1 0.1 + 2 20.1 + 3 10.0 + Name: B, dtype: float64 + """ + return self._apply_series_op( + lambda sg: sg._kser._cumsum(True, part_cols=sg._groupkeys_scols), + should_resolve=True, + numeric_only=True, + ) + + def apply(self, func, *args, **kwargs) -> Union[DataFrame, Series]: + """ + Apply function `func` group-wise and combine the results together. + + The function passed to `apply` must take a DataFrame as its first + argument and return a DataFrame. `apply` will + then take care of combining the results back together into a single + dataframe. `apply` is therefore a highly flexible + grouping method. + + While `apply` is a very flexible method, its downside is that + using it can be quite a bit slower than using more specific methods + like `agg` or `transform`. Koalas offers a wide range of method that will + be much faster than using `apply` for their specific purposes, so try to + use them before reaching for `apply`. + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def pandas_div(x) -> pp.DataFrame[float, float]: + ... return x[['B', 'C']] / x[['B', 'C']] + + If the return type is specified, the output column names become + `c0, c1, c2 ... cn`. These names are positionally mapped to the returned + DataFrame in ``func``. + + To specify the column names, you can assign them in a pandas friendly style as below: + + >>> def pandas_div(x) -> pp.DataFrame["a": float, "b": float]: + ... return x[['B', 'C']] / x[['B', 'C']] + + >>> pdf = pd.DataFrame({'B': [1.], 'C': [3.]}) + >>> def plus_one(x) -> pp.DataFrame[zip(pdf.columns, pdf.dtypes)]: + ... return x[['B', 'C']] / x[['B', 'C']] + + When the given function has the return type annotated, the original index of the + GroupBy object will be lost and a default index will be attached to the result. + Please be careful about configuring the default index. See also `Default Index Type + `_. + + .. note:: the dataframe within ``func`` is actually a pandas dataframe. Therefore, + any pandas APIs within this function is allowed. + + Parameters + ---------- + func : callable + A callable that takes a DataFrame as its first argument, and + returns a dataframe. + *args + Positional arguments to pass to func. + **kwargs + Keyword arguments to pass to func. + + Returns + ------- + applied : DataFrame or Series + + See Also + -------- + aggregate : Apply aggregate function to the GroupBy object. + DataFrame.apply : Apply a function to a DataFrame. + Series.apply : Apply a function to a Series. + + Examples + -------- + >>> df = pp.DataFrame({'A': 'a a b'.split(), + ... 'B': [1, 2, 3], + ... 'C': [4, 6, 5]}, columns=['A', 'B', 'C']) + >>> g = df.groupby('A') + + Notice that ``g`` has two groups, ``a`` and ``b``. + Calling `apply` in various ways, we can get different grouping results: + + Below the functions passed to `apply` takes a DataFrame as + its argument and returns a DataFrame. `apply` combines the result for + each group together into a new DataFrame: + + >>> def plus_min(x): + ... return x + x.min() + >>> g.apply(plus_min).sort_index() # doctest: +NORMALIZE_WHITESPACE + A B C + 0 aa 2 8 + 1 aa 3 10 + 2 bb 6 10 + + >>> g.apply(sum).sort_index() # doctest: +NORMALIZE_WHITESPACE + A B C + A + a aa 3 10 + b b 3 5 + + >>> g.apply(len).sort_index() # doctest: +NORMALIZE_WHITESPACE + A + a 2 + b 1 + dtype: int64 + + You can specify the type hint and prevent schema inference for better performance. + + >>> def pandas_div(x) -> pp.DataFrame[float, float]: + ... return x[['B', 'C']] / x[['B', 'C']] + >>> g.apply(pandas_div).sort_index() # doctest: +NORMALIZE_WHITESPACE + c0 c1 + 0 1.0 1.0 + 1 1.0 1.0 + 2 1.0 1.0 + + In case of Series, it works as below. + + >>> def plus_max(x) -> pp.Series[np.int]: + ... return x + x.max() + >>> df.B.groupby(df.A).apply(plus_max).sort_index() + 0 6 + 1 3 + 2 4 + Name: B, dtype: int64 + + >>> def plus_min(x): + ... return x + x.min() + >>> df.B.groupby(df.A).apply(plus_min).sort_index() + 0 2 + 1 3 + 2 6 + Name: B, dtype: int64 + + You can also return a scalar value as a aggregated value of the group: + + >>> def plus_length(x) -> np.int: + ... return len(x) + >>> df.B.groupby(df.A).apply(plus_length).sort_index() + 0 1 + 1 2 + Name: B, dtype: int64 + + The extra arguments to the function can be passed as below. + + >>> def calculation(x, y, z) -> np.int: + ... return len(x) + y * z + >>> df.B.groupby(df.A).apply(calculation, 5, z=10).sort_index() + 0 51 + 1 52 + Name: B, dtype: int64 + """ + from pandas.core.base import SelectionMixin + + if not isinstance(func, Callable): # type: ignore + raise TypeError("%s object is not callable" % type(func).__name__) + + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + should_infer_schema = return_sig is None + + is_series_groupby = isinstance(self, SeriesGroupBy) + + kdf = self._kdf + + if self._agg_columns_selected: + agg_columns = self._agg_columns + else: + agg_columns = [ + kdf._kser_for(label) + for label in kdf._internal.column_labels + if label not in self._column_labels_to_exlcude + ] + + kdf, groupkey_labels, groupkey_names = GroupBy._prepare_group_map_apply( + kdf, self._groupkeys, agg_columns + ) + + if is_series_groupby: + name = kdf.columns[-1] + pandas_apply = SelectionMixin._builtin_table.get(func, func) + else: + f = SelectionMixin._builtin_table.get(func, func) + + def pandas_apply(pdf, *a, **k): + return f(pdf.drop(groupkey_names, axis=1), *a, **k) + + should_return_series = False + + if should_infer_schema: + # Here we execute with the first 1000 to get the return type. + limit = get_option("compute.shortcut_limit") + pdf = kdf.head(limit + 1)._to_internal_pandas() + groupkeys = [ + pdf[groupkey_name].rename(kser.name) + for groupkey_name, kser in zip(groupkey_names, self._groupkeys) + ] + if is_series_groupby: + pser_or_pdf = pdf.groupby(groupkeys)[name].apply(pandas_apply, *args, **kwargs) + else: + pser_or_pdf = pdf.groupby(groupkeys).apply(pandas_apply, *args, **kwargs) + kser_or_kdf = pp.from_pandas(pser_or_pdf) + + if len(pdf) <= limit: + if isinstance(kser_or_kdf, pp.Series) and is_series_groupby: + kser_or_kdf = kser_or_kdf.rename(cast(SeriesGroupBy, self)._kser.name) + return cast(Union[Series, DataFrame], kser_or_kdf) + + if isinstance(kser_or_kdf, Series): + should_return_series = True + kdf_from_pandas = kser_or_kdf._kdf + else: + kdf_from_pandas = cast(DataFrame, kser_or_kdf) + + return_schema = force_decimal_precision_scale( + as_nullable_spark_type( + kdf_from_pandas._internal.spark_frame.drop(*HIDDEN_COLUMNS).schema + ) + ) + else: + return_type = infer_return_type(func) + if not is_series_groupby and isinstance(return_type, SeriesType): + raise TypeError( + "Series as a return type hint at frame groupby is not supported " + "currently; however got [%s]. Use DataFrame type hint instead." % return_sig + ) + + if isinstance(return_type, DataFrameType): + return_schema = cast(DataFrameType, return_type).spark_type + data_dtypes = cast(DataFrameType, return_type).dtypes + else: + should_return_series = True + return_schema = cast(Union[SeriesType, ScalarType], return_type).spark_type + if is_series_groupby: + return_schema = StructType([StructField(name, return_schema)]) + else: + return_schema = StructType( + [StructField(SPARK_DEFAULT_SERIES_NAME, return_schema)] + ) + data_dtypes = [cast(Union[SeriesType, ScalarType], return_type).dtype] + + def pandas_groupby_apply(pdf): + + if not is_series_groupby and LooseVersion(pd.__version__) < LooseVersion("0.25"): + # `groupby.apply` in pandas<0.25 runs the functions twice for the first group. + # https://github.com/pandas-dev/pandas/pull/24748 + + should_skip_first_call = True + + def wrapped_func(df, *a, **k): + nonlocal should_skip_first_call + if should_skip_first_call: + should_skip_first_call = False + if should_return_series: + return pd.Series() + else: + return pd.DataFrame() + else: + return pandas_apply(df, *a, **k) + + else: + wrapped_func = pandas_apply + + if is_series_groupby: + pdf_or_ser = pdf.groupby(groupkey_names)[name].apply(wrapped_func, *args, **kwargs) + else: + pdf_or_ser = pdf.groupby(groupkey_names).apply(wrapped_func, *args, **kwargs) + + if not isinstance(pdf_or_ser, pd.DataFrame): + return pd.DataFrame(pdf_or_ser) + else: + return pdf_or_ser + + sdf = GroupBy._spark_group_map_apply( + kdf, + pandas_groupby_apply, + [kdf._internal.spark_column_for(label) for label in groupkey_labels], + return_schema, + retain_index=should_infer_schema, + ) + + if should_infer_schema: + # If schema is inferred, we can restore indexes too. + internal = kdf_from_pandas._internal.with_new_sdf(sdf) + else: + # Otherwise, it loses index. + internal = InternalFrame( + spark_frame=sdf, index_spark_columns=None, data_dtypes=data_dtypes + ) + + if should_return_series: + kser = first_series(DataFrame(internal)) + if is_series_groupby: + kser = kser.rename(cast(SeriesGroupBy, self)._kser.name) + return kser + else: + return DataFrame(internal) + + # TODO: implement 'dropna' parameter + def filter(self, func) -> Union[DataFrame, Series]: + """ + Return a copy of a DataFrame excluding elements from groups that + do not satisfy the boolean criterion specified by func. + + Parameters + ---------- + f : function + Function to apply to each subframe. Should return True or False. + dropna : Drop groups that do not pass the filter. True by default; + if False, groups that evaluate False are filled with NaNs. + + Returns + ------- + filtered : DataFrame or Series + + Notes + ----- + Each subframe is endowed the attribute 'name' in case you need to know + which group you are working on. + + Examples + -------- + >>> df = pp.DataFrame({'A' : ['foo', 'bar', 'foo', 'bar', + ... 'foo', 'bar'], + ... 'B' : [1, 2, 3, 4, 5, 6], + ... 'C' : [2.0, 5., 8., 1., 2., 9.]}, columns=['A', 'B', 'C']) + >>> grouped = df.groupby('A') + >>> grouped.filter(lambda x: x['B'].mean() > 3.) + A B C + 1 bar 2 5.0 + 3 bar 4 1.0 + 5 bar 6 9.0 + + >>> df.B.groupby(df.A).filter(lambda x: x.mean() > 3.) + 1 2 + 3 4 + 5 6 + Name: B, dtype: int64 + """ + from pandas.core.base import SelectionMixin + + if not isinstance(func, Callable): # type: ignore + raise TypeError("%s object is not callable" % type(func).__name__) + + is_series_groupby = isinstance(self, SeriesGroupBy) + + kdf = self._kdf + + if self._agg_columns_selected: + agg_columns = self._agg_columns + else: + agg_columns = [ + kdf._kser_for(label) + for label in kdf._internal.column_labels + if label not in self._column_labels_to_exlcude + ] + + data_schema = ( + kdf[agg_columns]._internal.resolved_copy.spark_frame.drop(*HIDDEN_COLUMNS).schema + ) + + kdf, groupkey_labels, groupkey_names = GroupBy._prepare_group_map_apply( + kdf, self._groupkeys, agg_columns + ) + + if is_series_groupby: + + def pandas_filter(pdf): + return pd.DataFrame(pdf.groupby(groupkey_names)[pdf.columns[-1]].filter(func)) + + else: + f = SelectionMixin._builtin_table.get(func, func) + + def wrapped_func(pdf): + return f(pdf.drop(groupkey_names, axis=1)) + + def pandas_filter(pdf): + return pdf.groupby(groupkey_names).filter(wrapped_func).drop(groupkey_names, axis=1) + + sdf = GroupBy._spark_group_map_apply( + kdf, + pandas_filter, + [kdf._internal.spark_column_for(label) for label in groupkey_labels], + data_schema, + retain_index=True, + ) + + kdf = DataFrame(self._kdf[agg_columns]._internal.with_new_sdf(sdf)) + if is_series_groupby: + return first_series(kdf) + else: + return kdf + + @staticmethod + def _prepare_group_map_apply(kdf, groupkeys, agg_columns): + groupkey_labels = [ + verify_temp_column_name(kdf, "__groupkey_{}__".format(i)) for i in range(len(groupkeys)) + ] + kdf = kdf[[s.rename(label) for s, label in zip(groupkeys, groupkey_labels)] + agg_columns] + groupkey_names = [label if len(label) > 1 else label[0] for label in groupkey_labels] + return DataFrame(kdf._internal.resolved_copy), groupkey_labels, groupkey_names + + @staticmethod + def _spark_group_map_apply(kdf, func, groupkeys_scols, return_schema, retain_index): + output_func = GroupBy._make_pandas_df_builder_func(kdf, func, return_schema, retain_index) + grouped_map_func = pandas_udf(return_schema, PandasUDFType.GROUPED_MAP)(output_func) + sdf = kdf._internal.spark_frame.drop(*HIDDEN_COLUMNS) + return sdf.groupby(*groupkeys_scols).apply(grouped_map_func) + + @staticmethod + def _make_pandas_df_builder_func(kdf, func, return_schema, retain_index): + """ + Creates a function that can be used inside the pandas UDF. This function can construct + the same pandas DataFrame as if the Koalas DataFrame is collected to driver side. + The index, column labels, etc. are re-constructed within the function. + """ + arguments_for_restore_index = kdf._internal.arguments_for_restore_index + + def rename_output(pdf): + pdf = InternalFrame.restore_index(pdf.copy(), **arguments_for_restore_index) + + pdf = func(pdf) + + # If schema should be inferred, we don't restore index. pandas seems restoring + # the index in some cases. + # When Spark output type is specified, without executing it, we don't know + # if we should restore the index or not. For instance, see the example in + # https://github.com/pyspark.pandas/issues/628. + pdf, _, _, _, _ = InternalFrame.prepare_pandas_frame(pdf, retain_index=retain_index) + + # Just positionally map the column names to given schema's. + pdf.columns = return_schema.names + + return pdf + + return rename_output + + def rank(self, method="average", ascending=True) -> Union[DataFrame, Series]: + """ + Provide the rank of values within each group. + + Parameters + ---------- + method : {'average', 'min', 'max', 'first', 'dense'}, default 'average' + * average: average rank of group + * min: lowest rank in group + * max: highest rank in group + * first: ranks assigned in order they appear in the array + * dense: like 'min', but rank always increases by 1 between groups + ascending : boolean, default True + False for ranks by high (1) to low (N) + + Returns + ------- + DataFrame with ranking of values within each group + + Examples + -------- + + >>> df = pp.DataFrame({ + ... 'a': [1, 1, 1, 2, 2, 2, 3, 3, 3], + ... 'b': [1, 2, 2, 2, 3, 3, 3, 4, 4]}, columns=['a', 'b']) + >>> df + a b + 0 1 1 + 1 1 2 + 2 1 2 + 3 2 2 + 4 2 3 + 5 2 3 + 6 3 3 + 7 3 4 + 8 3 4 + + >>> df.groupby("a").rank().sort_index() + b + 0 1.0 + 1 2.5 + 2 2.5 + 3 1.0 + 4 2.5 + 5 2.5 + 6 1.0 + 7 2.5 + 8 2.5 + + >>> df.b.groupby(df.a).rank(method='max').sort_index() + 0 1.0 + 1 3.0 + 2 3.0 + 3 1.0 + 4 3.0 + 5 3.0 + 6 1.0 + 7 3.0 + 8 3.0 + Name: b, dtype: float64 + + """ + return self._apply_series_op( + lambda sg: sg._kser._rank(method, ascending, part_cols=sg._groupkeys_scols), + should_resolve=True, + ) + + # TODO: add axis parameter + def idxmax(self, skipna=True) -> Union[DataFrame, Series]: + """ + Return index of first occurrence of maximum over requested axis in group. + NA/null values are excluded. + + Parameters + ---------- + skipna : boolean, default True + Exclude NA/null values. If an entire row/column is NA, the result will be NA. + + See Also + -------- + Series.idxmax + DataFrame.idxmax + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 2, 2, 3], + ... 'b': [1, 2, 3, 4, 5], + ... 'c': [5, 4, 3, 2, 1]}, columns=['a', 'b', 'c']) + + >>> df.groupby(['a'])['b'].idxmax().sort_index() # doctest: +NORMALIZE_WHITESPACE + a + 1 1 + 2 3 + 3 4 + Name: b, dtype: int64 + + >>> df.groupby(['a']).idxmax().sort_index() # doctest: +NORMALIZE_WHITESPACE + b c + a + 1 1 0 + 2 3 2 + 3 4 4 + """ + if self._kdf._internal.index_level != 1: + raise ValueError("idxmax only support one-level index now") + + groupkey_names = ["__groupkey_{}__".format(i) for i in range(len(self._groupkeys))] + + sdf = self._kdf._internal.spark_frame + for s, name in zip(self._groupkeys, groupkey_names): + sdf = sdf.withColumn(name, s.spark.column) + index = self._kdf._internal.index_spark_column_names[0] + + stat_exprs = [] + for kser, c in zip(self._agg_columns, self._agg_columns_scols): + name = kser._internal.data_spark_column_names[0] + + if skipna: + order_column = Column(c._jc.desc_nulls_last()) + else: + order_column = Column(c._jc.desc_nulls_first()) + window = Window.partitionBy(groupkey_names).orderBy( + order_column, NATURAL_ORDER_COLUMN_NAME + ) + sdf = sdf.withColumn( + name, F.when(F.row_number().over(window) == 1, scol_for(sdf, index)).otherwise(None) + ) + stat_exprs.append(F.max(scol_for(sdf, name)).alias(name)) + + sdf = sdf.groupby(*groupkey_names).agg(*stat_exprs) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in groupkey_names], + index_names=[kser._column_label for kser in self._groupkeys], + index_dtypes=[kser.dtype for kser in self._groupkeys], + column_labels=[kser._column_label for kser in self._agg_columns], + data_spark_columns=[ + scol_for(sdf, kser._internal.data_spark_column_names[0]) + for kser in self._agg_columns + ], + ) + return DataFrame(internal) + + # TODO: add axis parameter + def idxmin(self, skipna=True) -> Union[DataFrame, Series]: + """ + Return index of first occurrence of minimum over requested axis in group. + NA/null values are excluded. + + Parameters + ---------- + skipna : boolean, default True + Exclude NA/null values. If an entire row/column is NA, the result will be NA. + + See Also + -------- + Series.idxmin + DataFrame.idxmin + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 2, 2, 3], + ... 'b': [1, 2, 3, 4, 5], + ... 'c': [5, 4, 3, 2, 1]}, columns=['a', 'b', 'c']) + + >>> df.groupby(['a'])['b'].idxmin().sort_index() # doctest: +NORMALIZE_WHITESPACE + a + 1 0 + 2 2 + 3 4 + Name: b, dtype: int64 + + >>> df.groupby(['a']).idxmin().sort_index() # doctest: +NORMALIZE_WHITESPACE + b c + a + 1 0 1 + 2 2 3 + 3 4 4 + """ + if self._kdf._internal.index_level != 1: + raise ValueError("idxmin only support one-level index now") + + groupkey_names = ["__groupkey_{}__".format(i) for i in range(len(self._groupkeys))] + + sdf = self._kdf._internal.spark_frame + for s, name in zip(self._groupkeys, groupkey_names): + sdf = sdf.withColumn(name, s.spark.column) + index = self._kdf._internal.index_spark_column_names[0] + + stat_exprs = [] + for kser, c in zip(self._agg_columns, self._agg_columns_scols): + name = kser._internal.data_spark_column_names[0] + + if skipna: + order_column = Column(c._jc.asc_nulls_last()) + else: + order_column = Column(c._jc.asc_nulls_first()) + window = Window.partitionBy(groupkey_names).orderBy( + order_column, NATURAL_ORDER_COLUMN_NAME + ) + sdf = sdf.withColumn( + name, F.when(F.row_number().over(window) == 1, scol_for(sdf, index)).otherwise(None) + ) + stat_exprs.append(F.max(scol_for(sdf, name)).alias(name)) + + sdf = sdf.groupby(*groupkey_names).agg(*stat_exprs) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in groupkey_names], + index_names=[kser._column_label for kser in self._groupkeys], + column_labels=[kser._column_label for kser in self._agg_columns], + data_spark_columns=[ + scol_for(sdf, kser._internal.data_spark_column_names[0]) + for kser in self._agg_columns + ], + ) + return DataFrame(internal) + + def fillna( + self, value=None, method=None, axis=None, inplace=False, limit=None + ) -> Union[DataFrame, Series]: + """Fill NA/NaN values in group. + + Parameters + ---------- + value : scalar, dict, Series + Value to use to fill holes. alternately a dict/Series of values + specifying which value to use for each column. + DataFrame is not supported. + method : {'backfill', 'bfill', 'pad', 'ffill', None}, default None + Method to use for filling holes in reindexed Series pad / ffill: propagate last valid + observation forward to next valid backfill / bfill: + use NEXT valid observation to fill gap + axis : {0 or `index`} + 1 and `columns` are not supported. + inplace : boolean, default False + Fill in place (do not create a new object) + limit : int, default None + If method is specified, this is the maximum number of consecutive NaN values to + forward/backward fill. In other words, if there is a gap with more than this number of + consecutive NaNs, it will only be partially filled. If method is not specified, + this is the maximum number of entries along the entire axis where NaNs will be filled. + Must be greater than 0 if not None + + Returns + ------- + DataFrame + DataFrame with NA entries filled. + + Examples + -------- + >>> df = pp.DataFrame({ + ... 'A': [1, 1, 2, 2], + ... 'B': [2, 4, None, 3], + ... 'C': [None, None, None, 1], + ... 'D': [0, 1, 5, 4] + ... }, + ... columns=['A', 'B', 'C', 'D']) + >>> df + A B C D + 0 1 2.0 NaN 0 + 1 1 4.0 NaN 1 + 2 2 NaN NaN 5 + 3 2 3.0 1.0 4 + + We can also propagate non-null values forward or backward in group. + + >>> df.groupby(['A'])['B'].fillna(method='ffill').sort_index() + 0 2.0 + 1 4.0 + 2 NaN + 3 3.0 + Name: B, dtype: float64 + + >>> df.groupby(['A']).fillna(method='bfill').sort_index() + B C D + 0 2.0 NaN 0 + 1 4.0 NaN 1 + 2 3.0 1.0 5 + 3 3.0 1.0 4 + """ + return self._apply_series_op( + lambda sg: sg._kser._fillna( + value=value, method=method, axis=axis, limit=limit, part_cols=sg._groupkeys_scols + ), + should_resolve=(method is not None), + ) + + def bfill(self, limit=None) -> Union[DataFrame, Series]: + """ + Synonym for `DataFrame.fillna()` with ``method=`bfill```. + + Parameters + ---------- + axis : {0 or `index`} + 1 and `columns` are not supported. + inplace : boolean, default False + Fill in place (do not create a new object) + limit : int, default None + If method is specified, this is the maximum number of consecutive NaN values to + forward/backward fill. In other words, if there is a gap with more than this number of + consecutive NaNs, it will only be partially filled. If method is not specified, + this is the maximum number of entries along the entire axis where NaNs will be filled. + Must be greater than 0 if not None + + Returns + ------- + DataFrame + DataFrame with NA entries filled. + + Examples + -------- + >>> df = pp.DataFrame({ + ... 'A': [1, 1, 2, 2], + ... 'B': [2, 4, None, 3], + ... 'C': [None, None, None, 1], + ... 'D': [0, 1, 5, 4] + ... }, + ... columns=['A', 'B', 'C', 'D']) + >>> df + A B C D + 0 1 2.0 NaN 0 + 1 1 4.0 NaN 1 + 2 2 NaN NaN 5 + 3 2 3.0 1.0 4 + + Propagate non-null values backward. + + >>> df.groupby(['A']).bfill().sort_index() + B C D + 0 2.0 NaN 0 + 1 4.0 NaN 1 + 2 3.0 1.0 5 + 3 3.0 1.0 4 + """ + return self.fillna(method="bfill", limit=limit) + + backfill = bfill + + def ffill(self, limit=None) -> Union[DataFrame, Series]: + """ + Synonym for `DataFrame.fillna()` with ``method=`ffill```. + + Parameters + ---------- + axis : {0 or `index`} + 1 and `columns` are not supported. + inplace : boolean, default False + Fill in place (do not create a new object) + limit : int, default None + If method is specified, this is the maximum number of consecutive NaN values to + forward/backward fill. In other words, if there is a gap with more than this number of + consecutive NaNs, it will only be partially filled. If method is not specified, + this is the maximum number of entries along the entire axis where NaNs will be filled. + Must be greater than 0 if not None + + Returns + ------- + DataFrame + DataFrame with NA entries filled. + + Examples + -------- + >>> df = pp.DataFrame({ + ... 'A': [1, 1, 2, 2], + ... 'B': [2, 4, None, 3], + ... 'C': [None, None, None, 1], + ... 'D': [0, 1, 5, 4] + ... }, + ... columns=['A', 'B', 'C', 'D']) + >>> df + A B C D + 0 1 2.0 NaN 0 + 1 1 4.0 NaN 1 + 2 2 NaN NaN 5 + 3 2 3.0 1.0 4 + + Propagate non-null values forward. + + >>> df.groupby(['A']).ffill().sort_index() + B C D + 0 2.0 NaN 0 + 1 4.0 NaN 1 + 2 NaN NaN 5 + 3 3.0 1.0 4 + """ + return self.fillna(method="ffill", limit=limit) + + pad = ffill + + def _limit(self, n: int, asc: bool): + """ + Private function for tail and head. + """ + kdf = self._kdf + + if self._agg_columns_selected: + agg_columns = self._agg_columns + else: + agg_columns = [ + kdf._kser_for(label) + for label in kdf._internal.column_labels + if label not in self._column_labels_to_exlcude + ] + + kdf, groupkey_labels, _ = GroupBy._prepare_group_map_apply( + kdf, self._groupkeys, agg_columns, + ) + + groupkey_scols = [kdf._internal.spark_column_for(label) for label in groupkey_labels] + + sdf = kdf._internal.spark_frame + tmp_col = verify_temp_column_name(sdf, "__row_number__") + + # This part is handled differently depending on whether it is a tail or a head. + window = ( + Window.partitionBy(groupkey_scols).orderBy(F.col(NATURAL_ORDER_COLUMN_NAME).asc()) + if asc + else Window.partitionBy(groupkey_scols).orderBy(F.col(NATURAL_ORDER_COLUMN_NAME).desc()) + ) + + sdf = ( + sdf.withColumn(tmp_col, F.row_number().over(window)) + .filter(F.col(tmp_col) <= n) + .drop(tmp_col) + ) + + internal = kdf._internal.with_new_sdf(sdf) + return DataFrame(internal).drop(groupkey_labels, axis=1) + + def head(self, n=5) -> Union[DataFrame, Series]: + """ + Return first n rows of each group. + + Returns + ------- + DataFrame or Series + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 1, 1, 2, 2, 2, 3, 3, 3], + ... 'b': [2, 3, 1, 4, 6, 9, 8, 10, 7, 5], + ... 'c': [3, 5, 2, 5, 1, 2, 6, 4, 3, 6]}, + ... columns=['a', 'b', 'c'], + ... index=[7, 2, 4, 1, 3, 4, 9, 10, 5, 6]) + >>> df + a b c + 7 1 2 3 + 2 1 3 5 + 4 1 1 2 + 1 1 4 5 + 3 2 6 1 + 4 2 9 2 + 9 2 8 6 + 10 3 10 4 + 5 3 7 3 + 6 3 5 6 + + >>> df.groupby('a').head(2).sort_index() + a b c + 2 1 3 5 + 3 2 6 1 + 4 2 9 2 + 5 3 7 3 + 7 1 2 3 + 10 3 10 4 + + >>> df.groupby('a')['b'].head(2).sort_index() + 2 3 + 3 6 + 4 9 + 5 7 + 7 2 + 10 10 + Name: b, dtype: int64 + """ + return self._limit(n, asc=True) + + def tail(self, n=5) -> Union[DataFrame, Series]: + """ + Return last n rows of each group. + + Similar to `.apply(lambda x: x.tail(n))`, but it returns a subset of rows from + the original DataFrame with original index and order preserved (`as_index` flag is ignored). + + Does not work for negative values of n. + + Returns + ------- + DataFrame or Series + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 1, 1, 2, 2, 2, 3, 3, 3], + ... 'b': [2, 3, 1, 4, 6, 9, 8, 10, 7, 5], + ... 'c': [3, 5, 2, 5, 1, 2, 6, 4, 3, 6]}, + ... columns=['a', 'b', 'c'], + ... index=[7, 2, 4, 1, 3, 4, 9, 10, 5, 6]) + >>> df + a b c + 7 1 2 3 + 2 1 3 5 + 4 1 1 2 + 1 1 4 5 + 3 2 6 1 + 4 2 9 2 + 9 2 8 6 + 10 3 10 4 + 5 3 7 3 + 6 3 5 6 + + >>> df.groupby('a').tail(2).sort_index() + a b c + 1 1 4 5 + 4 2 9 2 + 4 1 1 2 + 5 3 7 3 + 6 3 5 6 + 9 2 8 6 + + >>> df.groupby('a')['b'].tail(2).sort_index() + 1 4 + 4 9 + 4 1 + 5 7 + 6 5 + 9 8 + Name: b, dtype: int64 + """ + return self._limit(n, asc=False) + + def shift(self, periods=1, fill_value=None) -> Union[DataFrame, Series]: + """ + Shift each group by periods observations. + + Parameters + ---------- + periods : integer, default 1 + number of periods to shift + fill_value : optional + + Returns + ------- + Series or DataFrame + Object shifted within each group. + + Examples + -------- + + >>> df = pp.DataFrame({ + ... 'a': [1, 1, 1, 2, 2, 2, 3, 3, 3], + ... 'b': [1, 2, 2, 2, 3, 3, 3, 4, 4]}, columns=['a', 'b']) + >>> df + a b + 0 1 1 + 1 1 2 + 2 1 2 + 3 2 2 + 4 2 3 + 5 2 3 + 6 3 3 + 7 3 4 + 8 3 4 + + >>> df.groupby('a').shift().sort_index() # doctest: +SKIP + b + 0 NaN + 1 1.0 + 2 2.0 + 3 NaN + 4 2.0 + 5 3.0 + 6 NaN + 7 3.0 + 8 4.0 + + >>> df.groupby('a').shift(periods=-1, fill_value=0).sort_index() # doctest: +SKIP + b + 0 2 + 1 2 + 2 0 + 3 3 + 4 3 + 5 0 + 6 4 + 7 4 + 8 0 + """ + return self._apply_series_op( + lambda sg: sg._kser._shift(periods, fill_value, part_cols=sg._groupkeys_scols), + should_resolve=True, + ) + + def transform(self, func, *args, **kwargs) -> Union[DataFrame, Series]: + """ + Apply function column-by-column to the GroupBy object. + + The function passed to `transform` must take a Series as its first + argument and return a Series. The given function is executed for + each series in each grouped data. + + While `transform` is a very flexible method, its downside is that + using it can be quite a bit slower than using more specific methods + like `agg` or `transform`. Koalas offers a wide range of method that will + be much faster than using `transform` for their specific purposes, so try to + use them before reaching for `transform`. + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def convert_to_string(x) -> pp.Series[str]: + ... return x.apply("a string {}".format) + + When the given function has the return type annotated, the original index of the + GroupBy object will be lost and a default index will be attached to the result. + Please be careful about configuring the default index. See also `Default Index Type + `_. + + .. note:: the series within ``func`` is actually a pandas series. Therefore, + any pandas APIs within this function is allowed. + + + Parameters + ---------- + func : callable + A callable that takes a Series as its first argument, and + returns a Series. + *args + Positional arguments to pass to func. + **kwargs + Keyword arguments to pass to func. + + Returns + ------- + applied : DataFrame + + See Also + -------- + aggregate : Apply aggregate function to the GroupBy object. + Series.apply : Apply a function to a Series. + + Examples + -------- + + >>> df = pp.DataFrame({'A': [0, 0, 1], + ... 'B': [1, 2, 3], + ... 'C': [4, 6, 5]}, columns=['A', 'B', 'C']) + + >>> g = df.groupby('A') + + Notice that ``g`` has two groups, ``0`` and ``1``. + Calling `transform` in various ways, we can get different grouping results: + Below the functions passed to `transform` takes a Series as + its argument and returns a Series. `transform` applies the function on each series + in each grouped data, and combine them into a new DataFrame: + + >>> def convert_to_string(x) -> pp.Series[str]: + ... return x.apply("a string {}".format) + >>> g.transform(convert_to_string) # doctest: +NORMALIZE_WHITESPACE + B C + 0 a string 1 a string 4 + 1 a string 2 a string 6 + 2 a string 3 a string 5 + + >>> def plus_max(x) -> pp.Series[np.int]: + ... return x + x.max() + >>> g.transform(plus_max) # doctest: +NORMALIZE_WHITESPACE + B C + 0 3 10 + 1 4 12 + 2 6 10 + + You can omit the type hint and let Koalas infer its type. + + >>> def plus_min(x): + ... return x + x.min() + >>> g.transform(plus_min) # doctest: +NORMALIZE_WHITESPACE + B C + 0 2 8 + 1 3 10 + 2 6 10 + + In case of Series, it works as below. + + >>> df.B.groupby(df.A).transform(plus_max) + 0 3 + 1 4 + 2 6 + Name: B, dtype: int64 + + >>> (df * -1).B.groupby(df.A).transform(abs) + 0 1 + 1 2 + 2 3 + Name: B, dtype: int64 + + You can also specify extra arguments to pass to the function. + + >>> def calculation(x, y, z) -> pp.Series[np.int]: + ... return x + x.min() + y + z + >>> g.transform(calculation, 5, z=20) # doctest: +NORMALIZE_WHITESPACE + B C + 0 27 33 + 1 28 35 + 2 31 35 + """ + if not isinstance(func, Callable): # type: ignore + raise TypeError("%s object is not callable" % type(func).__name__) + + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + + kdf, groupkey_labels, groupkey_names = GroupBy._prepare_group_map_apply( + self._kdf, self._groupkeys, agg_columns=self._agg_columns + ) + + def pandas_transform(pdf): + return pdf.groupby(groupkey_names).transform(func, *args, **kwargs) + + should_infer_schema = return_sig is None + + if should_infer_schema: + # Here we execute with the first 1000 to get the return type. + # If the records were less than 1000, it uses pandas API directly for a shortcut. + limit = get_option("compute.shortcut_limit") + pdf = kdf.head(limit + 1)._to_internal_pandas() + pdf = pdf.groupby(groupkey_names).transform(func, *args, **kwargs) + kdf_from_pandas = DataFrame(pdf) # type: DataFrame + return_schema = force_decimal_precision_scale( + as_nullable_spark_type( + kdf_from_pandas._internal.spark_frame.drop(*HIDDEN_COLUMNS).schema + ) + ) + if len(pdf) <= limit: + return kdf_from_pandas + + sdf = GroupBy._spark_group_map_apply( + kdf, + pandas_transform, + [kdf._internal.spark_column_for(label) for label in groupkey_labels], + return_schema, + retain_index=True, + ) + # If schema is inferred, we can restore indexes too. + internal = kdf_from_pandas._internal.with_new_sdf(sdf) + else: + return_type = infer_return_type(func) + if not isinstance(return_type, SeriesType): + raise TypeError( + "Expected the return type of this function to be of Series type, " + "but found type {}".format(return_type) + ) + + return_schema = cast(SeriesType, return_type).spark_type + data_columns = kdf._internal.data_spark_column_names + return_schema = StructType( + [StructField(c, return_schema) for c in data_columns if c not in groupkey_names] + ) + data_dtypes = [ + cast(SeriesType, return_type).dtype for c in data_columns if c not in groupkey_names + ] + + sdf = GroupBy._spark_group_map_apply( + kdf, + pandas_transform, + [kdf._internal.spark_column_for(label) for label in groupkey_labels], + return_schema, + retain_index=False, + ) + # Otherwise, it loses index. + internal = InternalFrame( + spark_frame=sdf, index_spark_columns=None, data_dtypes=data_dtypes + ) + + return DataFrame(internal) + + def nunique(self, dropna=True) -> Union[DataFrame, Series]: + """ + Return DataFrame with number of distinct observations per group for each column. + + Parameters + ---------- + dropna : boolean, default True + Don’t include NaN in the counts. + + Returns + ------- + nunique : DataFrame or Series + + Examples + -------- + + >>> df = pp.DataFrame({'id': ['spam', 'egg', 'egg', 'spam', + ... 'ham', 'ham'], + ... 'value1': [1, 5, 5, 2, 5, 5], + ... 'value2': list('abbaxy')}, columns=['id', 'value1', 'value2']) + >>> df + id value1 value2 + 0 spam 1 a + 1 egg 5 b + 2 egg 5 b + 3 spam 2 a + 4 ham 5 x + 5 ham 5 y + + >>> df.groupby('id').nunique().sort_index() # doctest: +SKIP + value1 value2 + id + egg 1 1 + ham 1 2 + spam 2 1 + + >>> df.groupby('id')['value1'].nunique().sort_index() # doctest: +NORMALIZE_WHITESPACE + id + egg 1 + ham 1 + spam 2 + Name: value1, dtype: int64 + """ + if dropna: + stat_function = lambda col: F.countDistinct(col) + else: + stat_function = lambda col: ( + F.countDistinct(col) + + F.when(F.count(F.when(col.isNull(), 1).otherwise(None)) >= 1, 1).otherwise(0) + ) + + return self._reduce_for_stat_function(stat_function, only_numeric=False) + + def rolling(self, window, min_periods=None) -> RollingGroupby: + """ + Return an rolling grouper, providing rolling + functionality per group. + + .. note:: 'min_periods' in Koalas works as a fixed window size unlike pandas. + Unlike pandas, NA is also counted as the period. This might be changed + in the near future. + + Parameters + ---------- + window : int, or offset + Size of the moving window. + This is the number of observations used for calculating the statistic. + Each window will be a fixed size. + + min_periods : int, default 1 + Minimum number of observations in window required to have a value + (otherwise result is NA). + + See Also + -------- + Series.groupby + DataFrame.groupby + """ + return RollingGroupby(self, window, min_periods=min_periods) + + def expanding(self, min_periods=1) -> ExpandingGroupby: + """ + Return an expanding grouper, providing expanding + functionality per group. + + .. note:: 'min_periods' in Koalas works as a fixed window size unlike pandas. + Unlike pandas, NA is also counted as the period. This might be changed + in the near future. + + Parameters + ---------- + min_periods : int, default 1 + Minimum number of observations in window required to have a value + (otherwise result is NA). + + See Also + -------- + Series.groupby + DataFrame.groupby + """ + return ExpandingGroupby(self, min_periods=min_periods) + + def get_group(self, name) -> Union[DataFrame, Series]: + """ + Construct DataFrame from group with provided name. + + Parameters + ---------- + name : object + The name of the group to get as a DataFrame. + + Returns + ------- + group : same type as obj + + Examples + -------- + >>> kdf = pp.DataFrame([('falcon', 'bird', 389.0), + ... ('parrot', 'bird', 24.0), + ... ('lion', 'mammal', 80.5), + ... ('monkey', 'mammal', np.nan)], + ... columns=['name', 'class', 'max_speed'], + ... index=[0, 2, 3, 1]) + >>> kdf + name class max_speed + 0 falcon bird 389.0 + 2 parrot bird 24.0 + 3 lion mammal 80.5 + 1 monkey mammal NaN + + >>> kdf.groupby("class").get_group("bird").sort_index() + name class max_speed + 0 falcon bird 389.0 + 2 parrot bird 24.0 + + >>> kdf.groupby("class").get_group("mammal").sort_index() + name class max_speed + 1 monkey mammal NaN + 3 lion mammal 80.5 + """ + groupkeys = self._groupkeys + if not is_hashable(name): + raise TypeError("unhashable type: '{}'".format(type(name).__name__)) + elif len(groupkeys) > 1: + if not isinstance(name, tuple): + raise ValueError("must supply a tuple to get_group with multiple grouping keys") + if len(groupkeys) != len(name): + raise ValueError( + "must supply a same-length tuple to get_group with multiple grouping keys" + ) + if not is_list_like(name): + name = [name] + cond = F.lit(True) + for groupkey, item in zip(groupkeys, name): + scol = groupkey.spark.column + cond = cond & (scol == item) + if self._agg_columns_selected: + internal = self._kdf._internal + spark_frame = internal.spark_frame.select( + internal.index_spark_columns + self._agg_columns_scols + ).filter(cond) + + internal = internal.copy( + spark_frame=spark_frame, + index_spark_columns=[ + scol_for(spark_frame, col) for col in internal.index_spark_column_names + ], + column_labels=[s._column_label for s in self._agg_columns], + data_spark_columns=[ + scol_for(spark_frame, s._internal.data_spark_column_names[0]) + for s in self._agg_columns + ], + data_dtypes=[s.dtype for s in self._agg_columns], + ) + else: + internal = self._kdf._internal.with_filter(cond) + if internal.spark_frame.head() is None: + raise KeyError(name) + + return DataFrame(internal) + + def median(self, numeric_only=True, accuracy=10000) -> Union[DataFrame, Series]: + """ + Compute median of groups, excluding missing values. + + For multiple groupings, the result index will be a MultiIndex + + .. note:: Unlike pandas', the median in Koalas is an approximated median based upon + approximate percentile computation because computing median across a large dataset + is extremely expensive. + + Parameters + ---------- + numeric_only : bool, default True + Include only float, int, boolean columns. False is not supported. This parameter + is mainly for pandas compatibility. + + Returns + ------- + Series or DataFrame + Median of values within each group. + + Examples + -------- + >>> kdf = pp.DataFrame({'a': [1., 1., 1., 1., 2., 2., 2., 3., 3., 3.], + ... 'b': [2., 3., 1., 4., 6., 9., 8., 10., 7., 5.], + ... 'c': [3., 5., 2., 5., 1., 2., 6., 4., 3., 6.]}, + ... columns=['a', 'b', 'c'], + ... index=[7, 2, 4, 1, 3, 4, 9, 10, 5, 6]) + >>> kdf + a b c + 7 1.0 2.0 3.0 + 2 1.0 3.0 5.0 + 4 1.0 1.0 2.0 + 1 1.0 4.0 5.0 + 3 2.0 6.0 1.0 + 4 2.0 9.0 2.0 + 9 2.0 8.0 6.0 + 10 3.0 10.0 4.0 + 5 3.0 7.0 3.0 + 6 3.0 5.0 6.0 + + DataFrameGroupBy + + >>> kdf.groupby('a').median().sort_index() # doctest: +NORMALIZE_WHITESPACE + b c + a + 1.0 2.0 3.0 + 2.0 8.0 2.0 + 3.0 7.0 4.0 + + SeriesGroupBy + + >>> kdf.groupby('a')['b'].median().sort_index() + a + 1.0 2.0 + 2.0 8.0 + 3.0 7.0 + Name: b, dtype: float64 + """ + if not isinstance(accuracy, int): + raise ValueError( + "accuracy must be an integer; however, got [%s]" % type(accuracy).__name__ + ) + + stat_function = lambda col: SF.percentile_approx(col, 0.5, accuracy) + return self._reduce_for_stat_function(stat_function, only_numeric=numeric_only) + + def _reduce_for_stat_function(self, sfun, only_numeric): + agg_columns = self._agg_columns + agg_columns_scols = self._agg_columns_scols + + groupkey_names = [SPARK_INDEX_NAME_FORMAT(i) for i in range(len(self._groupkeys))] + groupkey_scols = [s.alias(name) for s, name in zip(self._groupkeys_scols, groupkey_names)] + + sdf = self._kdf._internal.spark_frame.select(groupkey_scols + agg_columns_scols) + + data_columns = [] + column_labels = [] + if len(agg_columns) > 0: + stat_exprs = [] + for kser in agg_columns: + spark_type = kser.spark.data_type + name = kser._internal.data_spark_column_names[0] + label = kser._column_label + scol = scol_for(sdf, name) + # TODO: we should have a function that takes dataframes and converts the numeric + # types. Converting the NaNs is used in a few places, it should be in utils. + # Special handle floating point types because Spark's count treats nan as a valid + # value, whereas pandas count doesn't include nan. + if isinstance(spark_type, DoubleType) or isinstance(spark_type, FloatType): + stat_exprs.append(sfun(F.nanvl(scol, F.lit(None))).alias(name)) + data_columns.append(name) + column_labels.append(label) + elif isinstance(spark_type, NumericType) or not only_numeric: + stat_exprs.append(sfun(scol).alias(name)) + data_columns.append(name) + column_labels.append(label) + sdf = sdf.groupby(*groupkey_names).agg(*stat_exprs) + else: + sdf = sdf.select(*groupkey_names).distinct() + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in groupkey_names], + index_names=[kser._column_label for kser in self._groupkeys], + index_dtypes=[kser.dtype for kser in self._groupkeys], + column_labels=column_labels, + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + column_label_names=self._kdf._internal.column_label_names, + ) + kdf = DataFrame(internal) + + if self._dropna: + kdf = DataFrame( + kdf._internal.with_new_sdf( + kdf._internal.spark_frame.dropna(subset=kdf._internal.index_spark_column_names) + ) + ) + + if not self._as_index: + should_drop_index = set( + i for i, gkey in enumerate(self._groupkeys) if gkey._kdf is not self._kdf + ) + if len(should_drop_index) > 0: + kdf = kdf.reset_index(level=should_drop_index, drop=True) + if len(should_drop_index) < len(self._groupkeys): + kdf = kdf.reset_index() + return kdf + + @staticmethod + def _resolve_grouping_from_diff_dataframes( + kdf: DataFrame, by: List[Union[Series, Tuple]] + ) -> Tuple[DataFrame, List[Series], Set[Tuple]]: + column_labels_level = kdf._internal.column_labels_level + + column_labels = [] + additional_ksers = [] + additional_column_labels = [] + tmp_column_labels = set() + for i, col_or_s in enumerate(by): + if isinstance(col_or_s, Series): + if col_or_s._kdf is kdf: + column_labels.append(col_or_s._column_label) + elif same_anchor(col_or_s, kdf): + temp_label = verify_temp_column_name(kdf, "__tmp_groupkey_{}__".format(i)) + column_labels.append(temp_label) + additional_ksers.append(col_or_s.rename(temp_label)) + additional_column_labels.append(temp_label) + else: + temp_label = verify_temp_column_name( + kdf, + tuple( + ([""] * (column_labels_level - 1)) + ["__tmp_groupkey_{}__".format(i)] + ), + ) + column_labels.append(temp_label) + tmp_column_labels.add(temp_label) + elif isinstance(col_or_s, tuple): + kser = kdf[col_or_s] + if not isinstance(kser, Series): + raise ValueError(name_like_string(col_or_s)) + column_labels.append(col_or_s) + else: + raise ValueError(col_or_s) + + kdf = DataFrame( + kdf._internal.with_new_columns( + [kdf._kser_for(label) for label in kdf._internal.column_labels] + additional_ksers + ) + ) + + def assign_columns(kdf, this_column_labels, that_column_labels): + raise NotImplementedError( + "Duplicated labels with groupby() and " + "'compute.ops_on_diff_frames' option are not supported currently " + "Please use unique labels in series and frames." + ) + + for col_or_s, label in zip(by, column_labels): + if label in tmp_column_labels: + kser = col_or_s + kdf = align_diff_frames( + assign_columns, + kdf, + kser.rename(label), + fillna=False, + how="inner", + preserve_order_column=True, + ) + + tmp_column_labels |= set(additional_column_labels) + + new_by_series = [] + for col_or_s, label in zip(by, column_labels): + if label in tmp_column_labels: + kser = col_or_s + new_by_series.append(kdf._kser_for(label).rename(kser.name)) + else: + new_by_series.append(kdf._kser_for(label)) + + return kdf, new_by_series, tmp_column_labels + + @staticmethod + def _resolve_grouping(kdf: DataFrame, by: List[Union[Series, Tuple]]) -> List[Series]: + new_by_series = [] + for col_or_s in by: + if isinstance(col_or_s, Series): + new_by_series.append(col_or_s) + elif isinstance(col_or_s, tuple): + kser = kdf[col_or_s] + if not isinstance(kser, Series): + raise ValueError(name_like_string(col_or_s)) + new_by_series.append(kser) + else: + raise ValueError(col_or_s) + return new_by_series + + +class DataFrameGroupBy(GroupBy): + @staticmethod + def _build( + kdf: DataFrame, by: List[Union[Series, Tuple]], as_index: bool, dropna: bool + ) -> "DataFrameGroupBy": + if any(isinstance(col_or_s, Series) and not same_anchor(kdf, col_or_s) for col_or_s in by): + ( + kdf, + new_by_series, + column_labels_to_exlcude, + ) = GroupBy._resolve_grouping_from_diff_dataframes(kdf, by) + else: + new_by_series = GroupBy._resolve_grouping(kdf, by) + column_labels_to_exlcude = set() + return DataFrameGroupBy( + kdf, + new_by_series, + as_index=as_index, + dropna=dropna, + column_labels_to_exlcude=column_labels_to_exlcude, + ) + + def __init__( + self, + kdf: DataFrame, + by: List[Series], + as_index: bool, + dropna: bool, + column_labels_to_exlcude: Set[Tuple], + agg_columns: List[Tuple] = None, + ): + + agg_columns_selected = agg_columns is not None + if agg_columns_selected: + for label in agg_columns: + if label in column_labels_to_exlcude: + raise KeyError(label) + else: + agg_columns = [ + label + for label in kdf._internal.column_labels + if not any(label == key._column_label and key._kdf is kdf for key in by) + and label not in column_labels_to_exlcude + ] + + super().__init__( + kdf=kdf, + groupkeys=by, + as_index=as_index, + dropna=dropna, + column_labels_to_exlcude=column_labels_to_exlcude, + agg_columns_selected=agg_columns_selected, + agg_columns=[kdf[label] for label in agg_columns], + ) + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeDataFrameGroupBy, item): + property_or_func = getattr(MissingPandasLikeDataFrameGroupBy, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + return self.__getitem__(item) + + def __getitem__(self, item): + if self._as_index and is_name_like_value(item): + return SeriesGroupBy( + self._kdf._kser_for(item if is_name_like_tuple(item) else (item,)), + self._groupkeys, + dropna=self._dropna, + ) + else: + if is_name_like_tuple(item): + item = [item] + elif is_name_like_value(item): + item = [(item,)] + else: + item = [i if is_name_like_tuple(i) else (i,) for i in item] + if not self._as_index: + groupkey_names = set(key._column_label for key in self._groupkeys) + for name in item: + if name in groupkey_names: + raise ValueError( + "cannot insert {}, already exists".format(name_like_string(name)) + ) + return DataFrameGroupBy( + self._kdf, + self._groupkeys, + as_index=self._as_index, + dropna=self._dropna, + column_labels_to_exlcude=self._column_labels_to_exlcude, + agg_columns=item, + ) + + def _apply_series_op(self, op, should_resolve: bool = False, numeric_only: bool = False): + applied = [] + for column in self._agg_columns: + applied.append(op(column.groupby(self._groupkeys))) + if numeric_only: + applied = [col for col in applied if isinstance(col.spark.data_type, NumericType)] + if not applied: + raise DataError("No numeric types to aggregate") + internal = self._kdf._internal.with_new_columns(applied, keep_order=False) + if should_resolve: + internal = internal.resolved_copy + return DataFrame(internal) + + # TODO: Implement 'percentiles', 'include', and 'exclude' arguments. + # TODO: Add ``DataFrame.select_dtypes`` to See Also when 'include' + # and 'exclude' arguments are implemented. + def describe(self) -> DataFrame: + """ + Generate descriptive statistics that summarize the central tendency, + dispersion and shape of a dataset's distribution, excluding + ``NaN`` values. + + Analyzes both numeric and object series, as well + as ``DataFrame`` column sets of mixed data types. The output + will vary depending on what is provided. Refer to the notes + below for more detail. + + .. note:: Unlike pandas, the percentiles in Koalas are based upon + approximate percentile computation because computing percentiles + across a large dataset is extremely expensive. + + Returns + ------- + DataFrame + Summary statistics of the DataFrame provided. + + See Also + -------- + DataFrame.count + DataFrame.max + DataFrame.min + DataFrame.mean + DataFrame.std + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 3], 'b': [4, 5, 6], 'c': [7, 8, 9]}) + >>> df + a b c + 0 1 4 7 + 1 1 5 8 + 2 3 6 9 + + Describing a ``DataFrame``. By default only numeric fields + are returned. + + >>> described = df.groupby('a').describe() + >>> described.sort_index() # doctest: +NORMALIZE_WHITESPACE + b c + count mean std min 25% 50% 75% max count mean std min 25% 50% 75% max + a + 1 2.0 4.5 0.707107 4.0 4.0 4.0 5.0 5.0 2.0 7.5 0.707107 7.0 7.0 7.0 8.0 8.0 + 3 1.0 6.0 NaN 6.0 6.0 6.0 6.0 6.0 1.0 9.0 NaN 9.0 9.0 9.0 9.0 9.0 + + """ + for col in self._agg_columns: + if isinstance(col.spark.data_type, StringType): + raise NotImplementedError( + "DataFrameGroupBy.describe() doesn't support for string type for now" + ) + + kdf = self.aggregate(["count", "mean", "std", "min", "quartiles", "max"]) + sdf = kdf._internal.spark_frame + agg_column_labels = [col._column_label for col in self._agg_columns] + formatted_percentiles = ["25%", "50%", "75%"] + + # Split "quartiles" columns into first, second, and third quartiles. + for label in agg_column_labels: + quartiles_col = name_like_string(tuple(list(label) + ["quartiles"])) + for i, percentile in enumerate(formatted_percentiles): + sdf = sdf.withColumn( + name_like_string(tuple(list(label) + [percentile])), + scol_for(sdf, quartiles_col)[i], + ) + sdf = sdf.drop(quartiles_col) + + # Reorder columns lexicographically by agg column followed by stats. + stats = ["count", "mean", "std", "min"] + formatted_percentiles + ["max"] + column_labels = [tuple(list(label) + [s]) for label, s in product(agg_column_labels, stats)] + data_columns = map(name_like_string, column_labels) + + # Reindex the DataFrame to reflect initial grouping and agg columns. + internal = kdf._internal.copy( + spark_frame=sdf, + column_labels=column_labels, + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + data_dtypes=None, + ) + + # Cast columns to ``"float64"`` to match `pandas.DataFrame.groupby`. + return DataFrame(internal).astype("float64") + + +class SeriesGroupBy(GroupBy): + @staticmethod + def _build( + kser: Series, by: List[Union[Series, Tuple]], as_index: bool, dropna: bool + ) -> "SeriesGroupBy": + if any(isinstance(col_or_s, Series) and not same_anchor(kser, col_or_s) for col_or_s in by): + kdf, new_by_series, _ = GroupBy._resolve_grouping_from_diff_dataframes( + kser.to_frame(), by + ) + return SeriesGroupBy( + first_series(kdf).rename(kser.name), new_by_series, as_index=as_index, dropna=dropna + ) + else: + new_by_series = GroupBy._resolve_grouping(kser._kdf, by) + return SeriesGroupBy(kser, new_by_series, as_index=as_index, dropna=dropna) + + def __init__(self, kser: Series, by: List[Series], as_index: bool = True, dropna: bool = True): + if not as_index: + raise TypeError("as_index=False only valid with DataFrame") + super().__init__( + kdf=kser._kdf, + groupkeys=by, + as_index=True, + dropna=dropna, + column_labels_to_exlcude=set(), + agg_columns_selected=True, + agg_columns=[kser], + ) + self._kser = kser + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeSeriesGroupBy, item): + property_or_func = getattr(MissingPandasLikeSeriesGroupBy, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError(item) + + def _apply_series_op(self, op, should_resolve: bool = False, numeric_only: bool = False): + if numeric_only and not isinstance(self._agg_columns[0].spark.data_type, NumericType): + raise DataError("No numeric types to aggregate") + kser = op(self) + if should_resolve: + internal = kser._internal.resolved_copy + return first_series(DataFrame(internal)) + else: + return kser + + def _reduce_for_stat_function(self, sfun, only_numeric): + return first_series(super()._reduce_for_stat_function(sfun, only_numeric)) + + def agg(self, *args, **kwargs) -> None: + return MissingPandasLikeSeriesGroupBy.agg(self, *args, **kwargs) + + def aggregate(self, *args, **kwargs) -> None: + return MissingPandasLikeSeriesGroupBy.aggregate(self, *args, **kwargs) + + def transform(self, func, *args, **kwargs) -> Series: + return first_series(super().transform(func, *args, **kwargs)).rename(self._kser.name) + + transform.__doc__ = GroupBy.transform.__doc__ + + def idxmin(self, skipna=True) -> Series: + return first_series(super().idxmin(skipna)) + + idxmin.__doc__ = GroupBy.idxmin.__doc__ + + def idxmax(self, skipna=True) -> Series: + return first_series(super().idxmax(skipna)) + + idxmax.__doc__ = GroupBy.idxmax.__doc__ + + def head(self, n=5) -> Series: + return first_series(super().head(n)).rename(self._kser.name) + + head.__doc__ = GroupBy.head.__doc__ + + def tail(self, n=5) -> Series: + return first_series(super().tail(n)).rename(self._kser.name) + + tail.__doc__ = GroupBy.tail.__doc__ + + def size(self) -> Series: + return super().size().rename(self._kser.name) + + size.__doc__ = GroupBy.size.__doc__ + + def get_group(self, name) -> Series: + return first_series(super().get_group(name)) + + get_group.__doc__ = GroupBy.get_group.__doc__ + + # TODO: add keep parameter + def nsmallest(self, n=5) -> Series: + """ + Return the first n rows ordered by columns in ascending order in group. + + Return the first n rows with the smallest values in columns, in ascending order. + The columns that are not specified are returned as well, but not used for ordering. + + Parameters + ---------- + n : int + Number of items to retrieve. + + See Also + -------- + pyspark.pandas.Series.nsmallest + pyspark.pandas.DataFrame.nsmallest + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 1, 2, 2, 2, 3, 3, 3], + ... 'b': [1, 2, 2, 2, 3, 3, 3, 4, 4]}, columns=['a', 'b']) + + >>> df.groupby(['a'])['b'].nsmallest(1).sort_index() # doctest: +NORMALIZE_WHITESPACE + a + 1 0 1 + 2 3 2 + 3 6 3 + Name: b, dtype: int64 + """ + if self._kser._internal.index_level > 1: + raise ValueError("nsmallest do not support multi-index now") + + groupkey_col_names = [SPARK_INDEX_NAME_FORMAT(i) for i in range(len(self._groupkeys))] + sdf = self._kser._internal.spark_frame.select( + [scol.alias(name) for scol, name in zip(self._groupkeys_scols, groupkey_col_names)] + + [ + scol.alias(SPARK_INDEX_NAME_FORMAT(i + len(self._groupkeys))) + for i, scol in enumerate(self._kser._internal.index_spark_columns) + ] + + [self._kser.spark.column] + + [NATURAL_ORDER_COLUMN_NAME] + ) + + window = Window.partitionBy(groupkey_col_names).orderBy( + scol_for(sdf, self._kser._internal.data_spark_column_names[0]).asc(), + NATURAL_ORDER_COLUMN_NAME, + ) + + temp_rank_column = verify_temp_column_name(sdf, "__rank__") + sdf = ( + sdf.withColumn(temp_rank_column, F.row_number().over(window)) + .filter(F.col(temp_rank_column) <= n) + .drop(temp_rank_column) + ).drop(NATURAL_ORDER_COLUMN_NAME) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=( + [scol_for(sdf, col) for col in groupkey_col_names] + + [ + scol_for(sdf, SPARK_INDEX_NAME_FORMAT(i + len(self._groupkeys))) + for i in range(self._kdf._internal.index_level) + ] + ), + index_names=( + [kser._column_label for kser in self._groupkeys] + self._kdf._internal.index_names + ), + index_dtypes=( + [kser.dtype for kser in self._groupkeys] + self._kdf._internal.index_dtypes + ), + column_labels=[self._kser._column_label], + data_spark_columns=[scol_for(sdf, self._kser._internal.data_spark_column_names[0])], + data_dtypes=[self._kser.dtype], + ) + return first_series(DataFrame(internal)) + + # TODO: add keep parameter + def nlargest(self, n=5) -> Series: + """ + Return the first n rows ordered by columns in descending order in group. + + Return the first n rows with the smallest values in columns, in descending order. + The columns that are not specified are returned as well, but not used for ordering. + + Parameters + ---------- + n : int + Number of items to retrieve. + + See Also + -------- + pyspark.pandas.Series.nlargest + pyspark.pandas.DataFrame.nlargest + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 1, 2, 2, 2, 3, 3, 3], + ... 'b': [1, 2, 2, 2, 3, 3, 3, 4, 4]}, columns=['a', 'b']) + + >>> df.groupby(['a'])['b'].nlargest(1).sort_index() # doctest: +NORMALIZE_WHITESPACE + a + 1 1 2 + 2 4 3 + 3 7 4 + Name: b, dtype: int64 + """ + if self._kser._internal.index_level > 1: + raise ValueError("nlargest do not support multi-index now") + + groupkey_col_names = [SPARK_INDEX_NAME_FORMAT(i) for i in range(len(self._groupkeys))] + sdf = self._kser._internal.spark_frame.select( + [scol.alias(name) for scol, name in zip(self._groupkeys_scols, groupkey_col_names)] + + [ + scol.alias(SPARK_INDEX_NAME_FORMAT(i + len(self._groupkeys))) + for i, scol in enumerate(self._kser._internal.index_spark_columns) + ] + + [self._kser.spark.column] + + [NATURAL_ORDER_COLUMN_NAME] + ) + + window = Window.partitionBy(groupkey_col_names).orderBy( + scol_for(sdf, self._kser._internal.data_spark_column_names[0]).desc(), + NATURAL_ORDER_COLUMN_NAME, + ) + + temp_rank_column = verify_temp_column_name(sdf, "__rank__") + sdf = ( + sdf.withColumn(temp_rank_column, F.row_number().over(window)) + .filter(F.col(temp_rank_column) <= n) + .drop(temp_rank_column) + ).drop(NATURAL_ORDER_COLUMN_NAME) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=( + [scol_for(sdf, col) for col in groupkey_col_names] + + [ + scol_for(sdf, SPARK_INDEX_NAME_FORMAT(i + len(self._groupkeys))) + for i in range(self._kdf._internal.index_level) + ] + ), + index_names=( + [kser._column_label for kser in self._groupkeys] + self._kdf._internal.index_names + ), + index_dtypes=( + [kser.dtype for kser in self._groupkeys] + self._kdf._internal.index_dtypes + ), + column_labels=[self._kser._column_label], + data_spark_columns=[scol_for(sdf, self._kser._internal.data_spark_column_names[0])], + data_dtypes=[self._kser.dtype], + ) + return first_series(DataFrame(internal)) + + # TODO: add bins, normalize parameter + def value_counts(self, sort=None, ascending=None, dropna=True) -> Series: + """ + Compute group sizes. + + Parameters + ---------- + sort : boolean, default None + Sort by frequencies. + ascending : boolean, default False + Sort in ascending order. + dropna : boolean, default True + Don't include counts of NaN. + + See Also + -------- + pyspark.pandas.Series.groupby + pyspark.pandas.DataFrame.groupby + + Examples + -------- + >>> df = pp.DataFrame({'A': [1, 2, 2, 3, 3, 3], + ... 'B': [1, 1, 2, 3, 3, 3]}, + ... columns=['A', 'B']) + >>> df + A B + 0 1 1 + 1 2 1 + 2 2 2 + 3 3 3 + 4 3 3 + 5 3 3 + + >>> df.groupby('A')['B'].value_counts().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + 1 1 1 + 2 1 1 + 2 1 + 3 3 3 + Name: B, dtype: int64 + """ + groupkeys = self._groupkeys + self._agg_columns + groupkey_names = [SPARK_INDEX_NAME_FORMAT(i) for i in range(len(groupkeys))] + groupkey_cols = [s.spark.column.alias(name) for s, name in zip(groupkeys, groupkey_names)] + + sdf = self._kdf._internal.spark_frame + agg_column = self._agg_columns[0]._internal.data_spark_column_names[0] + sdf = sdf.groupby(*groupkey_cols).count().withColumnRenamed("count", agg_column) + + if sort: + if ascending: + sdf = sdf.orderBy(scol_for(sdf, agg_column).asc()) + else: + sdf = sdf.orderBy(scol_for(sdf, agg_column).desc()) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in groupkey_names], + index_names=[kser._column_label for kser in groupkeys], + index_dtypes=[kser.dtype for kser in groupkeys], + column_labels=[self._agg_columns[0]._column_label], + data_spark_columns=[scol_for(sdf, agg_column)], + ) + return first_series(DataFrame(internal)) + + def unique(self) -> Series: + """ + Return unique values in group. + + Uniques are returned in order of unknown. It does NOT sort. + + See Also + -------- + pyspark.pandas.Series.unique + pyspark.pandas.Index.unique + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 1, 1, 2, 2, 2, 3, 3, 3], + ... 'b': [1, 2, 2, 2, 3, 3, 3, 4, 4]}, columns=['a', 'b']) + + >>> df.groupby(['a'])['b'].unique().sort_index() # doctest: +SKIP + a + 1 [1, 2] + 2 [2, 3] + 3 [3, 4] + Name: b, dtype: object + """ + return self._reduce_for_stat_function(F.collect_set, only_numeric=False) + + +def is_multi_agg_with_relabel(**kwargs): + """ + Check whether the kwargs pass to .agg look like multi-agg with relabling. + + Parameters + ---------- + **kwargs : dict + + Returns + ------- + bool + + Examples + -------- + >>> is_multi_agg_with_relabel(a='max') + False + >>> is_multi_agg_with_relabel(a_max=('a', 'max'), + ... a_min=('a', 'min')) + True + >>> is_multi_agg_with_relabel() + False + """ + if not kwargs: + return False + return all(isinstance(v, tuple) and len(v) == 2 for v in kwargs.values()) + + +def normalize_keyword_aggregation(kwargs): + """ + Normalize user-provided kwargs. + + Transforms from the new ``Dict[str, NamedAgg]`` style kwargs + to the old OrderedDict[str, List[scalar]]]. + + Parameters + ---------- + kwargs : dict + + Returns + ------- + aggspec : dict + The transformed kwargs. + columns : List[str] + The user-provided keys. + order : List[Tuple[str, str]] + Pairs of the input and output column names. + + Examples + -------- + >>> normalize_keyword_aggregation({'output': ('input', 'sum')}) + (OrderedDict([('input', ['sum'])]), ('output',), [('input', 'sum')]) + """ + # this is due to python version issue, not sure the impact on koalas + PY36 = sys.version_info >= (3, 6) + if not PY36: + kwargs = OrderedDict(sorted(kwargs.items())) + + # TODO(Py35): When we drop python 3.5, change this to defaultdict(list) + aggspec = OrderedDict() + order = [] + columns, pairs = list(zip(*kwargs.items())) + + for column, aggfunc in pairs: + if column in aggspec: + aggspec[column].append(aggfunc) + else: + aggspec[column] = [aggfunc] + + order.append((column, aggfunc)) + # For MultiIndex, we need to flatten the tuple, e.g. (('y', 'A'), 'max') needs to be + # flattened to ('y', 'A', 'max'), it won't do anything on normal Index. + if isinstance(order[0][0], tuple): + order = [(*levs, method) for levs, method in order] + return aggspec, columns, order diff --git a/python/pyspark/pandas/indexes/__init__.py b/python/pyspark/pandas/indexes/__init__.py new file mode 100644 index 0000000000..cd2adbaf9f --- /dev/null +++ b/python/pyspark/pandas/indexes/__init__.py @@ -0,0 +1,20 @@ +# +# 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. +# +from pyspark.pandas.indexes.base import Index # noqa: F401 +from pyspark.pandas.indexes.datetimes import DatetimeIndex # noqa: F401 +from pyspark.pandas.indexes.multi import MultiIndex # noqa: F401 +from pyspark.pandas.indexes.numeric import Float64Index, Int64Index # noqa: F401 diff --git a/python/pyspark/pandas/indexes/base.py b/python/pyspark/pandas/indexes/base.py new file mode 100644 index 0000000000..1b224dfc50 --- /dev/null +++ b/python/pyspark/pandas/indexes/base.py @@ -0,0 +1,2471 @@ +# +# 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. +# + +from functools import partial +from typing import Any, List, Optional, Tuple, Union +import warnings + +import pandas as pd +import numpy as np +from pandas.api.types import ( + is_list_like, + is_interval_dtype, + is_bool_dtype, + is_categorical_dtype, + is_integer_dtype, + is_float_dtype, + is_numeric_dtype, + is_object_dtype, +) +from pandas.core.accessor import CachedAccessor +from pandas.io.formats.printing import pprint_thing +from pandas.api.types import CategoricalDtype, is_hashable +from pandas._libs import lib + +from pyspark import sql as spark +from pyspark.sql import functions as F +from pyspark.sql.types import DataType, FractionalType, IntegralType, TimestampType + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas.config import get_option, option_context +from pyspark.pandas.base import IndexOpsMixin +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.missing.indexes import MissingPandasLikeIndex +from pyspark.pandas.series import Series, first_series +from pyspark.pandas.spark.accessors import SparkIndexMethods +from pyspark.pandas.utils import ( + is_name_like_tuple, + is_name_like_value, + name_like_string, + same_anchor, + scol_for, + verify_temp_column_name, + validate_bool_kwarg, + ERROR_MESSAGE_CANNOT_COMBINE, +) +from pyspark.pandas.internal import ( + InternalFrame, + DEFAULT_SERIES_NAME, + SPARK_DEFAULT_INDEX_NAME, + SPARK_INDEX_NAME_FORMAT, +) +from pyspark.pandas.typedef import Scalar + + +class Index(IndexOpsMixin): + """ + Koalas Index that corresponds to pandas Index logically. This might hold Spark Column + internally. + + Parameters + ---------- + data : array-like (1-dimensional) + dtype : dtype, default None + If dtype is None, we find the dtype that best fits the data. + If an actual dtype is provided, we coerce to that dtype if it's safe. + Otherwise, an error will be raised. + copy : bool + Make a copy of input ndarray. + name : object + Name to be stored in the index. + tupleize_cols : bool (default: True) + When True, attempt to create a MultiIndex if possible. + + See Also + -------- + MultiIndex : A multi-level, or hierarchical, Index. + DatetimeIndex : Index of datetime64 data. + Int64Index : A special case of :class:`Index` with purely integer labels. + Float64Index : A special case of :class:`Index` with purely float labels. + + Examples + -------- + >>> pp.DataFrame({'a': ['a', 'b', 'c']}, index=[1, 2, 3]).index + Int64Index([1, 2, 3], dtype='int64') + + >>> pp.DataFrame({'a': [1, 2, 3]}, index=list('abc')).index + Index(['a', 'b', 'c'], dtype='object') + + >>> pp.Index([1, 2, 3]) + Int64Index([1, 2, 3], dtype='int64') + + >>> pp.Index(list('abc')) + Index(['a', 'b', 'c'], dtype='object') + + From a Series: + + >>> s = pp.Series([1, 2, 3], index=[10, 20, 30]) + >>> pp.Index(s) + Int64Index([1, 2, 3], dtype='int64') + + From an Index: + + >>> idx = pp.Index([1, 2, 3]) + >>> pp.Index(idx) + Int64Index([1, 2, 3], dtype='int64') + """ + + def __new__(cls, data=None, dtype=None, copy=False, name=None, tupleize_cols=True, **kwargs): + if not is_hashable(name): + raise TypeError("Index.name must be a hashable type") + + if isinstance(data, Series): + if dtype is not None: + data = data.astype(dtype) + if name is not None: + data = data.rename(name) + + internal = InternalFrame( + spark_frame=data._internal.spark_frame, + index_spark_columns=data._internal.data_spark_columns, + index_names=data._internal.column_labels, + index_dtypes=data._internal.data_dtypes, + column_labels=[], + data_spark_columns=[], + data_dtypes=[], + ) + return DataFrame(internal).index + elif isinstance(data, Index): + if copy: + data = data.copy() + if dtype is not None: + data = data.astype(dtype) + if name is not None: + data = data.rename(name) + return data + + return pp.from_pandas( + pd.Index( + data=data, dtype=dtype, copy=copy, name=name, tupleize_cols=tupleize_cols, **kwargs + ) + ) + + @staticmethod + def _new_instance(anchor: DataFrame) -> "Index": + from pyspark.pandas.indexes.category import CategoricalIndex + from pyspark.pandas.indexes.datetimes import DatetimeIndex + from pyspark.pandas.indexes.multi import MultiIndex + from pyspark.pandas.indexes.numeric import Float64Index, Int64Index + + if anchor._internal.index_level > 1: + instance = object.__new__(MultiIndex) + elif isinstance(anchor._internal.index_dtypes[0], CategoricalDtype): + instance = object.__new__(CategoricalIndex) + elif isinstance( + anchor._internal.spark_type_for(anchor._internal.index_spark_columns[0]), IntegralType + ): + instance = object.__new__(Int64Index) + elif isinstance( + anchor._internal.spark_type_for(anchor._internal.index_spark_columns[0]), FractionalType + ): + instance = object.__new__(Float64Index) + elif isinstance( + anchor._internal.spark_type_for(anchor._internal.index_spark_columns[0]), TimestampType + ): + instance = object.__new__(DatetimeIndex) + else: + instance = object.__new__(Index) + + instance._anchor = anchor + return instance + + @property + def _kdf(self) -> DataFrame: + return self._anchor + + @property + def _internal(self) -> InternalFrame: + internal = self._kdf._internal + return internal.copy( + column_labels=internal.index_names, + data_spark_columns=internal.index_spark_columns, + data_dtypes=internal.index_dtypes, + column_label_names=None, + ) + + @property + def _column_label(self): + return self._kdf._internal.index_names[0] + + def _with_new_scol(self, scol: spark.Column, *, dtype=None) -> "Index": + """ + Copy Koalas Index with the new Spark Column. + + :param scol: the new Spark Column + :return: the copied Index + """ + internal = self._internal.copy( + index_spark_columns=[scol.alias(SPARK_DEFAULT_INDEX_NAME)], + index_dtypes=[dtype], + column_labels=[], + data_spark_columns=[], + data_dtypes=[], + ) + return DataFrame(internal).index + + spark = CachedAccessor("spark", SparkIndexMethods) + + # This method is used via `DataFrame.info` API internally. + def _summary(self, name=None) -> str: + """ + Return a summarized representation. + + Parameters + ---------- + name : str + name to use in the summary representation + + Returns + ------- + String with a summarized representation of the index + """ + head, tail, total_count = tuple( + self._internal.spark_frame.select( + F.first(self.spark.column), F.last(self.spark.column), F.count(F.expr("*")) + ) + .toPandas() + .iloc[0] + ) + + if total_count > 0: + index_summary = ", %s to %s" % (pprint_thing(head), pprint_thing(tail)) + else: + index_summary = "" + + if name is None: + name = type(self).__name__ + return "%s: %s entries%s" % (name, total_count, index_summary) + + @property + def size(self) -> int: + """ + Return an int representing the number of elements in this object. + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats'], + ... index=list('abcd')) + >>> df.index.size + 4 + + >>> df.set_index('dogs', append=True).index.size + 4 + """ + return len(self) + + @property + def shape(self) -> tuple: + """ + Return a tuple of the shape of the underlying data. + + Examples + -------- + >>> idx = pp.Index(['a', 'b', 'c']) + >>> idx + Index(['a', 'b', 'c'], dtype='object') + >>> idx.shape + (3,) + + >>> midx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> midx # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y'), + ('c', 'z')], + ) + >>> midx.shape + (3,) + """ + return (len(self._kdf),) + + def identical(self, other) -> bool: + """ + Similar to equals, but check that other comparable attributes are + also equal. + + Returns + ------- + bool + If two Index objects have equal elements and same type True, + otherwise False. + + Examples + -------- + + >>> from pyspark.pandas.config import option_context + >>> idx = pp.Index(['a', 'b', 'c']) + >>> midx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + + For Index + + >>> idx.identical(idx) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... idx.identical(pp.Index(['a', 'b', 'c'])) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... idx.identical(pp.Index(['b', 'b', 'a'])) + False + >>> idx.identical(midx) + False + + For MultiIndex + + >>> midx.identical(midx) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... midx.identical(pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')])) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... midx.identical(pp.MultiIndex.from_tuples([('c', 'z'), ('b', 'y'), ('a', 'x')])) + False + >>> midx.identical(idx) + False + """ + from pyspark.pandas.indexes.multi import MultiIndex + + self_name = self.names if isinstance(self, MultiIndex) else self.name + other_name = other.names if isinstance(other, MultiIndex) else other.name + + return ( + self_name == other_name # to support non-index comparison by short-circuiting. + and self.equals(other) + ) + + def equals(self, other) -> bool: + """ + Determine if two Index objects contain the same elements. + + Returns + ------- + bool + True if "other" is an Index and it has the same elements as calling + index; False otherwise. + + Examples + -------- + + >>> from pyspark.pandas.config import option_context + >>> idx = pp.Index(['a', 'b', 'c']) + >>> idx.name = "name" + >>> midx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> midx.names = ("nameA", "nameB") + + For Index + + >>> idx.equals(idx) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... idx.equals(pp.Index(['a', 'b', 'c'])) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... idx.equals(pp.Index(['b', 'b', 'a'])) + False + >>> idx.equals(midx) + False + + For MultiIndex + + >>> midx.equals(midx) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... midx.equals(pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')])) + True + >>> with option_context('compute.ops_on_diff_frames', True): + ... midx.equals(pp.MultiIndex.from_tuples([('c', 'z'), ('b', 'y'), ('a', 'x')])) + False + >>> midx.equals(idx) + False + """ + if same_anchor(self, other): + return True + elif type(self) == type(other): + if get_option("compute.ops_on_diff_frames"): + # TODO: avoid using default index? + with option_context("compute.default_index_type", "distributed-sequence"): + # Directly using Series from both self and other seems causing + # some exceptions when 'compute.ops_on_diff_frames' is enabled. + # Working around for now via using frame. + return ( + self.to_series("self").reset_index(drop=True) + == other.to_series("other").reset_index(drop=True) + ).all() + else: + raise ValueError(ERROR_MESSAGE_CANNOT_COMBINE) + else: + return False + + def transpose(self) -> "Index": + """ + Return the transpose, For index, It will be index itself. + + Examples + -------- + >>> idx = pp.Index(['a', 'b', 'c']) + >>> idx + Index(['a', 'b', 'c'], dtype='object') + + >>> idx.transpose() + Index(['a', 'b', 'c'], dtype='object') + + For MultiIndex + + >>> midx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> midx # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y'), + ('c', 'z')], + ) + + >>> midx.transpose() # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y'), + ('c', 'z')], + ) + """ + return self + + T = property(transpose) + + def _to_internal_pandas(self) -> pd.Index: + """ + Return a pandas Index directly from _internal to avoid overhead of copy. + + This method is for internal use only. + """ + return self._kdf._internal.to_pandas_frame.index + + def to_pandas(self) -> pd.Index: + """ + Return a pandas Index. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats'], + ... index=list('abcd')) + >>> df['dogs'].index.to_pandas() + Index(['a', 'b', 'c', 'd'], dtype='object') + """ + return self._to_internal_pandas().copy() + + def toPandas(self) -> pd.Index: + warnings.warn( + "Index.toPandas is deprecated as of Index.to_pandas. Please use the API instead.", + FutureWarning, + ) + return self.to_pandas() + + toPandas.__doc__ = to_pandas.__doc__ + + def to_numpy(self, dtype=None, copy=False) -> np.ndarray: + """ + A NumPy ndarray representing the values in this Index or MultiIndex. + + .. note:: This method should only be used if the resulting NumPy ndarray is expected + to be small, as all the data is loaded into the driver's memory. + + Parameters + ---------- + dtype : str or numpy.dtype, optional + The dtype to pass to :meth:`numpy.asarray` + copy : bool, default False + Whether to ensure that the returned value is a not a view on + another array. Note that ``copy=False`` does not *ensure* that + ``to_numpy()`` is no-copy. Rather, ``copy=True`` ensure that + a copy is made, even if not strictly necessary. + + Returns + ------- + numpy.ndarray + + Examples + -------- + >>> pp.Series([1, 2, 3, 4]).index.to_numpy() + array([0, 1, 2, 3]) + >>> pp.DataFrame({'a': ['a', 'b', 'c']}, index=[[1, 2, 3], [4, 5, 6]]).index.to_numpy() + array([(1, 4), (2, 5), (3, 6)], dtype=object) + """ + result = np.asarray(self._to_internal_pandas()._values, dtype=dtype) + if copy: + result = result.copy() + return result + + @property + def values(self) -> np.ndarray: + """ + Return an array representing the data in the Index. + + .. warning:: We recommend using `Index.to_numpy()` instead. + + .. note:: This method should only be used if the resulting NumPy ndarray is expected + to be small, as all the data is loaded into the driver's memory. + + Returns + ------- + numpy.ndarray + + Examples + -------- + >>> pp.Series([1, 2, 3, 4]).index.values + array([0, 1, 2, 3]) + >>> pp.DataFrame({'a': ['a', 'b', 'c']}, index=[[1, 2, 3], [4, 5, 6]]).index.values + array([(1, 4), (2, 5), (3, 6)], dtype=object) + """ + warnings.warn("We recommend using `{}.to_numpy()` instead.".format(type(self).__name__)) + return self.to_numpy() + + @property + def asi8(self) -> np.ndarray: + """ + Integer representation of the values. + + .. warning:: We recommend using `Index.to_numpy()` instead. + + .. note:: This method should only be used if the resulting NumPy ndarray is expected + to be small, as all the data is loaded into the driver's memory. + + Returns + ------- + numpy.ndarray + An ndarray with int64 dtype. + + Examples + -------- + >>> pp.Index([1, 2, 3]).asi8 + array([1, 2, 3]) + + Returns None for non-int64 dtype + + >>> pp.Index(['a', 'b', 'c']).asi8 is None + True + """ + warnings.warn("We recommend using `{}.to_numpy()` instead.".format(type(self).__name__)) + if isinstance(self.spark.data_type, IntegralType): + return self.to_numpy() + elif isinstance(self.spark.data_type, TimestampType): + return np.array(list(map(lambda x: x.astype(np.int64), self.to_numpy()))) + else: + return None + + @property + def spark_type(self) -> DataType: + """ Returns the data type as defined by Spark, as a Spark DataType object.""" + warnings.warn( + "Index.spark_type is deprecated as of Index.spark.data_type. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.data_type + + @property + def has_duplicates(self) -> bool: + """ + If index has duplicates, return True, otherwise False. + + Examples + -------- + >>> idx = pp.Index([1, 5, 7, 7]) + >>> idx.has_duplicates + True + + >>> idx = pp.Index([1, 5, 7]) + >>> idx.has_duplicates + False + + >>> idx = pp.Index(["Watermelon", "Orange", "Apple", + ... "Watermelon"]) + >>> idx.has_duplicates + True + + >>> idx = pp.Index(["Orange", "Apple", + ... "Watermelon"]) + >>> idx.has_duplicates + False + """ + sdf = self._internal.spark_frame.select(self.spark.column) + scol = scol_for(sdf, sdf.columns[0]) + + return sdf.select(F.count(scol) != F.countDistinct(scol)).first()[0] + + @property + def is_unique(self) -> bool: + """ + Return if the index has unique values. + + Examples + -------- + >>> idx = pp.Index([1, 5, 7, 7]) + >>> idx.is_unique + False + + >>> idx = pp.Index([1, 5, 7]) + >>> idx.is_unique + True + + >>> idx = pp.Index(["Watermelon", "Orange", "Apple", + ... "Watermelon"]) + >>> idx.is_unique + False + + >>> idx = pp.Index(["Orange", "Apple", + ... "Watermelon"]) + >>> idx.is_unique + True + """ + return not self.has_duplicates + + @property + def name(self) -> Union[Any, Tuple]: + """Return name of the Index.""" + return self.names[0] + + @name.setter + def name(self, name: Union[Any, Tuple]) -> None: + self.names = [name] + + @property + def names(self) -> List[Union[Any, Tuple]]: + """Return names of the Index.""" + return [ + name if name is None or len(name) > 1 else name[0] + for name in self._internal.index_names # type: ignore + ] + + @names.setter + def names(self, names: List[Union[Any, Tuple]]) -> None: + if not is_list_like(names): + raise ValueError("Names must be a list-like") + if self._internal.index_level != len(names): + raise ValueError( + "Length of new names must be {}, got {}".format( + self._internal.index_level, len(names) + ) + ) + if self._internal.index_level == 1: + self.rename(names[0], inplace=True) + else: + self.rename(names, inplace=True) + + @property + def nlevels(self) -> int: + """ + Number of levels in Index & MultiIndex. + + Examples + -------- + >>> kdf = pp.DataFrame({"a": [1, 2, 3]}, index=pd.Index(['a', 'b', 'c'], name="idx")) + >>> kdf.index.nlevels + 1 + + >>> kdf = pp.DataFrame({'a': [1, 2, 3]}, index=[list('abc'), list('def')]) + >>> kdf.index.nlevels + 2 + """ + return self._internal.index_level + + def rename( + self, name: Union[Any, Tuple, List[Union[Any, Tuple]]], inplace: bool = False + ) -> Optional["Index"]: + """ + Alter Index or MultiIndex name. + Able to set new names without level. Defaults to returning new index. + + Parameters + ---------- + name : label or list of labels + Name(s) to set. + inplace : boolean, default False + Modifies the object directly, instead of creating a new Index or MultiIndex. + + Returns + ------- + Index or MultiIndex + The same type as the caller or None if inplace is True. + + Examples + -------- + >>> df = pp.DataFrame({'a': ['A', 'C'], 'b': ['A', 'B']}, columns=['a', 'b']) + >>> df.index.rename("c") + Int64Index([0, 1], dtype='int64', name='c') + + >>> df.set_index("a", inplace=True) + >>> df.index.rename("d") + Index(['A', 'C'], dtype='object', name='d') + + You can also change the index name in place. + + >>> df.index.rename("e", inplace=True) + >>> df.index + Index(['A', 'C'], dtype='object', name='e') + + >>> df # doctest: +NORMALIZE_WHITESPACE + b + e + A A + C B + + Support for MultiIndex + + >>> kidx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y')]) + >>> kidx.names = ['hello', 'koalas'] + >>> kidx # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y')], + names=['hello', 'koalas']) + + >>> kidx.rename(['aloha', 'databricks']) # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y')], + names=['aloha', 'databricks']) + """ + names = self._verify_for_rename(name) + + internal = self._kdf._internal.copy(index_names=names) + + if inplace: + self._kdf._update_internal_frame(internal) + return None + else: + return DataFrame(internal).index + + def _verify_for_rename(self, name): + if is_hashable(name): + if is_name_like_tuple(name): + return [name] + elif is_name_like_value(name): + return [(name,)] + raise TypeError("Index.name must be a hashable type") + + # TODO: add downcast parameter for fillna function + def fillna(self, value: Scalar) -> "Index": + """ + Fill NA/NaN values with the specified value. + + Parameters + ---------- + value : scalar + Scalar value to use to fill holes (e.g. 0). This value cannot be a list-likes. + + Returns + ------- + Index : + filled with value + + Examples + -------- + >>> ki = pp.DataFrame({'a': ['a', 'b', 'c']}, index=[1, 2, None]).index + >>> ki + Float64Index([1.0, 2.0, nan], dtype='float64') + + >>> ki.fillna(0) + Float64Index([1.0, 2.0, 0.0], dtype='float64') + """ + if not isinstance(value, (float, int, str, bool)): + raise TypeError("Unsupported type %s" % type(value).__name__) + sdf = self._internal.spark_frame.fillna(value) + result = DataFrame(self._kdf._internal.with_new_sdf(sdf)).index # TODO: dtype? + return result + + # TODO: ADD keep parameter + def drop_duplicates(self) -> "Index": + """ + Return Index with duplicate values removed. + + Returns + ------- + deduplicated : Index + + See Also + -------- + Series.drop_duplicates : Equivalent method on Series. + DataFrame.drop_duplicates : Equivalent method on DataFrame. + + Examples + -------- + Generate an pandas.Index with duplicate values. + + >>> idx = pp.Index(['lama', 'cow', 'lama', 'beetle', 'lama', 'hippo']) + + >>> idx.drop_duplicates().sort_values() + Index(['beetle', 'cow', 'hippo', 'lama'], dtype='object') + """ + sdf = self._internal.spark_frame.select( + self._internal.index_spark_columns + ).drop_duplicates() + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + return DataFrame(internal).index + + def to_series(self, name: Union[Any, Tuple] = None) -> Series: + """ + Create a Series with both index and values equal to the index keys + useful with map for returning an indexer based on an index. + + Parameters + ---------- + name : string, optional + name of resulting Series. If None, defaults to name of original + index + + Returns + ------- + Series : dtype will be based on the type of the Index values. + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats'], + ... index=list('abcd')) + >>> df['dogs'].index.to_series() + a a + b b + c c + d d + dtype: object + """ + if not is_hashable(name): + raise TypeError("Series.name must be a hashable type") + scol = self.spark.column + if name is not None: + scol = scol.alias(name_like_string(name)) + elif self._internal.index_level == 1: + name = self.name + column_labels = [ + name if is_name_like_tuple(name) else (name,) + ] # type: List[Optional[Tuple]] + internal = self._internal.copy( + column_labels=column_labels, data_spark_columns=[scol], column_label_names=None + ) + return first_series(DataFrame(internal)) + + def to_frame(self, index=True, name=None) -> DataFrame: + """ + Create a DataFrame with a column containing the Index. + + Parameters + ---------- + index : boolean, default True + Set the index of the returned DataFrame as the original Index. + name : object, default None + The passed name should substitute for the index name (if it has + one). + + Returns + ------- + DataFrame + DataFrame containing the original Index data. + + See Also + -------- + Index.to_series : Convert an Index to a Series. + Series.to_frame : Convert Series to DataFrame. + + Examples + -------- + >>> idx = pp.Index(['Ant', 'Bear', 'Cow'], name='animal') + >>> idx.to_frame() # doctest: +NORMALIZE_WHITESPACE + animal + animal + Ant Ant + Bear Bear + Cow Cow + + By default, the original Index is reused. To enforce a new Index: + + >>> idx.to_frame(index=False) + animal + 0 Ant + 1 Bear + 2 Cow + + To override the name of the resulting column, specify `name`: + + >>> idx.to_frame(name='zoo') # doctest: +NORMALIZE_WHITESPACE + zoo + animal + Ant Ant + Bear Bear + Cow Cow + """ + if name is None: + if self._internal.index_names[0] is None: + name = (DEFAULT_SERIES_NAME,) + else: + name = self._internal.index_names[0] + elif not is_name_like_tuple(name): + if is_name_like_value(name): + name = (name,) + else: + raise TypeError("unhashable type: '{}'".format(type(name).__name__)) + + return self._to_frame(index=index, names=[name]) + + def _to_frame(self, index, names): + if index: + index_spark_columns = self._internal.index_spark_columns + index_names = self._internal.index_names + index_dtypes = self._internal.index_dtypes + else: + index_spark_columns = [] + index_names = [] + index_dtypes = [] + + internal = InternalFrame( + spark_frame=self._internal.spark_frame, + index_spark_columns=index_spark_columns, + index_names=index_names, + index_dtypes=index_dtypes, + column_labels=names, + data_spark_columns=self._internal.index_spark_columns, + data_dtypes=self._internal.index_dtypes, + ) + return DataFrame(internal) + + def is_boolean(self) -> bool: + """ + Return if the current index type is a boolean type. + + Examples + -------- + >>> pp.DataFrame({'a': [1]}, index=[True]).index.is_boolean() + True + """ + return is_bool_dtype(self.dtype) + + def is_categorical(self) -> bool: + """ + Return if the current index type is a categorical type. + + Examples + -------- + >>> pp.DataFrame({'a': [1]}, index=[1]).index.is_categorical() + False + """ + return is_categorical_dtype(self.dtype) + + def is_floating(self) -> bool: + """ + Return if the current index type is a floating type. + + Examples + -------- + >>> pp.DataFrame({'a': [1]}, index=[1]).index.is_floating() + False + """ + return is_float_dtype(self.dtype) + + def is_integer(self) -> bool: + """ + Return if the current index type is a integer type. + + Examples + -------- + >>> pp.DataFrame({'a': [1]}, index=[1]).index.is_integer() + True + """ + return is_integer_dtype(self.dtype) + + def is_interval(self) -> bool: + """ + Return if the current index type is an interval type. + + Examples + -------- + >>> pp.DataFrame({'a': [1]}, index=[1]).index.is_interval() + False + """ + return is_interval_dtype(self.dtype) + + def is_numeric(self) -> bool: + """ + Return if the current index type is a numeric type. + + Examples + -------- + >>> pp.DataFrame({'a': [1]}, index=[1]).index.is_numeric() + True + """ + return is_numeric_dtype(self.dtype) + + def is_object(self) -> bool: + """ + Return if the current index type is a object type. + + Examples + -------- + >>> pp.DataFrame({'a': [1]}, index=["a"]).index.is_object() + True + """ + return is_object_dtype(self.dtype) + + def is_type_compatible(self, kind) -> bool: + """ + Whether the index type is compatible with the provided type. + + Examples + -------- + >>> kidx = pp.Index([1, 2, 3]) + >>> kidx.is_type_compatible('integer') + True + + >>> kidx = pp.Index([1.0, 2.0, 3.0]) + >>> kidx.is_type_compatible('integer') + False + >>> kidx.is_type_compatible('floating') + True + """ + return kind == self.inferred_type + + def dropna(self) -> "Index": + """ + Return Index or MultiIndex without NA/NaN values + + Examples + -------- + + >>> df = pp.DataFrame([[1, 2], [4, 5], [7, 8]], + ... index=['cobra', 'viper', None], + ... columns=['max_speed', 'shield']) + >>> df + max_speed shield + cobra 1 2 + viper 4 5 + NaN 7 8 + + >>> df.index.dropna() + Index(['cobra', 'viper'], dtype='object') + + Also support for MultiIndex + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... [None, 'weight', 'length']], + ... [[0, 1, 1, 1, 1, 1, 2, 2, 2], + ... [0, 1, 1, 0, 1, 2, 1, 1, 2]]) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, None], + ... index=midx) + >>> s + lama NaN 45.0 + cow weight 200.0 + weight 1.2 + NaN 30.0 + weight 250.0 + length 1.5 + falcon weight 320.0 + weight 1.0 + length NaN + dtype: float64 + + >>> s.index.dropna() # doctest: +SKIP + MultiIndex([( 'cow', 'weight'), + ( 'cow', 'weight'), + ( 'cow', 'weight'), + ( 'cow', 'length'), + ('falcon', 'weight'), + ('falcon', 'weight'), + ('falcon', 'length')], + ) + """ + sdf = self._internal.spark_frame.select(self._internal.index_spark_columns).dropna() + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + return DataFrame(internal).index + + def unique(self, level=None) -> "Index": + """ + Return unique values in the index. + + Be aware the order of unique values might be different than pandas.Index.unique + + Parameters + ---------- + level : int or str, optional, default is None + + Returns + ------- + Index without duplicates + + See Also + -------- + Series.unique + groupby.SeriesGroupBy.unique + + Examples + -------- + >>> pp.DataFrame({'a': ['a', 'b', 'c']}, index=[1, 1, 3]).index.unique().sort_values() + Int64Index([1, 3], dtype='int64') + + >>> pp.DataFrame({'a': ['a', 'b', 'c']}, index=['d', 'e', 'e']).index.unique().sort_values() + Index(['d', 'e'], dtype='object') + + MultiIndex + + >>> pp.MultiIndex.from_tuples([("A", "X"), ("A", "Y"), ("A", "X")]).unique() + ... # doctest: +SKIP + MultiIndex([('A', 'X'), + ('A', 'Y')], + ) + """ + if level is not None: + self._validate_index_level(level) + scols = self._internal.index_spark_columns + sdf = self._kdf._internal.spark_frame.select(scols).distinct() + return DataFrame( + InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + ).index + + # TODO: add error parameter + def drop(self, labels) -> "Index": + """ + Make new Index with passed list of labels deleted. + + Parameters + ---------- + labels : array-like + + Returns + ------- + dropped : Index + + Examples + -------- + >>> index = pp.Index([1, 2, 3]) + >>> index + Int64Index([1, 2, 3], dtype='int64') + + >>> index.drop([1]) + Int64Index([2, 3], dtype='int64') + """ + internal = self._internal.resolved_copy + sdf = internal.spark_frame[~internal.index_spark_columns[0].isin(labels)] + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + column_labels=[], + data_spark_columns=[], + data_dtypes=[], + ) + return DataFrame(internal).index + + def _validate_index_level(self, level): + """ + Validate index level. + For single-level Index getting level number is a no-op, but some + verification must be done like in MultiIndex. + """ + if isinstance(level, int): + if level < 0 and level != -1: + raise IndexError( + "Too many levels: Index has only 1 level," + " %d is not a valid level number" % (level,) + ) + elif level > 0: + raise IndexError("Too many levels:" " Index has only 1 level, not %d" % (level + 1)) + elif level != self.name: + raise KeyError( + "Requested level ({}) does not match index name ({})".format(level, self.name) + ) + + def get_level_values(self, level) -> "Index": + """ + Return Index if a valid level is given. + + Examples: + -------- + >>> kidx = pp.Index(['a', 'b', 'c'], name='ks') + >>> kidx.get_level_values(0) + Index(['a', 'b', 'c'], dtype='object', name='ks') + + >>> kidx.get_level_values('ks') + Index(['a', 'b', 'c'], dtype='object', name='ks') + """ + self._validate_index_level(level) + return self + + def copy(self, name=None, deep=None) -> "Index": + """ + Make a copy of this object. name sets those attributes on the new object. + + Parameters + ---------- + name : string, optional + to set name of index + deep : None + this parameter is not supported but just dummy parameter to match pandas. + + Examples + -------- + >>> df = pp.DataFrame([[1, 2], [4, 5], [7, 8]], + ... index=['cobra', 'viper', 'sidewinder'], + ... columns=['max_speed', 'shield']) + >>> df + max_speed shield + cobra 1 2 + viper 4 5 + sidewinder 7 8 + >>> df.index + Index(['cobra', 'viper', 'sidewinder'], dtype='object') + + Copy index + + >>> df.index.copy() + Index(['cobra', 'viper', 'sidewinder'], dtype='object') + + Copy index with name + + >>> df.index.copy(name='snake') + Index(['cobra', 'viper', 'sidewinder'], dtype='object', name='snake') + """ + result = self._kdf.copy().index + if name: + result.name = name + return result + + def droplevel(self, level) -> "Index": + """ + Return index with requested level(s) removed. + If resulting index has only 1 level left, the result will be + of Index type, not MultiIndex. + + Parameters + ---------- + level : int, str, tuple, or list-like, default 0 + If a string is given, must be the name of a level + If list-like, elements must be names or indexes of levels. + + Returns + ------- + Index or MultiIndex + + Examples + -------- + >>> midx = pp.DataFrame({'a': ['a', 'b']}, index=[['a', 'x'], ['b', 'y'], [1, 2]]).index + >>> midx # doctest: +SKIP + MultiIndex([('a', 'b', 1), + ('x', 'y', 2)], + ) + >>> midx.droplevel([0, 1]) # doctest: +SKIP + Int64Index([1, 2], dtype='int64') + >>> midx.droplevel(0) # doctest: +SKIP + MultiIndex([('b', 1), + ('y', 2)], + ) + >>> midx.names = [("a", "b"), "b", "c"] + >>> midx.droplevel([('a', 'b')]) # doctest: +SKIP + MultiIndex([('b', 1), + ('y', 2)], + names=['b', 'c']) + """ + names = self.names + nlevels = self.nlevels + if not is_list_like(level): + level = [level] + + int_level = set() + for n in level: + if isinstance(n, int): + if n < 0: + n = n + nlevels + if n < 0: + raise IndexError( + "Too many levels: Index has only {} levels, " + "{} is not a valid level number".format(nlevels, (n - nlevels)) + ) + if n >= nlevels: + raise IndexError( + "Too many levels: Index has only {} levels, not {}".format(nlevels, n + 1) + ) + else: + if n not in names: + raise KeyError("Level {} not found".format(n)) + n = names.index(n) + int_level.add(n) + + if len(level) >= nlevels: + raise ValueError( + "Cannot remove {} levels from an index with {} " + "levels: at least one level must be " + "left.".format(len(level), nlevels) + ) + + index_spark_columns, index_names, index_dtypes = zip( + *[ + item + for i, item in enumerate( + zip( + self._internal.index_spark_columns, + self._internal.index_names, + self._internal.index_dtypes, + ) + ) + if i not in int_level + ] + ) + + internal = self._internal.copy( + index_spark_columns=list(index_spark_columns), + index_names=list(index_names), + index_dtypes=list(index_dtypes), + column_labels=[], + data_spark_columns=[], + data_dtypes=[], + ) + return DataFrame(internal).index + + def symmetric_difference(self, other, result_name=None, sort=None) -> "Index": + """ + Compute the symmetric difference of two Index objects. + + Parameters + ---------- + other : Index or array-like + result_name : str + sort : True or None, default None + Whether to sort the resulting index. + * True : Attempt to sort the result. + * None : Do not sort the result. + + Returns + ------- + symmetric_difference : Index + + Notes + ----- + ``symmetric_difference`` contains elements that appear in either + ``idx1`` or ``idx2`` but not both. Equivalent to the Index created by + ``idx1.difference(idx2) | idx2.difference(idx1)`` with duplicates + dropped. + + Examples + -------- + >>> s1 = pp.Series([1, 2, 3, 4], index=[1, 2, 3, 4]) + >>> s2 = pp.Series([1, 2, 3, 4], index=[2, 3, 4, 5]) + + >>> s1.index.symmetric_difference(s2.index) + Int64Index([5, 1], dtype='int64') + + You can set name of result Index. + + >>> s1.index.symmetric_difference(s2.index, result_name='koalas') + Int64Index([5, 1], dtype='int64', name='koalas') + + You can set sort to `True`, if you want to sort the resulting index. + + >>> s1.index.symmetric_difference(s2.index, sort=True) + Int64Index([1, 5], dtype='int64') + + You can also use the ``^`` operator: + + >>> s1.index ^ s2.index + Int64Index([5, 1], dtype='int64') + """ + if type(self) != type(other): + raise NotImplementedError( + "Doesn't support symmetric_difference between Index & MultiIndex for now" + ) + + sdf_self = self._kdf._internal.spark_frame.select(self._internal.index_spark_columns) + sdf_other = other._kdf._internal.spark_frame.select(other._internal.index_spark_columns) + + sdf_symdiff = sdf_self.union(sdf_other).subtract(sdf_self.intersect(sdf_other)) + + if sort: + sdf_symdiff = sdf_symdiff.sort(self._internal.index_spark_column_names) + + internal = InternalFrame( + spark_frame=sdf_symdiff, + index_spark_columns=[ + scol_for(sdf_symdiff, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + result = DataFrame(internal).index + + if result_name: + result.name = result_name + + return result + + # TODO: return_indexer + def sort_values(self, ascending=True) -> "Index": + """ + Return a sorted copy of the index. + + .. note:: This method is not supported for pandas when index has NaN value. + pandas raises unexpected TypeError, but we support treating NaN + as the smallest value. + + Parameters + ---------- + ascending : bool, default True + Should the index values be sorted in an ascending order. + + Returns + ------- + sorted_index : pp.Index or pp.MultiIndex + Sorted copy of the index. + + See Also + -------- + Series.sort_values : Sort values of a Series. + DataFrame.sort_values : Sort values in a DataFrame. + + Examples + -------- + >>> idx = pp.Index([10, 100, 1, 1000]) + >>> idx + Int64Index([10, 100, 1, 1000], dtype='int64') + + Sort values in ascending order (default behavior). + + >>> idx.sort_values() + Int64Index([1, 10, 100, 1000], dtype='int64') + + Sort values in descending order. + + >>> idx.sort_values(ascending=False) + Int64Index([1000, 100, 10, 1], dtype='int64') + + Support for MultiIndex. + + >>> kidx = pp.MultiIndex.from_tuples([('a', 'x', 1), ('c', 'y', 2), ('b', 'z', 3)]) + >>> kidx # doctest: +SKIP + MultiIndex([('a', 'x', 1), + ('c', 'y', 2), + ('b', 'z', 3)], + ) + + >>> kidx.sort_values() # doctest: +SKIP + MultiIndex([('a', 'x', 1), + ('b', 'z', 3), + ('c', 'y', 2)], + ) + + >>> kidx.sort_values(ascending=False) # doctest: +SKIP + MultiIndex([('c', 'y', 2), + ('b', 'z', 3), + ('a', 'x', 1)], + ) + """ + sdf = self._internal.spark_frame + sdf = sdf.orderBy(self._internal.index_spark_columns, ascending=ascending).select( + self._internal.index_spark_columns + ) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + return DataFrame(internal).index + + def sort(self, *args, **kwargs) -> None: + """ + Use sort_values instead. + """ + raise TypeError("cannot sort an Index object in-place, use sort_values instead") + + def min(self) -> Union[Scalar, Tuple[Scalar, ...]]: + """ + Return the minimum value of the Index. + + Returns + ------- + scalar + Minimum value. + + See Also + -------- + Index.max : Return the maximum value of the object. + Series.min : Return the minimum value in a Series. + DataFrame.min : Return the minimum values in a DataFrame. + + Examples + -------- + >>> idx = pp.Index([3, 2, 1]) + >>> idx.min() + 1 + + >>> idx = pp.Index(['c', 'b', 'a']) + >>> idx.min() + 'a' + + For a MultiIndex, the maximum is determined lexicographically. + + >>> idx = pp.MultiIndex.from_tuples([('a', 'x', 1), ('b', 'y', 2)]) + >>> idx.min() + ('a', 'x', 1) + """ + sdf = self._internal.spark_frame + min_row = ( + sdf.select(F.min(F.struct(self._internal.index_spark_columns)).alias("min_row")) + .select("min_row.*") + .toPandas() + ) + result = tuple(min_row.iloc[0]) + + return result if len(result) > 1 else result[0] + + def max(self) -> Union[Scalar, Tuple[Scalar, ...]]: + """ + Return the maximum value of the Index. + + Returns + ------- + scalar + Maximum value. + + See Also + -------- + Index.min : Return the minimum value in an Index. + Series.max : Return the maximum value in a Series. + DataFrame.max : Return the maximum values in a DataFrame. + + Examples + -------- + >>> idx = pp.Index([3, 2, 1]) + >>> idx.max() + 3 + + >>> idx = pp.Index(['c', 'b', 'a']) + >>> idx.max() + 'c' + + For a MultiIndex, the maximum is determined lexicographically. + + >>> idx = pp.MultiIndex.from_tuples([('a', 'x', 1), ('b', 'y', 2)]) + >>> idx.max() + ('b', 'y', 2) + """ + sdf = self._internal.spark_frame + max_row = ( + sdf.select(F.max(F.struct(self._internal.index_spark_columns)).alias("max_row")) + .select("max_row.*") + .toPandas() + ) + result = tuple(max_row.iloc[0]) + + return result if len(result) > 1 else result[0] + + def delete(self, loc) -> "Index": + """ + Make new Index with passed location(-s) deleted. + + .. note:: this API can be pretty expensive since it is based on + a global sequence internally. + + Returns + ------- + new_index : Index + + Examples + -------- + >>> kidx = pp.Index([10, 10, 9, 8, 4, 2, 4, 4, 2, 2, 10, 10]) + >>> kidx + Int64Index([10, 10, 9, 8, 4, 2, 4, 4, 2, 2, 10, 10], dtype='int64') + + >>> kidx.delete(0).sort_values() + Int64Index([2, 2, 2, 4, 4, 4, 8, 9, 10, 10, 10], dtype='int64') + + >>> kidx.delete([0, 1, 2, 3, 10, 11]).sort_values() + Int64Index([2, 2, 2, 4, 4, 4], dtype='int64') + + MultiIndex + + >>> kidx = pp.MultiIndex.from_tuples([('a', 'x', 1), ('b', 'y', 2), ('c', 'z', 3)]) + >>> kidx # doctest: +SKIP + MultiIndex([('a', 'x', 1), + ('b', 'y', 2), + ('c', 'z', 3)], + ) + + >>> kidx.delete([0, 2]).sort_values() # doctest: +SKIP + MultiIndex([('b', 'y', 2)], + ) + """ + length = len(self) + + def is_len_exceeded(index): + """Check if the given index is exceeded the length or not""" + return index >= length if index >= 0 else abs(index) > length + + if not is_list_like(loc): + if is_len_exceeded(loc): + raise IndexError( + "index {} is out of bounds for axis 0 with size {}".format(loc, length) + ) + loc = [loc] + else: + for index in loc: + if is_len_exceeded(index): + raise IndexError( + "index {} is out of bounds for axis 0 with size {}".format(index, length) + ) + + loc = [int(item) for item in loc] + loc = [item if item >= 0 else length + item for item in loc] + + # we need a temporary column such as '__index_value_0__' + # since 'InternalFrame.attach_default_index' will be failed + # when self._scol has name of '__index_level_0__' + index_value_column_format = "__index_value_{}__" + + sdf = self._internal._sdf + index_value_column_names = [ + verify_temp_column_name(sdf, index_value_column_format.format(i)) + for i in range(self._internal.index_level) + ] + index_value_columns = [ + index_scol.alias(index_vcol_name) + for index_scol, index_vcol_name in zip( + self._internal.index_spark_columns, index_value_column_names + ) + ] + sdf = sdf.select(index_value_columns) + + sdf = InternalFrame.attach_default_index(sdf, default_index_type="distributed-sequence") + # sdf here looks as below + # +-----------------+-----------------+-----------------+-----------------+ + # |__index_level_0__|__index_value_0__|__index_value_1__|__index_value_2__| + # +-----------------+-----------------+-----------------+-----------------+ + # | 0| a| x| 1| + # | 1| b| y| 2| + # | 2| c| z| 3| + # +-----------------+-----------------+-----------------+-----------------+ + + # delete rows which are matched with given `loc` + sdf = sdf.where(~F.col(SPARK_INDEX_NAME_FORMAT(0)).isin(loc)) + sdf = sdf.select(index_value_column_names) + # sdf here looks as below, we should alias them back to origin spark column names + # +-----------------+-----------------+-----------------+ + # |__index_value_0__|__index_value_1__|__index_value_2__| + # +-----------------+-----------------+-----------------+ + # | c| z| 3| + # +-----------------+-----------------+-----------------+ + index_origin_columns = [ + F.col(index_vcol_name).alias(index_scol_name) + for index_vcol_name, index_scol_name in zip( + index_value_column_names, self._internal.index_spark_column_names + ) + ] + sdf = sdf.select(index_origin_columns) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + + return DataFrame(internal).index + + def append(self, other: "Index") -> "Index": + """ + Append a collection of Index options together. + + Parameters + ---------- + other : Index + + Returns + ------- + appended : Index + + Examples + -------- + >>> kidx = pp.Index([10, 5, 0, 5, 10, 5, 0, 10]) + >>> kidx + Int64Index([10, 5, 0, 5, 10, 5, 0, 10], dtype='int64') + + >>> kidx.append(kidx) + Int64Index([10, 5, 0, 5, 10, 5, 0, 10, 10, 5, 0, 5, 10, 5, 0, 10], dtype='int64') + + Support for MiltiIndex + + >>> kidx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y')]) + >>> kidx # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y')], + ) + + >>> kidx.append(kidx) # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y'), + ('a', 'x'), + ('b', 'y')], + ) + """ + from pyspark.pandas.indexes.multi import MultiIndex + + if type(self) is not type(other): + raise NotImplementedError( + "append() between Index & MultiIndex currently is not supported" + ) + + sdf_self = self._internal.spark_frame.select(self._internal.index_spark_columns) + sdf_other = other._internal.spark_frame.select(other._internal.index_spark_columns) + sdf_appended = sdf_self.union(sdf_other) + + # names should be kept when MultiIndex, but Index wouldn't keep its name. + if isinstance(self, MultiIndex): + index_names = self._internal.index_names + else: + index_names = None + + internal = InternalFrame( # TODO: dtypes? + spark_frame=sdf_appended, + index_spark_columns=[ + scol_for(sdf_appended, col) for col in self._internal.index_spark_column_names + ], + index_names=index_names, + ) + + return DataFrame(internal).index + + def argmax(self) -> int: + """ + Return a maximum argument indexer. + + Parameters + ---------- + skipna : bool, default True + + Returns + ------- + maximum argument indexer + + Examples + -------- + >>> kidx = pp.Index([10, 9, 8, 7, 100, 5, 4, 3, 100, 3]) + >>> kidx + Int64Index([10, 9, 8, 7, 100, 5, 4, 3, 100, 3], dtype='int64') + + >>> kidx.argmax() + 4 + """ + sdf = self._internal.spark_frame.select(self.spark.column) + sequence_col = verify_temp_column_name(sdf, "__distributed_sequence_column__") + sdf = InternalFrame.attach_distributed_sequence_column(sdf, column_name=sequence_col) + # spark_frame here looks like below + # +-----------------+---------------+ + # |__index_level_0__|__index_value__| + # +-----------------+---------------+ + # | 0| 10| + # | 4| 100| + # | 2| 8| + # | 3| 7| + # | 6| 4| + # | 5| 5| + # | 7| 3| + # | 8| 100| + # | 1| 9| + # +-----------------+---------------+ + + return ( + sdf.orderBy( + scol_for(sdf, self._internal.data_spark_column_names[0]).desc(), + F.col(sequence_col).asc(), + ) + .select(sequence_col) + .first()[0] + ) + + def argmin(self) -> int: + """ + Return a minimum argument indexer. + + Parameters + ---------- + skipna : bool, default True + + Returns + ------- + minimum argument indexer + + Examples + -------- + >>> kidx = pp.Index([10, 9, 8, 7, 100, 5, 4, 3, 100, 3]) + >>> kidx + Int64Index([10, 9, 8, 7, 100, 5, 4, 3, 100, 3], dtype='int64') + + >>> kidx.argmin() + 7 + """ + sdf = self._internal.spark_frame.select(self.spark.column) + sequence_col = verify_temp_column_name(sdf, "__distributed_sequence_column__") + sdf = InternalFrame.attach_distributed_sequence_column(sdf, column_name=sequence_col) + + return ( + sdf.orderBy( + scol_for(sdf, self._internal.data_spark_column_names[0]).asc(), + F.col(sequence_col).asc(), + ) + .select(sequence_col) + .first()[0] + ) + + def set_names(self, names, level=None, inplace=False) -> Optional["Index"]: + """ + Set Index or MultiIndex name. + Able to set new names partially and by level. + + Parameters + ---------- + names : label or list of label + Name(s) to set. + level : int, label or list of int or label, optional + If the index is a MultiIndex, level(s) to set (None for all + levels). Otherwise level must be None. + inplace : bool, default False + Modifies the object directly, instead of creating a new Index or + MultiIndex. + + Returns + ------- + Index + The same type as the caller or None if inplace is True. + + See Also + -------- + Index.rename : Able to set new names without level. + + Examples + -------- + >>> idx = pp.Index([1, 2, 3, 4]) + >>> idx + Int64Index([1, 2, 3, 4], dtype='int64') + + >>> idx.set_names('quarter') + Int64Index([1, 2, 3, 4], dtype='int64', name='quarter') + + For MultiIndex + + >>> idx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y')]) + >>> idx # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y')], + ) + + >>> idx.set_names(['kind', 'year'], inplace=True) + >>> idx # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y')], + names=['kind', 'year']) + + >>> idx.set_names('species', level=0) # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y')], + names=['species', 'year']) + """ + from pyspark.pandas.indexes.multi import MultiIndex + + if isinstance(self, MultiIndex): + if level is not None: + self_names = self.names + self_names[level] = names + names = self_names + return self.rename(name=names, inplace=inplace) + + def difference(self, other, sort=None) -> "Index": + """ + Return a new Index with elements from the index that are not in + `other`. + + This is the set difference of two Index objects. + + Parameters + ---------- + other : Index or array-like + sort : True or None, default None + Whether to sort the resulting index. + * True : Attempt to sort the result. + * None : Do not sort the result. + + Returns + ------- + difference : Index + + Examples + -------- + + >>> idx1 = pp.Index([2, 1, 3, 4]) + >>> idx2 = pp.Index([3, 4, 5, 6]) + >>> idx1.difference(idx2, sort=True) + Int64Index([1, 2], dtype='int64') + + MultiIndex + + >>> midx1 = pp.MultiIndex.from_tuples([('a', 'x', 1), ('b', 'y', 2), ('c', 'z', 3)]) + >>> midx2 = pp.MultiIndex.from_tuples([('a', 'x', 1), ('b', 'z', 2), ('k', 'z', 3)]) + >>> midx1.difference(midx2) # doctest: +SKIP + MultiIndex([('b', 'y', 2), + ('c', 'z', 3)], + ) + """ + from pyspark.pandas.indexes.multi import MultiIndex + + if not is_list_like(other): + raise TypeError("Input must be Index or array-like") + if not isinstance(sort, (type(None), type(True))): + raise ValueError( + "The 'sort' keyword only takes the values of None or True; {} was passed.".format( + sort + ) + ) + # Handling MultiIndex + if isinstance(self, MultiIndex) and not isinstance(other, MultiIndex): + if not all([isinstance(item, tuple) for item in other]): + raise TypeError("other must be a MultiIndex or a list of tuples") + other = MultiIndex.from_tuples(other) + + if not isinstance(other, Index): + other = Index(other) + + sdf_self = self._internal.spark_frame + sdf_other = other._internal.spark_frame + idx_self = self._internal.index_spark_columns + idx_other = other._internal.index_spark_columns + sdf_diff = sdf_self.select(idx_self).subtract(sdf_other.select(idx_other)) + internal = InternalFrame( + spark_frame=sdf_diff, + index_spark_columns=[ + scol_for(sdf_diff, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + result = DataFrame(internal).index + # Name(s) will be kept when only name(s) of (Multi)Index are the same. + if isinstance(self, type(other)) and isinstance(self, MultiIndex): + if self.names == other.names: + result.names = self.names + elif isinstance(self, type(other)) and not isinstance(self, MultiIndex): + if self.name == other.name: + result.name = self.name + return result if sort is None else result.sort_values() + + @property + def is_all_dates(self) -> bool: + """ + Return if all data types of the index are datetime. + remember that since Koalas does not support multiple data types in an index, + so it returns True if any type of data is datetime. + + Examples + -------- + >>> from datetime import datetime + + >>> idx = pp.Index([datetime(2019, 1, 1, 0, 0, 0), datetime(2019, 2, 3, 0, 0, 0)]) + >>> idx + DatetimeIndex(['2019-01-01', '2019-02-03'], dtype='datetime64[ns]', freq=None) + + >>> idx.is_all_dates + True + + >>> idx = pp.Index([datetime(2019, 1, 1, 0, 0, 0), None]) + >>> idx + DatetimeIndex(['2019-01-01', 'NaT'], dtype='datetime64[ns]', freq=None) + + >>> idx.is_all_dates + True + + >>> idx = pp.Index([0, 1, 2]) + >>> idx + Int64Index([0, 1, 2], dtype='int64') + + >>> idx.is_all_dates + False + """ + return isinstance(self.spark.data_type, TimestampType) + + def repeat(self, repeats: int) -> "Index": + """ + Repeat elements of a Index/MultiIndex. + + Returns a new Index/MultiIndex where each element of the current Index/MultiIndex + is repeated consecutively a given number of times. + + Parameters + ---------- + repeats : int + The number of repetitions for each element. This should be a + non-negative integer. Repeating 0 times will return an empty + Index. + + Returns + ------- + repeated_index : Index/MultiIndex + Newly created Index/MultiIndex with repeated elements. + + See Also + -------- + Series.repeat : Equivalent function for Series. + + Examples + -------- + >>> idx = pp.Index(['a', 'b', 'c']) + >>> idx + Index(['a', 'b', 'c'], dtype='object') + >>> idx.repeat(2) + Index(['a', 'b', 'c', 'a', 'b', 'c'], dtype='object') + + For MultiIndex, + + >>> midx = pp.MultiIndex.from_tuples([('x', 'a'), ('x', 'b'), ('y', 'c')]) + >>> midx # doctest: +SKIP + MultiIndex([('x', 'a'), + ('x', 'b'), + ('y', 'c')], + ) + >>> midx.repeat(2) # doctest: +SKIP + MultiIndex([('x', 'a'), + ('x', 'b'), + ('y', 'c'), + ('x', 'a'), + ('x', 'b'), + ('y', 'c')], + ) + >>> midx.repeat(0) # doctest: +SKIP + MultiIndex([], ) + """ + if not isinstance(repeats, int): + raise ValueError( + "`repeats` argument must be integer, but got {}".format(type(repeats).__name__) + ) + elif repeats < 0: + raise ValueError("negative dimensions are not allowed") + + kdf = DataFrame(self._internal.resolved_copy) # type: DataFrame + if repeats == 0: + return DataFrame(kdf._internal.with_filter(F.lit(False))).index + else: + return pp.concat([kdf] * repeats).index + + def asof(self, label) -> Scalar: + """ + Return the label from the index, or, if not present, the previous one. + + Assuming that the index is sorted, return the passed index label if it + is in the index, or return the previous index label if the passed one + is not in the index. + + .. note:: This API is dependent on :meth:`Index.is_monotonic_increasing` + which can be expensive. + + Parameters + ---------- + label : object + The label up to which the method returns the latest index label. + + Returns + ------- + object + The passed label if it is in the index. The previous label if the + passed label is not in the sorted index or `NaN` if there is no + such label. + + Examples + -------- + `Index.asof` returns the latest index label up to the passed label. + + >>> idx = pp.Index(['2013-12-31', '2014-01-02', '2014-01-03']) + >>> idx.asof('2014-01-01') + '2013-12-31' + + If the label is in the index, the method returns the passed label. + + >>> idx.asof('2014-01-02') + '2014-01-02' + + If all of the labels in the index are later than the passed label, + NaN is returned. + + >>> idx.asof('1999-01-02') + nan + """ + sdf = self._internal.spark_frame + if self.is_monotonic_increasing: + sdf = sdf.where(self.spark.column <= F.lit(label).cast(self.spark.data_type)).select( + F.max(self.spark.column) + ) + elif self.is_monotonic_decreasing: + sdf = sdf.where(self.spark.column >= F.lit(label).cast(self.spark.data_type)).select( + F.min(self.spark.column) + ) + else: + raise ValueError("index must be monotonic increasing or decreasing") + + result = sdf.toPandas().iloc[0, 0] + return result if result is not None else np.nan + + def union(self, other, sort=None) -> "Index": + """ + Form the union of two Index objects. + + Parameters + ---------- + other : Index or array-like + sort : bool or None, default None + Whether to sort the resulting Index. + + Returns + ------- + union : Index + + Examples + -------- + + Index + + >>> idx1 = pp.Index([1, 2, 3, 4]) + >>> idx2 = pp.Index([3, 4, 5, 6]) + >>> idx1.union(idx2).sort_values() + Int64Index([1, 2, 3, 4, 5, 6], dtype='int64') + + MultiIndex + + >>> midx1 = pp.MultiIndex.from_tuples([("x", "a"), ("x", "b"), ("x", "c"), ("x", "d")]) + >>> midx2 = pp.MultiIndex.from_tuples([("x", "c"), ("x", "d"), ("x", "e"), ("x", "f")]) + >>> midx1.union(midx2).sort_values() # doctest: +SKIP + MultiIndex([('x', 'a'), + ('x', 'b'), + ('x', 'c'), + ('x', 'd'), + ('x', 'e'), + ('x', 'f')], + ) + """ + from pyspark.pandas.indexes.multi import MultiIndex + + sort = True if sort is None else sort + sort = validate_bool_kwarg(sort, "sort") + if type(self) is not type(other): + if isinstance(self, MultiIndex): + if not isinstance(other, list) or not all( + [isinstance(item, tuple) for item in other] + ): + raise TypeError("other must be a MultiIndex or a list of tuples") + other = MultiIndex.from_tuples(other) + else: + if isinstance(other, MultiIndex): + # TODO: We can't support different type of values in a single column for now. + raise NotImplementedError( + "Union between Index and MultiIndex is not yet supported" + ) + elif isinstance(other, Series): + other = other.to_frame() + other = other.set_index(other.columns[0]).index + elif isinstance(other, DataFrame): + raise ValueError("Index data must be 1-dimensional") + else: + other = Index(other) + sdf_self = self._internal.spark_frame.select(self._internal.index_spark_columns) + sdf_other = other._internal.spark_frame.select(other._internal.index_spark_columns) + sdf = sdf_self.union(sdf_other.subtract(sdf_self)) + if isinstance(self, MultiIndex): + sdf = sdf.drop_duplicates() + if sort: + sdf = sdf.sort(self._internal.index_spark_column_names) + internal = InternalFrame( # TODO: dtypes? + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + ) + + return DataFrame(internal).index + + def holds_integer(self) -> bool: + """ + Whether the type is an integer type. + Always return False for MultiIndex. + + Notes + ----- + When Index contains null values the result can be different with pandas + since Koalas cast integer to float when Index contains null values. + + >>> pp.Index([1, 2, 3, None]) + Float64Index([1.0, 2.0, 3.0, nan], dtype='float64') + + Examples + -------- + >>> kidx = pp.Index([1, 2, 3, 4]) + >>> kidx.holds_integer() + True + + Returns False for string type. + + >>> kidx = pp.Index(["A", "B", "C", "D"]) + >>> kidx.holds_integer() + False + + Returns False for float type. + + >>> kidx = pp.Index([1.1, 2.2, 3.3, 4.4]) + >>> kidx.holds_integer() + False + """ + return isinstance(self.spark.data_type, IntegralType) + + def intersection(self, other) -> "Index": + """ + Form the intersection of two Index objects. + + This returns a new Index with elements common to the index and `other`. + + Parameters + ---------- + other : Index or array-like + + Returns + ------- + intersection : Index + + Examples + -------- + >>> idx1 = pp.Index([1, 2, 3, 4]) + >>> idx2 = pp.Index([3, 4, 5, 6]) + >>> idx1.intersection(idx2).sort_values() + Int64Index([3, 4], dtype='int64') + """ + from pyspark.pandas.indexes.multi import MultiIndex + + if isinstance(other, DataFrame): + raise ValueError("Index data must be 1-dimensional") + elif isinstance(other, MultiIndex): + # Always returns a no-named empty Index if `other` is MultiIndex. + return self._kdf.head(0).index.rename(None) + elif isinstance(other, Index): + spark_frame_other = other.to_frame().to_spark() + keep_name = self.name == other.name + elif isinstance(other, Series): + spark_frame_other = other.to_frame().to_spark() + keep_name = True + elif is_list_like(other): + other = Index(other) + if isinstance(other, MultiIndex): + return other.to_frame().head(0).index + spark_frame_other = other.to_frame().to_spark() + keep_name = True + else: + raise TypeError("Input must be Index or array-like") + + spark_frame_self = self.to_frame(name=SPARK_DEFAULT_INDEX_NAME).to_spark() + spark_frame_intersected = spark_frame_self.intersect(spark_frame_other) + if keep_name: + index_names = self._internal.index_names + else: + index_names = None + internal = InternalFrame( # TODO: dtypes? + spark_frame=spark_frame_intersected, + index_spark_columns=[scol_for(spark_frame_intersected, SPARK_DEFAULT_INDEX_NAME)], + index_names=index_names, + ) + + return DataFrame(internal).index + + def item(self) -> Union[Scalar, Tuple[Scalar, ...]]: + """ + Return the first element of the underlying data as a python scalar. + + Returns + ------- + scalar + The first element of Index. + + Raises + ------ + ValueError + If the data is not length-1. + + Examples + -------- + >>> kidx = pp.Index([10]) + >>> kidx.item() + 10 + """ + return self.to_series().item() + + def insert(self, loc: int, item) -> "Index": + """ + Make new Index inserting new item at location. + + Follows Python list.append semantics for negative values. + + Parameters + ---------- + loc : int + item : object + + Returns + ------- + new_index : Index + + Examples + -------- + >>> kidx = pp.Index([1, 2, 3, 4, 5]) + >>> kidx.insert(3, 100) + Int64Index([1, 2, 3, 100, 4, 5], dtype='int64') + + For negative values + + >>> kidx = pp.Index([1, 2, 3, 4, 5]) + >>> kidx.insert(-3, 100) + Int64Index([1, 2, 100, 3, 4, 5], dtype='int64') + """ + if loc < 0: + length = len(self) + loc = loc + length + loc = 0 if loc < 0 else loc + + index_name = self._internal.index_spark_column_names[0] + sdf_before = self.to_frame(name=index_name)[:loc].to_spark() + sdf_middle = Index([item]).to_frame(name=index_name).to_spark() + sdf_after = self.to_frame(name=index_name)[loc:].to_spark() + sdf = sdf_before.union(sdf_middle).union(sdf_after) + + internal = self._internal.with_new_sdf(sdf) # TODO: dtype? + return DataFrame(internal).index + + def view(self) -> "Index": + """ + this is defined as a copy with the same identity + """ + return self.copy() + + def to_list(self) -> List: + """ + Return a list of the values. + + These are each a scalar type, which is a Python scalar + (for str, int, float) or a pandas scalar + (for Timestamp/Timedelta/Interval/Period) + + .. note:: This method should only be used if the resulting list is expected + to be small, as all the data is loaded into the driver's memory. + + Examples + -------- + Index + + >>> idx = pp.Index([1, 2, 3, 4, 5]) + >>> idx.to_list() + [1, 2, 3, 4, 5] + + MultiIndex + + >>> tuples = [(1, 'red'), (1, 'blue'), (2, 'red'), (2, 'green')] + >>> midx = pp.MultiIndex.from_tuples(tuples) + >>> midx.to_list() + [(1, 'red'), (1, 'blue'), (2, 'red'), (2, 'green')] + """ + return self._to_internal_pandas().tolist() + + tolist = to_list + + @property + def inferred_type(self) -> str: + """ + Return a string of the type inferred from the values. + + Examples + -------- + >>> from datetime import datetime + >>> pp.Index([1, 2, 3]).inferred_type + 'integer' + + >>> pp.Index([1.0, 2.0, 3.0]).inferred_type + 'floating' + + >>> pp.Index(['a', 'b', 'c']).inferred_type + 'string' + + >>> pp.Index([True, False, True, False]).inferred_type + 'boolean' + """ + return lib.infer_dtype([self.to_series().head(1).item()]) + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeIndex, item): + property_or_func = getattr(MissingPandasLikeIndex, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError("'{}' object has no attribute '{}'".format(type(self).__name__, item)) + + def __repr__(self): + max_display_count = get_option("display.max_rows") + if max_display_count is None: + return repr(self._to_internal_pandas()) + + pindex = self._kdf._get_or_create_repr_pandas_cache(max_display_count).index + + pindex_length = len(pindex) + repr_string = repr(pindex[:max_display_count]) + + if pindex_length > max_display_count: + footer = "\nShowing only the first {}".format(max_display_count) + return repr_string + footer + return repr_string + + def __iter__(self): + return MissingPandasLikeIndex.__iter__(self) + + def __xor__(self, other): + return self.symmetric_difference(other) + + def __bool__(self): + raise ValueError( + "The truth value of a {0} is ambiguous. " + "Use a.empty, a.bool(), a.item(), a.any() or a.all().".format(self.__class__.__name__) + ) diff --git a/python/pyspark/pandas/indexes/category.py b/python/pyspark/pandas/indexes/category.py new file mode 100644 index 0000000000..8bdec59858 --- /dev/null +++ b/python/pyspark/pandas/indexes/category.py @@ -0,0 +1,188 @@ +# +# 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. +# +from functools import partial +from typing import Any + +import pandas as pd +from pandas.api.types import is_hashable + +from pyspark import pandas as pp +from pyspark.pandas.indexes.base import Index +from pyspark.pandas.missing.indexes import MissingPandasLikeCategoricalIndex +from pyspark.pandas.series import Series + + +class CategoricalIndex(Index): + """ + Index based on an underlying `Categorical`. + + CategoricalIndex can only take on a limited, + and usually fixed, number of possible values (`categories`). Also, + it might have an order, but numerical operations + (additions, divisions, ...) are not possible. + + Parameters + ---------- + data : array-like (1-dimensional) + The values of the categorical. If `categories` are given, values not in + `categories` will be replaced with NaN. + categories : index-like, optional + The categories for the categorical. Items need to be unique. + If the categories are not given here (and also not in `dtype`), they + will be inferred from the `data`. + ordered : bool, optional + Whether or not this categorical is treated as an ordered + categorical. If not given here or in `dtype`, the resulting + categorical will be unordered. + dtype : CategoricalDtype or "category", optional + If :class:`CategoricalDtype`, cannot be used together with + `categories` or `ordered`. + copy : bool, default False + Make a copy of input ndarray. + name : object, optional + Name to be stored in the index. + + See Also + -------- + Index : The base Koalas Index type. + + Examples + -------- + >>> pp.CategoricalIndex(["a", "b", "c", "a", "b", "c"]) # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'c', 'a', 'b', 'c'], + categories=['a', 'b', 'c'], ordered=False, dtype='category') + + ``CategoricalIndex`` can also be instantiated from a ``Categorical``: + + >>> c = pd.Categorical(["a", "b", "c", "a", "b", "c"]) + >>> pp.CategoricalIndex(c) # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'c', 'a', 'b', 'c'], + categories=['a', 'b', 'c'], ordered=False, dtype='category') + + Ordered ``CategoricalIndex`` can have a min and max value. + + >>> ci = pp.CategoricalIndex( + ... ["a", "b", "c", "a", "b", "c"], ordered=True, categories=["c", "b", "a"] + ... ) + >>> ci # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'c', 'a', 'b', 'c'], + categories=['c', 'b', 'a'], ordered=True, dtype='category') + + From a Series: + + >>> s = pp.Series(["a", "b", "c", "a", "b", "c"], index=[10, 20, 30, 40, 50, 60]) + >>> pp.CategoricalIndex(s) # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'c', 'a', 'b', 'c'], + categories=['a', 'b', 'c'], ordered=False, dtype='category') + + From an Index: + + >>> idx = pp.Index(["a", "b", "c", "a", "b", "c"]) + >>> pp.CategoricalIndex(idx) # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'c', 'a', 'b', 'c'], + categories=['a', 'b', 'c'], ordered=False, dtype='category') + """ + + def __new__(cls, data=None, categories=None, ordered=None, dtype=None, copy=False, name=None): + if not is_hashable(name): + raise TypeError("Index.name must be a hashable type") + + if isinstance(data, (Series, Index)): + if dtype is None: + dtype = "category" + return Index(data, dtype=dtype, copy=copy, name=name) + + return pp.from_pandas( + pd.CategoricalIndex( + data=data, categories=categories, ordered=ordered, dtype=dtype, name=name + ) + ) + + @property + def codes(self) -> Index: + """ + The category codes of this categorical. + + Codes are an Index of integers which are the positions of the actual + values in the categories Index. + + There is no setter, use the other categorical methods and the normal item + setter to change values in the categorical. + + Returns + ------- + Index + A non-writable view of the `codes` Index. + + Examples + -------- + >>> idx = pp.CategoricalIndex(list("abbccc")) + >>> idx # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'b', 'c', 'c', 'c'], + categories=['a', 'b', 'c'], ordered=False, dtype='category') + + >>> idx.codes + Int64Index([0, 1, 1, 2, 2, 2], dtype='int64') + """ + return self._with_new_scol(self.spark.column).rename(None) + + @property + def categories(self) -> pd.Index: + """ + The categories of this categorical. + + Examples + -------- + >>> idx = pp.CategoricalIndex(list("abbccc")) + >>> idx # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'b', 'c', 'c', 'c'], + categories=['a', 'b', 'c'], ordered=False, dtype='category') + + >>> idx.categories + Index(['a', 'b', 'c'], dtype='object') + """ + return self.dtype.categories + + @categories.setter + def categories(self, categories): + raise NotImplementedError() + + @property + def ordered(self) -> bool: + """ + Whether the categories have an ordered relationship. + + Examples + -------- + >>> idx = pp.CategoricalIndex(list("abbccc")) + >>> idx # doctest: +NORMALIZE_WHITESPACE + CategoricalIndex(['a', 'b', 'b', 'c', 'c', 'c'], + categories=['a', 'b', 'c'], ordered=False, dtype='category') + + >>> idx.ordered + False + """ + return self.dtype.ordered + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeCategoricalIndex, item): + property_or_func = getattr(MissingPandasLikeCategoricalIndex, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError("'CategoricalIndex' object has no attribute '{}'".format(item)) diff --git a/python/pyspark/pandas/indexes/datetimes.py b/python/pyspark/pandas/indexes/datetimes.py new file mode 100644 index 0000000000..88ba7d224a --- /dev/null +++ b/python/pyspark/pandas/indexes/datetimes.py @@ -0,0 +1,742 @@ +# +# 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 datetime +from functools import partial +from typing import Any, Optional, Union + +import pandas as pd +from pandas.api.types import is_hashable +from pyspark._globals import _NoValue + +from pyspark import pandas as pp +from pyspark.pandas.indexes.base import Index +from pyspark.pandas.missing.indexes import MissingPandasLikeDatetimeIndex +from pyspark.pandas.series import Series, first_series +from pyspark.pandas.utils import verify_temp_column_name + + +class DatetimeIndex(Index): + """ + Immutable ndarray-like of datetime64 data. + + Parameters + ---------- + data : array-like (1-dimensional), optional + Optional datetime-like data to construct index with. + freq : str or pandas offset object, optional + One of pandas date offset strings or corresponding objects. The string + 'infer' can be passed in order to set the frequency of the index as the + inferred frequency upon creation. + normalize : bool, default False + Normalize start/end dates to midnight before generating date range. + closed : {'left', 'right'}, optional + Set whether to include `start` and `end` that are on the + boundary. The default includes boundary points on either end. + ambiguous : 'infer', bool-ndarray, 'NaT', default 'raise' + When clocks moved backward due to DST, ambiguous times may arise. + For example in Central European Time (UTC+01), when going from 03:00 + DST to 02:00 non-DST, 02:30:00 local time occurs both at 00:30:00 UTC + and at 01:30:00 UTC. In such a situation, the `ambiguous` parameter + dictates how ambiguous times should be handled. + + - 'infer' will attempt to infer fall dst-transition hours based on + order + - bool-ndarray where True signifies a DST time, False signifies a + non-DST time (note that this flag is only applicable for ambiguous + times) + - 'NaT' will return NaT where there are ambiguous times + - 'raise' will raise an AmbiguousTimeError if there are ambiguous times. + dayfirst : bool, default False + If True, parse dates in `data` with the day first order. + yearfirst : bool, default False + If True parse dates in `data` with the year first order. + dtype : numpy.dtype or str, default None + Note that the only NumPy dtype allowed is ‘datetime64[ns]’. + copy : bool, default False + Make a copy of input ndarray. + name : label, default None + Name to be stored in the index. + + See Also + -------- + Index : The base pandas Index type. + to_datetime : Convert argument to datetime. + + Examples + -------- + >>> pp.DatetimeIndex(['1970-01-01', '1970-01-01', '1970-01-01']) + DatetimeIndex(['1970-01-01', '1970-01-01', '1970-01-01'], dtype='datetime64[ns]', freq=None) + + From a Series: + + >>> from datetime import datetime + >>> s = pp.Series([datetime(2021, 3, 1), datetime(2021, 3, 2)], index=[10, 20]) + >>> pp.DatetimeIndex(s) + DatetimeIndex(['2021-03-01', '2021-03-02'], dtype='datetime64[ns]', freq=None) + + From an Index: + + >>> idx = pp.DatetimeIndex(['1970-01-01', '1970-01-01', '1970-01-01']) + >>> pp.DatetimeIndex(idx) + DatetimeIndex(['1970-01-01', '1970-01-01', '1970-01-01'], dtype='datetime64[ns]', freq=None) + """ + + def __new__( + cls, + data=None, + freq=_NoValue, + normalize=False, + closed=None, + ambiguous="raise", + dayfirst=False, + yearfirst=False, + dtype=None, + copy=False, + name=None, + ): + if not is_hashable(name): + raise TypeError("Index.name must be a hashable type") + + if isinstance(data, (Series, Index)): + if dtype is None: + dtype = "datetime64[ns]" + return Index(data, dtype=dtype, copy=copy, name=name) + + kwargs = dict( + data=data, + normalize=normalize, + closed=closed, + ambiguous=ambiguous, + dayfirst=dayfirst, + yearfirst=yearfirst, + dtype=dtype, + copy=copy, + name=name, + ) + if freq is not _NoValue: + kwargs["freq"] = freq + return pp.from_pandas(pd.DatetimeIndex(**kwargs)) + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeDatetimeIndex, item): + property_or_func = getattr(MissingPandasLikeDatetimeIndex, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError("'DatetimeIndex' object has no attribute '{}'".format(item)) + + # Properties + @property + def year(self) -> Index: + """ + The year of the datetime. + """ + return Index(self.to_series().dt.year) + + @property + def month(self) -> Index: + """ + The month of the timestamp as January = 1 December = 12. + """ + return Index(self.to_series().dt.month) + + @property + def day(self) -> Index: + """ + The days of the datetime. + """ + return Index(self.to_series().dt.day) + + @property + def hour(self) -> Index: + """ + The hours of the datetime. + """ + return Index(self.to_series().dt.hour) + + @property + def minute(self) -> Index: + """ + The minutes of the datetime. + """ + return Index(self.to_series().dt.minute) + + @property + def second(self) -> Index: + """ + The seconds of the datetime. + """ + return Index(self.to_series().dt.second) + + @property + def microsecond(self) -> Index: + """ + The microseconds of the datetime. + """ + return Index(self.to_series().dt.microsecond) + + @property + def week(self) -> Index: + """ + The week ordinal of the year. + """ + return Index(self.to_series().dt.week) + + @property + def weekofyear(self) -> Index: + return Index(self.to_series().dt.weekofyear) + + weekofyear.__doc__ = week.__doc__ + + @property + def dayofweek(self) -> Index: + """ + The day of the week with Monday=0, Sunday=6. + Return the day of the week. It is assumed the week starts on + Monday, which is denoted by 0 and ends on Sunday which is denoted + by 6. This method is available on both Series with datetime + values (using the `dt` accessor) or DatetimeIndex. + + Returns + ------- + Series or Index + Containing integers indicating the day number. + + See Also + -------- + Series.dt.dayofweek : Alias. + Series.dt.weekday : Alias. + Series.dt.day_name : Returns the name of the day of the week. + + Examples + -------- + >>> idx = pp.date_range('2016-12-31', '2017-01-08', freq='D') + >>> idx.dayofweek + Int64Index([5, 6, 0, 1, 2, 3, 4, 5, 6], dtype='int64') + """ + return Index(self.to_series().dt.dayofweek) + + @property + def day_of_week(self) -> Index: + return self.dayofweek + + day_of_week.__doc__ = dayofweek.__doc__ + + @property + def weekday(self) -> Index: + return Index(self.to_series().dt.weekday) + + weekday.__doc__ = dayofweek.__doc__ + + @property + def dayofyear(self) -> Index: + """ + The ordinal day of the year. + """ + return Index(self.to_series().dt.dayofyear) + + @property + def day_of_year(self) -> Index: + return self.dayofyear + + day_of_year.__doc__ = dayofyear.__doc__ + + @property + def quarter(self) -> Index: + """ + The quarter of the date. + """ + return Index(self.to_series().dt.quarter) + + @property + def is_month_start(self) -> Index: + """ + Indicates whether the date is the first day of the month. + + Returns + ------- + Index + Returns a Index with boolean values + + See Also + -------- + is_month_end : Return a boolean indicating whether the date + is the last day of the month. + + Examples + -------- + >>> idx = pp.date_range("2018-02-27", periods=3) + >>> idx.is_month_start + Index([False, False, True], dtype='object') + """ + return Index(self.to_series().dt.is_month_start) + + @property + def is_month_end(self) -> Index: + """ + Indicates whether the date is the last day of the month. + + Returns + ------- + Index + Returns a Index with boolean values. + + See Also + -------- + is_month_start : Return a boolean indicating whether the date + is the first day of the month. + + Examples + -------- + >>> idx = pp.date_range("2018-02-27", periods=3) + >>> idx.is_month_end + Index([False, True, False], dtype='object') + """ + return Index(self.to_series().dt.is_month_end) + + @property + def is_quarter_start(self) -> Index: + """ + Indicator for whether the date is the first day of a quarter. + + Returns + ------- + is_quarter_start : Index + Returns an Index with boolean values. + + See Also + -------- + quarter : Return the quarter of the date. + is_quarter_end : Similar property for indicating the quarter start. + + Examples + -------- + >>> idx = pp.date_range('2017-03-30', periods=4) + >>> idx.is_quarter_start + Index([False, False, True, False], dtype='object') + """ + return Index(self.to_series().dt.is_quarter_start) + + @property + def is_quarter_end(self) -> Index: + """ + Indicator for whether the date is the last day of a quarter. + + Returns + ------- + is_quarter_end : Index + Returns an Index with boolean values. + + See Also + -------- + quarter : Return the quarter of the date. + is_quarter_start : Similar property indicating the quarter start. + + Examples + -------- + >>> idx = pp.date_range('2017-03-30', periods=4) + >>> idx.is_quarter_end + Index([False, True, False, False], dtype='object') + """ + return Index(self.to_series().dt.is_quarter_end) + + @property + def is_year_start(self) -> Index: + """ + Indicate whether the date is the first day of a year. + + Returns + ------- + Index + Returns an Index with boolean values. + + See Also + -------- + is_year_end : Similar property indicating the last day of the year. + + Examples + -------- + >>> idx = pp.date_range("2017-12-30", periods=3) + >>> idx.is_year_start + Index([False, False, True], dtype='object') + """ + return Index(self.to_series().dt.is_year_start) + + @property + def is_year_end(self) -> Index: + """ + Indicate whether the date is the last day of the year. + + Returns + ------- + Index + Returns an Index with boolean values. + + See Also + -------- + is_year_start : Similar property indicating the start of the year. + + Examples + -------- + >>> idx = pp.date_range("2017-12-30", periods=3) + >>> idx.is_year_end + Index([False, True, False], dtype='object') + """ + return Index(self.to_series().dt.is_year_end) + + @property + def is_leap_year(self) -> Index: + """ + Boolean indicator if the date belongs to a leap year. + + A leap year is a year, which has 366 days (instead of 365) including + 29th of February as an intercalary day. + Leap years are years which are multiples of four with the exception + of years divisible by 100 but not by 400. + + Returns + ------- + Index + Booleans indicating if dates belong to a leap year. + + Examples + -------- + >>> idx = pp.date_range("2012-01-01", "2015-01-01", freq="Y") + >>> idx.is_leap_year + Index([True, False, False], dtype='object') + """ + return Index(self.to_series().dt.is_leap_year) + + @property + def daysinmonth(self) -> Index: + """ + The number of days in the month. + """ + return Index(self.to_series().dt.daysinmonth) + + @property + def days_in_month(self) -> Index: + return Index(self.to_series().dt.days_in_month) + + days_in_month.__doc__ = daysinmonth.__doc__ + + # Methods + def ceil(self, freq, *args, **kwargs) -> "DatetimeIndex": + """ + Perform ceil operation on the data to the specified freq. + + Parameters + ---------- + freq : str or Offset + The frequency level to ceil the index to. Must be a fixed + frequency like 'S' (second) not 'ME' (month end). + + Returns + ------- + DatetimeIndex + + Raises + ------ + ValueError if the `freq` cannot be converted. + + Examples + -------- + >>> rng = pp.date_range('1/1/2018 11:59:00', periods=3, freq='min') + >>> rng.ceil('H') # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-01 12:00:00', '2018-01-01 12:00:00', + '2018-01-01 13:00:00'], + dtype='datetime64[ns]', freq=None) + """ + disallow_nanoseconds(freq) + + return DatetimeIndex(self.to_series().dt.ceil(freq, *args, **kwargs)) + + def floor(self, freq, *args, **kwargs) -> "DatetimeIndex": + """ + Perform floor operation on the data to the specified freq. + + Parameters + ---------- + freq : str or Offset + The frequency level to floor the index to. Must be a fixed + frequency like 'S' (second) not 'ME' (month end). + + Returns + ------- + DatetimeIndex + + Raises + ------ + ValueError if the `freq` cannot be converted. + + Examples + -------- + >>> rng = pp.date_range('1/1/2018 11:59:00', periods=3, freq='min') + >>> rng.floor("H") # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-01 11:00:00', '2018-01-01 12:00:00', + '2018-01-01 12:00:00'], + dtype='datetime64[ns]', freq=None) + """ + disallow_nanoseconds(freq) + + return DatetimeIndex(self.to_series().dt.floor(freq, *args, **kwargs)) + + def round(self, freq, *args, **kwargs) -> "DatetimeIndex": + """ + Perform round operation on the data to the specified freq. + + Parameters + ---------- + freq : str or Offset + The frequency level to round the index to. Must be a fixed + frequency like 'S' (second) not 'ME' (month end). + + Returns + ------- + DatetimeIndex + + Raises + ------ + ValueError if the `freq` cannot be converted. + + Examples + -------- + >>> rng = pp.date_range('1/1/2018 11:59:00', periods=3, freq='min') + >>> rng.round("H") # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-01 12:00:00', '2018-01-01 12:00:00', + '2018-01-01 12:00:00'], + dtype='datetime64[ns]', freq=None) + """ + disallow_nanoseconds(freq) + + return DatetimeIndex(self.to_series().dt.round(freq, *args, **kwargs)) + + def month_name(self, locale: Optional[str] = None) -> Index: + """ + Return the month names of the DatetimeIndex with specified locale. + + Parameters + ---------- + locale : str, optional + Locale determining the language in which to return the month name. + Default is English locale. + + Returns + ------- + Index + Index of month names. + + Examples + -------- + >>> idx = pp.date_range(start='2018-01', freq='M', periods=3) + >>> idx.month_name() + Index(['January', 'February', 'March'], dtype='object') + """ + return Index(self.to_series().dt.month_name(locale)) + + def day_name(self, locale: Optional[str] = None) -> Index: + """ + Return the day names of the series with specified locale. + + Parameters + ---------- + locale : str, optional + Locale determining the language in which to return the day name. + Default is English locale. + + Returns + ------- + Index + Index of day names. + + Examples + -------- + >>> idx = pp.date_range(start='2018-01-01', freq='D', periods=3) + >>> idx.day_name() + Index(['Monday', 'Tuesday', 'Wednesday'], dtype='object') + """ + return Index(self.to_series().dt.day_name(locale)) + + def normalize(self) -> "DatetimeIndex": + """ + Convert times to midnight. + + The time component of the date-time is converted to midnight i.e. + 00:00:00. This is useful in cases, when the time does not matter. + Length is unaltered. The timezones are unaffected. + + This method is available on Series with datetime values under + the ``.dt`` accessor. + + Returns + ------- + DatetimeIndex + The same type as the original data. + + See Also + -------- + floor : Floor the series to the specified freq. + ceil : Ceil the series to the specified freq. + round : Round the series to the specified freq. + + Examples + -------- + >>> idx = pp.date_range(start='2014-08-01 10:00', freq='H', periods=3) + >>> idx.normalize() + DatetimeIndex(['2014-08-01', '2014-08-01', '2014-08-01'], dtype='datetime64[ns]', freq=None) + """ + return DatetimeIndex(self.to_series().dt.normalize()) + + def strftime(self, date_format: str) -> Index: + """ + Convert to a string Index using specified date_format. + + Return an Index of formatted strings specified by date_format, which + supports the same string format as the python standard library. Details + of the string format can be found in python string format + doc. + + Parameters + ---------- + date_format : str + Date format string (e.g. "%%Y-%%m-%%d"). + + Returns + ------- + Index + Index of formatted strings. + + See Also + -------- + normalize : Return series with times to midnight. + round : Round the series to the specified freq. + floor : Floor the series to the specified freq. + + Examples + -------- + >>> idx = pp.date_range(pd.Timestamp("2018-03-10 09:00"), periods=3, freq='s') + >>> idx.strftime('%B %d, %Y, %r') # doctest: +NORMALIZE_WHITESPACE + Index(['March 10, 2018, 09:00:00 AM', 'March 10, 2018, 09:00:01 AM', + 'March 10, 2018, 09:00:02 AM'], + dtype='object') + """ + return Index(self.to_series().dt.strftime(date_format)) + + def indexer_between_time( + self, + start_time: Union[datetime.time, str], + end_time: Union[datetime.time, str], + include_start: bool = True, + include_end: bool = True, + ) -> Index: + """ + Return index locations of values between particular times of day + (e.g., 9:00-9:30AM). + + Parameters + ---------- + start_time, end_time : datetime.time, str + Time passed either as object (datetime.time) or as string in + appropriate format ("%H:%M", "%H%M", "%I:%M%p", "%I%M%p", + "%H:%M:%S", "%H%M%S", "%I:%M:%S%p","%I%M%S%p"). + include_start : bool, default True + include_end : bool, default True + + Returns + ------- + values_between_time : Index of integers + + Examples + -------- + >>> kidx = pp.date_range("2000-01-01", periods=3, freq="T") + >>> kidx # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2000-01-01 00:00:00', '2000-01-01 00:01:00', + '2000-01-01 00:02:00'], + dtype='datetime64[ns]', freq=None) + + >>> kidx.indexer_between_time("00:01", "00:02").sort_values() + Int64Index([1, 2], dtype='int64') + + >>> kidx.indexer_between_time("00:01", "00:02", include_end=False) + Int64Index([1], dtype='int64') + + >>> kidx.indexer_between_time("00:01", "00:02", include_start=False) + Int64Index([2], dtype='int64') + """ + + def pandas_between_time(pdf) -> pp.DataFrame[int]: + return pdf.between_time(start_time, end_time, include_start, include_end) + + kdf = self.to_frame()[[]] + id_column_name = verify_temp_column_name(kdf, "__id_column__") + kdf = kdf.koalas.attach_id_column("distributed-sequence", id_column_name) + with pp.option_context("compute.default_index_type", "distributed"): + # The attached index in the statement below will be dropped soon, + # so we enforce “distributed” default index type + kdf = kdf.koalas.apply_batch(pandas_between_time) + return pp.Index(first_series(kdf).rename(self.name)) + + def indexer_at_time(self, time: Union[datetime.time, str], asof: bool = False) -> Index: + """ + Return index locations of values at particular time of day + (e.g. 9:30AM). + + Parameters + ---------- + time : datetime.time or str + Time passed in either as object (datetime.time) or as string in + appropriate format ("%H:%M", "%H%M", "%I:%M%p", "%I%M%p", + "%H:%M:%S", "%H%M%S", "%I:%M:%S%p", "%I%M%S%p"). + + Returns + ------- + values_at_time : Index of integers + + Examples + -------- + >>> kidx = pp.date_range("2000-01-01", periods=3, freq="T") + >>> kidx # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2000-01-01 00:00:00', '2000-01-01 00:01:00', + '2000-01-01 00:02:00'], + dtype='datetime64[ns]', freq=None) + + >>> kidx.indexer_at_time("00:00") + Int64Index([0], dtype='int64') + + >>> kidx.indexer_at_time("00:01") + Int64Index([1], dtype='int64') + """ + if asof: + raise NotImplementedError("'asof' argument is not supported") + + def pandas_at_time(pdf) -> pp.DataFrame[int]: + return pdf.at_time(time, asof) + + kdf = self.to_frame()[[]] + id_column_name = verify_temp_column_name(kdf, "__id_column__") + kdf = kdf.koalas.attach_id_column("distributed-sequence", id_column_name) + with pp.option_context("compute.default_index_type", "distributed"): + # The attached index in the statement below will be dropped soon, + # so we enforce “distributed” default index type + kdf = kdf.koalas.apply_batch(pandas_at_time) + return pp.Index(first_series(kdf).rename(self.name)) + + +def disallow_nanoseconds(freq): + if freq in ["N", "ns"]: + raise ValueError("nanoseconds is not supported") diff --git a/python/pyspark/pandas/indexes/multi.py b/python/pyspark/pandas/indexes/multi.py new file mode 100644 index 0000000000..ee8ea1e235 --- /dev/null +++ b/python/pyspark/pandas/indexes/multi.py @@ -0,0 +1,1170 @@ +# +# 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. +# + +from distutils.version import LooseVersion +from functools import partial +from typing import Any, Optional, Tuple, Union, cast +import warnings + +import pandas as pd +from pandas.api.types import is_list_like +from pandas.api.types import is_hashable + +import pyspark +from pyspark import sql as spark +from pyspark.sql import functions as F, Window + +# For running doctests and reference resolution in PyCharm. +from pyspark import pandas as pp # noqa: F401 +from pyspark.pandas.exceptions import PandasNotImplementedError +from pyspark.pandas.base import IndexOpsMixin +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.indexes.base import Index +from pyspark.pandas.missing.indexes import MissingPandasLikeMultiIndex +from pyspark.pandas.series import Series, first_series +from pyspark.pandas.utils import ( + compare_disallow_null, + default_session, + is_name_like_tuple, + name_like_string, + scol_for, + verify_temp_column_name, +) +from pyspark.pandas.internal import ( + InternalFrame, + NATURAL_ORDER_COLUMN_NAME, + SPARK_INDEX_NAME_FORMAT, +) +from pyspark.pandas.typedef import Scalar + + +class MultiIndex(Index): + """ + Koalas MultiIndex that corresponds to pandas MultiIndex logically. This might hold Spark Column + internally. + + Parameters + ---------- + levels : sequence of arrays + The unique labels for each level. + codes : sequence of arrays + Integers for each level designating which label at each location. + sortorder : optional int + Level of sortedness (must be lexicographically sorted by that + level). + names : optional sequence of objects + Names for each of the index levels. (name is accepted for compat). + copy : bool, default False + Copy the meta-data. + verify_integrity : bool, default True + Check that the levels/codes are consistent and valid. + + See Also + -------- + MultiIndex.from_arrays : Convert list of arrays to MultiIndex. + MultiIndex.from_product : Create a MultiIndex from the cartesian product + of iterables. + MultiIndex.from_tuples : Convert list of tuples to a MultiIndex. + MultiIndex.from_frame : Make a MultiIndex from a DataFrame. + Index : A single-level Index. + + Examples + -------- + >>> pp.DataFrame({'a': ['a', 'b', 'c']}, index=[[1, 2, 3], [4, 5, 6]]).index # doctest: +SKIP + MultiIndex([(1, 4), + (2, 5), + (3, 6)], + ) + + >>> pp.DataFrame({'a': [1, 2, 3]}, index=[list('abc'), list('def')]).index # doctest: +SKIP + MultiIndex([('a', 'd'), + ('b', 'e'), + ('c', 'f')], + ) + """ + + def __new__( + cls, + levels=None, + codes=None, + sortorder=None, + names=None, + dtype=None, + copy=False, + name=None, + verify_integrity: bool = True, + ): + if LooseVersion(pd.__version__) < LooseVersion("0.24"): + if levels is None or codes is None: + raise TypeError("Must pass both levels and codes") + + pidx = pd.MultiIndex( + levels=levels, + labels=codes, + sortorder=sortorder, + names=names, + dtype=dtype, + copy=copy, + name=name, + verify_integrity=verify_integrity, + ) + else: + pidx = pd.MultiIndex( + levels=levels, + codes=codes, + sortorder=sortorder, + names=names, + dtype=dtype, + copy=copy, + name=name, + verify_integrity=verify_integrity, + ) + return pp.from_pandas(pidx) + + @property + def _internal(self): + internal = self._kdf._internal + scol = F.struct(internal.index_spark_columns) + return internal.copy( + column_labels=[None], + data_spark_columns=[scol], + data_dtypes=[None], + column_label_names=None, + ) + + @property + def _column_label(self): + return None + + def __abs__(self): + raise TypeError("TypeError: cannot perform __abs__ with this index type: MultiIndex") + + def _with_new_scol(self, scol: spark.Column, *, dtype=None): + raise NotImplementedError("Not supported for type MultiIndex") + + def _align_and_column_op(self, f, *args) -> Index: + raise NotImplementedError("Not supported for type MultiIndex") + + def any(self, *args, **kwargs) -> None: + raise TypeError("cannot perform any with this index type: MultiIndex") + + def all(self, *args, **kwargs) -> None: + raise TypeError("cannot perform all with this index type: MultiIndex") + + @staticmethod + def from_tuples(tuples, sortorder=None, names=None) -> "MultiIndex": + """ + Convert list of tuples to MultiIndex. + + Parameters + ---------- + tuples : list / sequence of tuple-likes + Each tuple is the index of one row/column. + sortorder : int or None + Level of sortedness (must be lexicographically sorted by that level). + names : list / sequence of str, optional + Names for the levels in the index. + + Returns + ------- + index : MultiIndex + + Examples + -------- + + >>> tuples = [(1, 'red'), (1, 'blue'), + ... (2, 'red'), (2, 'blue')] + >>> pp.MultiIndex.from_tuples(tuples, names=('number', 'color')) # doctest: +SKIP + MultiIndex([(1, 'red'), + (1, 'blue'), + (2, 'red'), + (2, 'blue')], + names=['number', 'color']) + """ + return cast( + MultiIndex, + pp.from_pandas( + pd.MultiIndex.from_tuples(tuples=tuples, sortorder=sortorder, names=names) + ), + ) + + @staticmethod + def from_arrays(arrays, sortorder=None, names=None) -> "MultiIndex": + """ + Convert arrays to MultiIndex. + + Parameters + ---------- + arrays: list / sequence of array-likes + Each array-like gives one level’s value for each data point. len(arrays) + is the number of levels. + sortorder: int or None + Level of sortedness (must be lexicographically sorted by that level). + names: list / sequence of str, optional + Names for the levels in the index. + + Returns + ------- + index: MultiIndex + + Examples + -------- + + >>> arrays = [[1, 1, 2, 2], ['red', 'blue', 'red', 'blue']] + >>> pp.MultiIndex.from_arrays(arrays, names=('number', 'color')) # doctest: +SKIP + MultiIndex([(1, 'red'), + (1, 'blue'), + (2, 'red'), + (2, 'blue')], + names=['number', 'color']) + """ + return cast( + MultiIndex, + pp.from_pandas( + pd.MultiIndex.from_arrays(arrays=arrays, sortorder=sortorder, names=names) + ), + ) + + @staticmethod + def from_product(iterables, sortorder=None, names=None) -> "MultiIndex": + """ + Make a MultiIndex from the cartesian product of multiple iterables. + + Parameters + ---------- + iterables : list / sequence of iterables + Each iterable has unique labels for each level of the index. + sortorder : int or None + Level of sortedness (must be lexicographically sorted by that + level). + names : list / sequence of str, optional + Names for the levels in the index. + + Returns + ------- + index : MultiIndex + + See Also + -------- + MultiIndex.from_arrays : Convert list of arrays to MultiIndex. + MultiIndex.from_tuples : Convert list of tuples to MultiIndex. + + Examples + -------- + >>> numbers = [0, 1, 2] + >>> colors = ['green', 'purple'] + >>> pp.MultiIndex.from_product([numbers, colors], + ... names=['number', 'color']) # doctest: +SKIP + MultiIndex([(0, 'green'), + (0, 'purple'), + (1, 'green'), + (1, 'purple'), + (2, 'green'), + (2, 'purple')], + names=['number', 'color']) + """ + return cast( + MultiIndex, + pp.from_pandas( + pd.MultiIndex.from_product(iterables=iterables, sortorder=sortorder, names=names) + ), + ) + + @staticmethod + def from_frame(df, names=None) -> "MultiIndex": + """ + Make a MultiIndex from a DataFrame. + + Parameters + ---------- + df : DataFrame + DataFrame to be converted to MultiIndex. + names : list-like, optional + If no names are provided, use the column names, or tuple of column + names if the columns is a MultiIndex. If a sequence, overwrite + names with the given sequence. + + Returns + ------- + MultiIndex + The MultiIndex representation of the given DataFrame. + + See Also + -------- + MultiIndex.from_arrays : Convert list of arrays to MultiIndex. + MultiIndex.from_tuples : Convert list of tuples to MultiIndex. + MultiIndex.from_product : Make a MultiIndex from cartesian product + of iterables. + + Examples + -------- + >>> df = pp.DataFrame([['HI', 'Temp'], ['HI', 'Precip'], + ... ['NJ', 'Temp'], ['NJ', 'Precip']], + ... columns=['a', 'b']) + >>> df # doctest: +SKIP + a b + 0 HI Temp + 1 HI Precip + 2 NJ Temp + 3 NJ Precip + + >>> pp.MultiIndex.from_frame(df) # doctest: +SKIP + MultiIndex([('HI', 'Temp'), + ('HI', 'Precip'), + ('NJ', 'Temp'), + ('NJ', 'Precip')], + names=['a', 'b']) + + Using explicit names, instead of the column names + + >>> pp.MultiIndex.from_frame(df, names=['state', 'observation']) # doctest: +SKIP + MultiIndex([('HI', 'Temp'), + ('HI', 'Precip'), + ('NJ', 'Temp'), + ('NJ', 'Precip')], + names=['state', 'observation']) + """ + if not isinstance(df, DataFrame): + raise TypeError("Input must be a DataFrame") + sdf = df.to_spark() + + if names is None: + names = df._internal.column_labels + elif not is_list_like(names): + raise ValueError("Names should be list-like for a MultiIndex") + else: + names = [name if is_name_like_tuple(name) else (name,) for name in names] + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in sdf.columns], + index_names=names, + ) + return cast(MultiIndex, DataFrame(internal).index) + + @property + def name(self) -> str: + raise PandasNotImplementedError(class_name="pd.MultiIndex", property_name="name") + + @name.setter + def name(self, name: str) -> None: + raise PandasNotImplementedError(class_name="pd.MultiIndex", property_name="name") + + def _verify_for_rename(self, name): + if is_list_like(name): + if self._internal.index_level != len(name): + raise ValueError( + "Length of new names must be {}, got {}".format( + self._internal.index_level, len(name) + ) + ) + if any(not is_hashable(n) for n in name): + raise TypeError("MultiIndex.name must be a hashable type") + return [n if is_name_like_tuple(n) else (n,) for n in name] + else: + raise TypeError("Must pass list-like as `names`.") + + def swaplevel(self, i=-2, j=-1) -> "MultiIndex": + """ + Swap level i with level j. + Calling this method does not change the ordering of the values. + + Parameters + ---------- + i : int, str, default -2 + First level of index to be swapped. Can pass level name as string. + Type of parameters can be mixed. + j : int, str, default -1 + Second level of index to be swapped. Can pass level name as string. + Type of parameters can be mixed. + + Returns + ------- + MultiIndex + A new MultiIndex. + + Examples + -------- + >>> midx = pp.MultiIndex.from_arrays([['a', 'b'], [1, 2]], names = ['word', 'number']) + >>> midx # doctest: +SKIP + MultiIndex([('a', 1), + ('b', 2)], + names=['word', 'number']) + + >>> midx.swaplevel(0, 1) # doctest: +SKIP + MultiIndex([(1, 'a'), + (2, 'b')], + names=['number', 'word']) + + >>> midx.swaplevel('number', 'word') # doctest: +SKIP + MultiIndex([(1, 'a'), + (2, 'b')], + names=['number', 'word']) + """ + for index in (i, j): + if not isinstance(index, int) and index not in self.names: + raise KeyError("Level %s not found" % index) + + i = i if isinstance(i, int) else self.names.index(i) + j = j if isinstance(j, int) else self.names.index(j) + + for index in (i, j): + if index >= len(self.names) or index < -len(self.names): + raise IndexError( + "Too many levels: Index has only %s levels, " + "%s is not a valid level number" % (len(self.names), index) + ) + + index_map = list( + zip( + self._internal.index_spark_columns, + self._internal.index_names, + self._internal.index_dtypes, + ) + ) + index_map[i], index_map[j], = index_map[j], index_map[i] + index_spark_columns, index_names, index_dtypes = zip(*index_map) + internal = self._internal.copy( + index_spark_columns=list(index_spark_columns), + index_names=list(index_names), + index_dtypes=list(index_dtypes), + column_labels=[], + data_spark_columns=[], + data_dtypes=[], + ) + return cast(MultiIndex, DataFrame(internal).index) + + @property + def levshape(self) -> Tuple[int, ...]: + """ + A tuple with the length of each level. + + Examples + -------- + >>> midx = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> midx # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y'), + ('c', 'z')], + ) + + >>> midx.levshape + (3, 3) + """ + result = self._internal.spark_frame.agg( + *(F.countDistinct(c) for c in self._internal.index_spark_columns) + ).collect()[0] + return tuple(result) + + @staticmethod + def _comparator_for_monotonic_increasing(data_type): + return compare_disallow_null + + def _is_monotonic(self, order): + if order == "increasing": + return self._is_monotonic_increasing().all() + else: + return self._is_monotonic_decreasing().all() + + def _is_monotonic_increasing(self): + window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween(-1, -1) + + cond = F.lit(True) + has_not_null = F.lit(True) + for scol in self._internal.index_spark_columns[::-1]: + data_type = self._internal.spark_type_for(scol) + prev = F.lag(scol, 1).over(window) + compare = MultiIndex._comparator_for_monotonic_increasing(data_type) + # Since pandas 1.1.4, null value is not allowed at any levels of MultiIndex. + # Therefore, we should check `has_not_null` over the all levels. + has_not_null = has_not_null & scol.isNotNull() + cond = F.when(scol.eqNullSafe(prev), cond).otherwise( + compare(scol, prev, spark.Column.__gt__) + ) + + cond = has_not_null & (prev.isNull() | cond) + + cond_name = verify_temp_column_name( + self._internal.spark_frame.select(self._internal.index_spark_columns), + "__is_monotonic_increasing_cond__", + ) + + sdf = self._internal.spark_frame.select( + self._internal.index_spark_columns + [cond.alias(cond_name)] + ) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + + return first_series(DataFrame(internal)) + + @staticmethod + def _comparator_for_monotonic_decreasing(data_type): + return compare_disallow_null + + def _is_monotonic_decreasing(self): + window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween(-1, -1) + + cond = F.lit(True) + has_not_null = F.lit(True) + for scol in self._internal.index_spark_columns[::-1]: + data_type = self._internal.spark_type_for(scol) + prev = F.lag(scol, 1).over(window) + compare = MultiIndex._comparator_for_monotonic_increasing(data_type) + # Since pandas 1.1.4, null value is not allowed at any levels of MultiIndex. + # Therefore, we should check `has_not_null` over the all levels. + has_not_null = has_not_null & scol.isNotNull() + cond = F.when(scol.eqNullSafe(prev), cond).otherwise( + compare(scol, prev, spark.Column.__lt__) + ) + + cond = has_not_null & (prev.isNull() | cond) + + cond_name = verify_temp_column_name( + self._internal.spark_frame.select(self._internal.index_spark_columns), + "__is_monotonic_decreasing_cond__", + ) + + sdf = self._internal.spark_frame.select( + self._internal.index_spark_columns + [cond.alias(cond_name)] + ) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + ) + + return first_series(DataFrame(internal)) + + def to_frame(self, index=True, name=None) -> DataFrame: + """ + Create a DataFrame with the levels of the MultiIndex as columns. + Column ordering is determined by the DataFrame constructor with data as + a dict. + + Parameters + ---------- + index : boolean, default True + Set the index of the returned DataFrame as the original MultiIndex. + name : list / sequence of strings, optional + The passed names should substitute index level names. + + Returns + ------- + DataFrame : a DataFrame containing the original MultiIndex data. + + See Also + -------- + DataFrame + + Examples + -------- + >>> tuples = [(1, 'red'), (1, 'blue'), + ... (2, 'red'), (2, 'blue')] + >>> idx = pp.MultiIndex.from_tuples(tuples, names=('number', 'color')) + >>> idx # doctest: +SKIP + MultiIndex([(1, 'red'), + (1, 'blue'), + (2, 'red'), + (2, 'blue')], + names=['number', 'color']) + >>> idx.to_frame() # doctest: +NORMALIZE_WHITESPACE + number color + number color + 1 red 1 red + blue 1 blue + 2 red 2 red + blue 2 blue + + By default, the original Index is reused. To enforce a new Index: + + >>> idx.to_frame(index=False) + number color + 0 1 red + 1 1 blue + 2 2 red + 3 2 blue + + To override the name of the resulting column, specify `name`: + + >>> idx.to_frame(name=['n', 'c']) # doctest: +NORMALIZE_WHITESPACE + n c + number color + 1 red 1 red + blue 1 blue + 2 red 2 red + blue 2 blue + """ + if name is None: + name = [ + name if name is not None else (i,) + for i, name in enumerate(self._internal.index_names) + ] + elif is_list_like(name): + if len(name) != self._internal.index_level: + raise ValueError("'name' should have same length as number of levels on index.") + name = [n if is_name_like_tuple(n) else (n,) for n in name] + else: + raise TypeError("'name' must be a list / sequence of column names.") + + return self._to_frame(index=index, names=name) + + def to_pandas(self) -> pd.MultiIndex: + """ + Return a pandas MultiIndex. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats'], + ... index=[list('abcd'), list('efgh')]) + >>> df['dogs'].index.to_pandas() # doctest: +SKIP + MultiIndex([('a', 'e'), + ('b', 'f'), + ('c', 'g'), + ('d', 'h')], + ) + """ + # TODO: We might need to handle internal state change. + # So far, we don't have any functions to change the internal state of MultiIndex except for + # series-like operations. In that case, it creates new Index object instead of MultiIndex. + return super().to_pandas() + + def toPandas(self) -> pd.MultiIndex: + warnings.warn( + "MultiIndex.toPandas is deprecated as of MultiIndex.to_pandas. " + "Please use the API instead.", + FutureWarning, + ) + return self.to_pandas() + + toPandas.__doc__ = to_pandas.__doc__ + + def nunique(self, dropna=True) -> None: # type: ignore + raise NotImplementedError("nunique is not defined for MultiIndex") + + # TODO: add 'name' parameter after pd.MultiIndex.name is implemented + def copy(self, deep=None) -> "MultiIndex": # type: ignore + """ + Make a copy of this object. + + Parameters + ---------- + deep : None + this parameter is not supported but just dummy parameter to match pandas. + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats'], + ... index=[list('abcd'), list('efgh')]) + >>> df['dogs'].index # doctest: +SKIP + MultiIndex([('a', 'e'), + ('b', 'f'), + ('c', 'g'), + ('d', 'h')], + ) + + Copy index + + >>> df.index.copy() # doctest: +SKIP + MultiIndex([('a', 'e'), + ('b', 'f'), + ('c', 'g'), + ('d', 'h')], + ) + """ + return super().copy(deep=deep) # type: ignore + + def symmetric_difference(self, other, result_name=None, sort=None) -> "MultiIndex": + """ + Compute the symmetric difference of two MultiIndex objects. + + Parameters + ---------- + other : Index or array-like + result_name : list + sort : True or None, default None + Whether to sort the resulting index. + * True : Attempt to sort the result. + * None : Do not sort the result. + + Returns + ------- + symmetric_difference : MiltiIndex + + Notes + ----- + ``symmetric_difference`` contains elements that appear in either + ``idx1`` or ``idx2`` but not both. Equivalent to the Index created by + ``idx1.difference(idx2) | idx2.difference(idx1)`` with duplicates + dropped. + + Examples + -------- + >>> midx1 = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 0, 0, 0, 1, 2, 0, 1, 2]]) + >>> midx2 = pd.MultiIndex([['koalas', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 0, 0, 0, 1, 2, 0, 1, 2]]) + >>> s1 = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], + ... index=midx1) + >>> s2 = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], + ... index=midx2) + + >>> s1.index.symmetric_difference(s2.index) # doctest: +SKIP + MultiIndex([('koalas', 'speed'), + ( 'lama', 'speed')], + ) + + You can set names of result Index. + + >>> s1.index.symmetric_difference(s2.index, result_name=['a', 'b']) # doctest: +SKIP + MultiIndex([('koalas', 'speed'), + ( 'lama', 'speed')], + names=['a', 'b']) + + You can set sort to `True`, if you want to sort the resulting index. + + >>> s1.index.symmetric_difference(s2.index, sort=True) # doctest: +SKIP + MultiIndex([('koalas', 'speed'), + ( 'lama', 'speed')], + ) + + You can also use the ``^`` operator: + + >>> s1.index ^ s2.index # doctest: +SKIP + MultiIndex([('koalas', 'speed'), + ( 'lama', 'speed')], + ) + """ + if type(self) != type(other): + raise NotImplementedError( + "Doesn't support symmetric_difference between Index & MultiIndex for now" + ) + + sdf_self = self._kdf._internal.spark_frame.select(self._internal.index_spark_columns) + sdf_other = other._kdf._internal.spark_frame.select(other._internal.index_spark_columns) + + sdf_symdiff = sdf_self.union(sdf_other).subtract(sdf_self.intersect(sdf_other)) + + if sort: + sdf_symdiff = sdf_symdiff.sort(self._internal.index_spark_columns) + + internal = InternalFrame( # TODO: dtypes? + spark_frame=sdf_symdiff, + index_spark_columns=[ + scol_for(sdf_symdiff, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + ) + result = cast(MultiIndex, DataFrame(internal).index) + + if result_name: + result.names = result_name + + return result + + # TODO: ADD error parameter + def drop(self, codes, level=None) -> "MultiIndex": + """ + Make new MultiIndex with passed list of labels deleted + + Parameters + ---------- + codes : array-like + Must be a list of tuples + level : int or level name, default None + + Returns + ------- + dropped : MultiIndex + + Examples + -------- + >>> index = pp.MultiIndex.from_tuples([('a', 'x'), ('b', 'y'), ('c', 'z')]) + >>> index # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y'), + ('c', 'z')], + ) + + >>> index.drop(['a']) # doctest: +SKIP + MultiIndex([('b', 'y'), + ('c', 'z')], + ) + + >>> index.drop(['x', 'y'], level=1) # doctest: +SKIP + MultiIndex([('c', 'z')], + ) + """ + internal = self._internal.resolved_copy + sdf = internal.spark_frame + index_scols = internal.index_spark_columns + if level is None: + scol = index_scols[0] + elif isinstance(level, int): + scol = index_scols[level] + else: + scol = None + for index_spark_column, index_name in zip( + internal.index_spark_columns, internal.index_names + ): + if not isinstance(level, tuple): + level = (level,) + if level == index_name: + if scol is not None: + raise ValueError( + "The name {} occurs multiple times, use a level number".format( + name_like_string(level) + ) + ) + scol = index_spark_column + if scol is None: + raise KeyError("Level {} not found".format(name_like_string(level))) + sdf = sdf[~scol.isin(codes)] + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in internal.index_spark_column_names], + index_names=internal.index_names, + index_dtypes=internal.index_dtypes, + column_labels=[], + data_spark_columns=[], + data_dtypes=[], + ) + return cast(MultiIndex, DataFrame(internal).index) + + def value_counts( + self, normalize=False, sort=True, ascending=False, bins=None, dropna=True + ) -> Series: + if ( + LooseVersion(pyspark.__version__) < LooseVersion("2.4") + and default_session().conf.get("spark.sql.execution.arrow.enabled") == "true" + and isinstance(self, MultiIndex) + ): + raise RuntimeError( + "if you're using pyspark < 2.4, set conf " + "'spark.sql.execution.arrow.enabled' to 'false' " + "for using this function with MultiIndex" + ) + return super().value_counts( + normalize=normalize, sort=sort, ascending=ascending, bins=bins, dropna=dropna + ) + + value_counts.__doc__ = IndexOpsMixin.value_counts.__doc__ + + def argmax(self) -> None: + raise TypeError("reduction operation 'argmax' not allowed for this dtype") + + def argmin(self) -> None: + raise TypeError("reduction operation 'argmin' not allowed for this dtype") + + def asof(self, label) -> None: + raise NotImplementedError( + "only the default get_loc method is currently supported for MultiIndex" + ) + + @property + def is_all_dates(self) -> bool: + """ + is_all_dates always returns False for MultiIndex + + Examples + -------- + >>> from datetime import datetime + + >>> idx = pp.MultiIndex.from_tuples( + ... [(datetime(2019, 1, 1, 0, 0, 0), datetime(2019, 1, 1, 0, 0, 0)), + ... (datetime(2019, 1, 1, 0, 0, 0), datetime(2019, 1, 1, 0, 0, 0))]) + >>> idx # doctest: +SKIP + MultiIndex([('2019-01-01', '2019-01-01'), + ('2019-01-01', '2019-01-01')], + ) + + >>> idx.is_all_dates + False + """ + return False + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeMultiIndex, item): + property_or_func = getattr(MissingPandasLikeMultiIndex, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError("'MultiIndex' object has no attribute '{}'".format(item)) + + def _get_level_number(self, level) -> Optional[int]: + """ + Return the level number if a valid level is given. + """ + count = self.names.count(level) + if (count > 1) and not isinstance(level, int): + raise ValueError("The name %s occurs multiple times, use a level number" % level) + if level in self.names: + level = self.names.index(level) + elif isinstance(level, int): + nlevels = self.nlevels + if level >= nlevels: + raise IndexError( + "Too many levels: Index has only %d " + "levels, %d is not a valid level number" % (nlevels, level) + ) + if level < 0: + if (level + nlevels) < 0: + raise IndexError( + "Too many levels: Index has only %d levels, " + "not %d" % (nlevels, level + 1) + ) + level = level + nlevels + else: + raise KeyError("Level %s not found" % str(level)) + return None + + return level + + def get_level_values(self, level) -> Index: + """ + Return vector of label values for requested level, + equal to the length of the index. + + Parameters + ---------- + level : int or str + ``level`` is either the integer position of the level in the + MultiIndex, or the name of the level. + + Returns + ------- + values : Index + Values is a level of this MultiIndex converted to + a single :class:`Index` (or subclass thereof). + + Examples + -------- + + Create a MultiIndex: + + >>> mi = pp.MultiIndex.from_tuples([('x', 'a'), ('x', 'b'), ('y', 'a')]) + >>> mi.names = ['level_1', 'level_2'] + + Get level values by supplying level as either integer or name: + + >>> mi.get_level_values(0) + Index(['x', 'x', 'y'], dtype='object', name='level_1') + + >>> mi.get_level_values('level_2') + Index(['a', 'b', 'a'], dtype='object', name='level_2') + """ + level = self._get_level_number(level) + index_scol = self._internal.index_spark_columns[level] + index_name = self._internal.index_names[level] + index_dtype = self._internal.index_dtypes[level] + internal = self._internal.copy( + index_spark_columns=[index_scol], + index_names=[index_name], + index_dtypes=[index_dtype], + column_labels=[], + data_spark_columns=[], + data_dtypes=[], + ) + return DataFrame(internal).index + + def insert(self, loc: int, item) -> Index: + """ + Make new MultiIndex inserting new item at location. + + Follows Python list.append semantics for negative values. + + Parameters + ---------- + loc : int + item : object + + Returns + ------- + new_index : MultiIndex + + Examples + -------- + >>> kmidx = pp.MultiIndex.from_tuples([("a", "x"), ("b", "y"), ("c", "z")]) + >>> kmidx.insert(3, ("h", "j")) # doctest: +SKIP + MultiIndex([('a', 'x'), + ('b', 'y'), + ('c', 'z'), + ('h', 'j')], + ) + + For negative values + + >>> kmidx.insert(-2, ("h", "j")) # doctest: +SKIP + MultiIndex([('a', 'x'), + ('h', 'j'), + ('b', 'y'), + ('c', 'z')], + ) + """ + length = len(self) + if loc < 0: + loc = loc + length + if loc < 0: + raise IndexError( + "index {} is out of bounds for axis 0 with size {}".format( + (loc - length), length + ) + ) + else: + if loc > length: + raise IndexError( + "index {} is out of bounds for axis 0 with size {}".format(loc, length) + ) + + index_name = self._internal.index_spark_column_names + sdf_before = self.to_frame(name=index_name)[:loc].to_spark() + sdf_middle = Index([item]).to_frame(name=index_name).to_spark() + sdf_after = self.to_frame(name=index_name)[loc:].to_spark() + sdf = sdf_before.union(sdf_middle).union(sdf_after) + + internal = InternalFrame( # TODO: dtypes? + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + ) + return DataFrame(internal).index + + def item(self) -> Tuple[Scalar, ...]: + """ + Return the first element of the underlying data as a python tuple. + + Returns + ------- + tuple + The first element of MultiIndex. + + Raises + ------ + ValueError + If the data is not length-1. + + Examples + -------- + >>> kmidx = pp.MultiIndex.from_tuples([('a', 'x')]) + >>> kmidx.item() + ('a', 'x') + """ + return self._kdf.head(2)._to_internal_pandas().index.item() + + def intersection(self, other) -> "MultiIndex": + """ + Form the intersection of two Index objects. + + This returns a new Index with elements common to the index and `other`. + + Parameters + ---------- + other : Index or array-like + + Returns + ------- + intersection : MultiIndex + + Examples + -------- + >>> midx1 = pp.MultiIndex.from_tuples([("a", "x"), ("b", "y"), ("c", "z")]) + >>> midx2 = pp.MultiIndex.from_tuples([("c", "z"), ("d", "w")]) + >>> midx1.intersection(midx2).sort_values() # doctest: +SKIP + MultiIndex([('c', 'z')], + ) + """ + if isinstance(other, Series) or not is_list_like(other): + raise TypeError("other must be a MultiIndex or a list of tuples") + elif isinstance(other, DataFrame): + raise ValueError("Index data must be 1-dimensional") + elif isinstance(other, MultiIndex): + spark_frame_other = other.to_frame().to_spark() + keep_name = self.names == other.names + elif isinstance(other, Index): + # Always returns an empty MultiIndex if `other` is Index. + return self.to_frame().head(0).index # type: ignore + elif not all(isinstance(item, tuple) for item in other): + raise TypeError("other must be a MultiIndex or a list of tuples") + else: + other = MultiIndex.from_tuples(list(other)) + spark_frame_other = other.to_frame().to_spark() + keep_name = True + + default_name = [SPARK_INDEX_NAME_FORMAT(i) for i in range(self.nlevels)] + spark_frame_self = self.to_frame(name=default_name).to_spark() + spark_frame_intersected = spark_frame_self.intersect(spark_frame_other) + if keep_name: + index_names = self._internal.index_names + else: + index_names = None + internal = InternalFrame( # TODO: dtypes? + spark_frame=spark_frame_intersected, + index_spark_columns=[scol_for(spark_frame_intersected, col) for col in default_name], + index_names=index_names, + ) + return cast(MultiIndex, DataFrame(internal).index) + + @property + def hasnans(self): + raise NotImplementedError("hasnans is not defined for MultiIndex") + + @property + def inferred_type(self) -> str: + """ + Return a string of the type inferred from the values. + """ + # Always returns "mixed" for MultiIndex + return "mixed" + + @property + def asi8(self) -> None: + """ + Integer representation of the values. + """ + # Always returns None for MultiIndex + return None + + def factorize( + self, sort: bool = True, na_sentinel: Optional[int] = -1 + ) -> Tuple[Union["Series", "Index"], pd.Index]: + return MissingPandasLikeMultiIndex.factorize(self, sort=sort, na_sentinel=na_sentinel) + + def __iter__(self): + return MissingPandasLikeMultiIndex.__iter__(self) diff --git a/python/pyspark/pandas/indexes/numeric.py b/python/pyspark/pandas/indexes/numeric.py new file mode 100644 index 0000000000..1acad3f62c --- /dev/null +++ b/python/pyspark/pandas/indexes/numeric.py @@ -0,0 +1,147 @@ +# +# 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 pandas as pd +from pandas.api.types import is_hashable + +from pyspark import pandas as pp +from pyspark.pandas.indexes.base import Index +from pyspark.pandas.series import Series + + +class NumericIndex(Index): + """ + Provide numeric type operations. + This is an abstract class. + """ + + pass + + +class IntegerIndex(NumericIndex): + """ + This is an abstract class for Int64Index. + """ + + pass + + +class Int64Index(IntegerIndex): + """ + Immutable sequence used for indexing and alignment. The basic object + storing axis labels for all pandas objects. Int64Index is a special case + of `Index` with purely integer labels. + + Parameters + ---------- + data : array-like (1-dimensional) + dtype : NumPy dtype (default: int64) + copy : bool + Make a copy of input ndarray. + name : object + Name to be stored in the index. + + See Also + -------- + Index : The base Koalas Index type. + Float64Index : A special case of :class:`Index` with purely float labels. + + Notes + ----- + An Index instance can **only** contain hashable objects. + + Examples + -------- + >>> pp.Int64Index([1, 2, 3]) + Int64Index([1, 2, 3], dtype='int64') + + From a Series: + + >>> s = pp.Series([1, 2, 3], index=[10, 20, 30]) + >>> pp.Int64Index(s) + Int64Index([1, 2, 3], dtype='int64') + + From an Index: + + >>> idx = pp.Index([1, 2, 3]) + >>> pp.Int64Index(idx) + Int64Index([1, 2, 3], dtype='int64') + """ + + def __new__(cls, data=None, dtype=None, copy=False, name=None): + if not is_hashable(name): + raise TypeError("Index.name must be a hashable type") + + if isinstance(data, (Series, Index)): + if dtype is None: + dtype = "int64" + return Index(data, dtype=dtype, copy=copy, name=name) + + return pp.from_pandas(pd.Int64Index(data=data, dtype=dtype, copy=copy, name=name)) + + +class Float64Index(NumericIndex): + """ + Immutable sequence used for indexing and alignment. The basic object + storing axis labels for all pandas objects. Float64Index is a special case + of `Index` with purely float labels. + + Parameters + ---------- + data : array-like (1-dimensional) + dtype : NumPy dtype (default: float64) + copy : bool + Make a copy of input ndarray. + name : object + Name to be stored in the index. + + See Also + -------- + Index : The base Koalas Index type. + Int64Index : A special case of :class:`Index` with purely integer labels. + + Notes + ----- + An Index instance can **only** contain hashable objects. + + Examples + -------- + >>> pp.Float64Index([1.0, 2.0, 3.0]) + Float64Index([1.0, 2.0, 3.0], dtype='float64') + + From a Series: + + >>> s = pp.Series([1, 2, 3], index=[10, 20, 30]) + >>> pp.Float64Index(s) + Float64Index([1.0, 2.0, 3.0], dtype='float64') + + From an Index: + + >>> idx = pp.Index([1, 2, 3]) + >>> pp.Float64Index(idx) + Float64Index([1.0, 2.0, 3.0], dtype='float64') + """ + + def __new__(cls, data=None, dtype=None, copy=False, name=None): + if not is_hashable(name): + raise TypeError("Index.name must be a hashable type") + + if isinstance(data, (Series, Index)): + if dtype is None: + dtype = "float64" + return Index(data, dtype=dtype, copy=copy, name=name) + + return pp.from_pandas(pd.Float64Index(data=data, dtype=dtype, copy=copy, name=name)) diff --git a/python/pyspark/pandas/indexing.py b/python/pyspark/pandas/indexing.py new file mode 100644 index 0000000000..0016d4638c --- /dev/null +++ b/python/pyspark/pandas/indexing.py @@ -0,0 +1,1708 @@ +# +# 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. +# + +""" +A loc indexer for Koalas DataFrame/Series. +""" +from abc import ABCMeta, abstractmethod +from collections.abc import Iterable +from functools import reduce +from typing import Any, Optional, List, Tuple, TYPE_CHECKING, Union, cast, Sized + +import pandas as pd +from pandas.api.types import is_list_like +from pyspark import sql as spark +from pyspark.sql import functions as F +from pyspark.sql.types import BooleanType, LongType +from pyspark.sql.utils import AnalysisException +import numpy as np + +from pyspark import pandas as pp # noqa: F401 +from pyspark.pandas.internal import ( + InternalFrame, + NATURAL_ORDER_COLUMN_NAME, + SPARK_DEFAULT_SERIES_NAME, +) +from pyspark.pandas.exceptions import SparkPandasIndexingError, SparkPandasNotImplementedError +from pyspark.pandas.typedef.typehints import ( + Dtype, + Scalar, + extension_dtypes, + spark_type_to_pandas_dtype, +) +from pyspark.pandas.utils import ( + is_name_like_tuple, + is_name_like_value, + lazy_property, + name_like_string, + same_anchor, + scol_for, + verify_temp_column_name, +) + +if TYPE_CHECKING: + from pyspark.pandas.frame import DataFrame # noqa: F401 (SPARK-34943) + from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943) + + +class IndexerLike(object): + def __init__(self, kdf_or_kser): + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.series import Series + + assert isinstance(kdf_or_kser, (DataFrame, Series)), "unexpected argument type: {}".format( + type(kdf_or_kser) + ) + self._kdf_or_kser = kdf_or_kser + + @property + def _is_df(self): + from pyspark.pandas.frame import DataFrame + + return isinstance(self._kdf_or_kser, DataFrame) + + @property + def _is_series(self): + from pyspark.pandas.series import Series + + return isinstance(self._kdf_or_kser, Series) + + @property + def _kdf(self): + if self._is_df: + return self._kdf_or_kser + else: + assert self._is_series + return self._kdf_or_kser._kdf + + @property + def _internal(self): + return self._kdf._internal + + +class AtIndexer(IndexerLike): + """ + Access a single value for a row/column label pair. + If the index is not unique, all matching pairs are returned as an array. + Similar to ``loc``, in that both provide label-based lookups. Use ``at`` if you only need to + get a single value in a DataFrame or Series. + + .. note:: Unlike pandas, Koalas only allows using ``at`` to get values but not to set them. + + .. note:: Warning: If ``row_index`` matches a lot of rows, large amounts of data will be + fetched, potentially causing your machine to run out of memory. + + Raises + ------ + KeyError + When label does not exist in DataFrame + + Examples + -------- + >>> kdf = pp.DataFrame([[0, 2, 3], [0, 4, 1], [10, 20, 30]], + ... index=[4, 5, 5], columns=['A', 'B', 'C']) + >>> kdf + A B C + 4 0 2 3 + 5 0 4 1 + 5 10 20 30 + + Get value at specified row/column pair + + >>> kdf.at[4, 'B'] + 2 + + Get array if an index occurs multiple times + + >>> kdf.at[5, 'B'] + array([ 4, 20]) + """ + + def __getitem__(self, key) -> Union["Series", "DataFrame", Scalar]: + if self._is_df: + if not isinstance(key, tuple) or len(key) != 2: + raise TypeError("Use DataFrame.at like .at[row_index, column_name]") + row_sel, col_sel = key + else: + assert self._is_series, type(self._kdf_or_kser) + if isinstance(key, tuple) and len(key) != 1: + raise TypeError("Use Series.at like .at[row_index]") + row_sel = key + col_sel = self._kdf_or_kser._column_label + + if self._internal.index_level == 1: + if not is_name_like_value(row_sel, allow_none=False, allow_tuple=False): + raise ValueError("At based indexing on a single index can only have a single value") + row_sel = (row_sel,) + else: + if not is_name_like_tuple(row_sel, allow_none=False): + raise ValueError("At based indexing on multi-index can only have tuple values") + + if col_sel is not None: + if not is_name_like_value(col_sel, allow_none=False): + raise ValueError("At based indexing on multi-index can only have tuple values") + if not is_name_like_tuple(col_sel): + col_sel = (col_sel,) + + cond = reduce( + lambda x, y: x & y, + [scol == row for scol, row in zip(self._internal.index_spark_columns, row_sel)], + ) + pdf = ( + self._internal.spark_frame.drop(NATURAL_ORDER_COLUMN_NAME) + .filter(cond) + .select(self._internal.spark_column_for(col_sel)) + .toPandas() + ) + + if len(pdf) < 1: + raise KeyError(name_like_string(row_sel)) + + values = pdf.iloc[:, 0].values + return ( + values if (len(row_sel) < self._internal.index_level or len(values) > 1) else values[0] + ) + + +class iAtIndexer(IndexerLike): + """ + Access a single value for a row/column pair by integer position. + + Similar to ``iloc``, in that both provide integer-based lookups. Use + ``iat`` if you only need to get or set a single value in a DataFrame + or Series. + + Raises + ------ + KeyError + When label does not exist in DataFrame + + Examples + -------- + >>> df = pp.DataFrame([[0, 2, 3], [0, 4, 1], [10, 20, 30]], + ... columns=['A', 'B', 'C']) + >>> df + A B C + 0 0 2 3 + 1 0 4 1 + 2 10 20 30 + + Get value at specified row/column pair + + >>> df.iat[1, 2] + 1 + + Get value within a series + + >>> kser = pp.Series([1, 2, 3], index=[10, 20, 30]) + >>> kser + 10 1 + 20 2 + 30 3 + dtype: int64 + + >>> kser.iat[1] + 2 + """ + + def __getitem__(self, key) -> Union["Series", "DataFrame", Scalar]: + if self._is_df: + if not isinstance(key, tuple) or len(key) != 2: + raise TypeError( + "Use DataFrame.iat like .iat[row_integer_position, column_integer_position]" + ) + row_sel, col_sel = key + if not isinstance(row_sel, int) or not isinstance(col_sel, int): + raise ValueError("iAt based indexing can only have integer indexers") + return self._kdf_or_kser.iloc[row_sel, col_sel] + else: + assert self._is_series, type(self._kdf_or_kser) + if not isinstance(key, int) and len(key) != 1: + raise TypeError("Use Series.iat like .iat[row_integer_position]") + if not isinstance(key, int): + raise ValueError("iAt based indexing can only have integer indexers") + return self._kdf_or_kser.iloc[key] + + +class LocIndexerLike(IndexerLike, metaclass=ABCMeta): + def _select_rows( + self, rows_sel: Any + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + """ + Dispatch the logic for select rows to more specific methods by `rows_sel` argument types. + + Parameters + ---------- + rows_sel : the key specified to select rows. + + Returns + ------- + Tuple of Spark column, int, int: + + * The Spark column for the condition to filter the rows. + * The number of rows when the selection can be simplified by limit. + * The remaining index rows if the result index size is shrunk. + """ + from pyspark.pandas.series import Series + + if rows_sel is None: + return None, None, None + elif isinstance(rows_sel, Series): + return self._select_rows_by_series(rows_sel) + elif isinstance(rows_sel, spark.Column): + return self._select_rows_by_spark_column(rows_sel) + elif isinstance(rows_sel, slice): + if rows_sel == slice(None): + # If slice is None - select everything, so nothing to do + return None, None, None + return self._select_rows_by_slice(rows_sel) + elif isinstance(rows_sel, tuple): + return self._select_rows_else(rows_sel) + elif is_list_like(rows_sel): + return self._select_rows_by_iterable(rows_sel) + else: + return self._select_rows_else(rows_sel) + + def _select_cols( + self, cols_sel: Any, missing_keys: Optional[List[Tuple]] = None + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + """ + Dispatch the logic for select columns to more specific methods by `cols_sel` argument types. + + Parameters + ---------- + cols_sel : the key specified to select columns. + + Returns + ------- + Tuple of list of column label, list of Spark columns, list of dtypes, bool: + + * The column labels selected. + * The Spark columns selected. + * The dtypes selected. + * The boolean value whether Series should be returned or not. + * The Series name if needed. + """ + from pyspark.pandas.series import Series + + if cols_sel is None: + column_labels = self._internal.column_labels + data_spark_columns = self._internal.data_spark_columns + data_dtypes = self._internal.data_dtypes + return column_labels, data_spark_columns, data_dtypes, False, None + elif isinstance(cols_sel, Series): + return self._select_cols_by_series(cols_sel, missing_keys) + elif isinstance(cols_sel, spark.Column): + return self._select_cols_by_spark_column(cols_sel, missing_keys) + elif isinstance(cols_sel, slice): + if cols_sel == slice(None): + # If slice is None - select everything, so nothing to do + column_labels = self._internal.column_labels + data_spark_columns = self._internal.data_spark_columns + data_dtypes = self._internal.data_dtypes + return column_labels, data_spark_columns, data_dtypes, False, None + return self._select_cols_by_slice(cols_sel, missing_keys) + elif isinstance(cols_sel, tuple): + return self._select_cols_else(cols_sel, missing_keys) + elif is_list_like(cols_sel): + return self._select_cols_by_iterable(cols_sel, missing_keys) + else: + return self._select_cols_else(cols_sel, missing_keys) + + # Methods for row selection + + @abstractmethod + def _select_rows_by_series( + self, rows_sel: "Series" + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + """ Select rows by `Series` type key. """ + pass + + @abstractmethod + def _select_rows_by_spark_column( + self, rows_sel: spark.column + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + """ Select rows by Spark `Column` type key. """ + pass + + @abstractmethod + def _select_rows_by_slice( + self, rows_sel: slice + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + """ Select rows by `slice` type key. """ + pass + + @abstractmethod + def _select_rows_by_iterable( + self, rows_sel: Iterable + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + """ Select rows by `Iterable` type key. """ + pass + + @abstractmethod + def _select_rows_else( + self, rows_sel: Any + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + """ Select rows by other type key. """ + pass + + # Methods for col selection + + @abstractmethod + def _select_cols_by_series( + self, cols_sel: "Series", missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + """ Select columns by `Series` type key. """ + pass + + @abstractmethod + def _select_cols_by_spark_column( + self, cols_sel: spark.Column, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + """ Select columns by Spark `Column` type key. """ + pass + + @abstractmethod + def _select_cols_by_slice( + self, cols_sel: slice, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + """ Select columns by `slice` type key. """ + pass + + @abstractmethod + def _select_cols_by_iterable( + self, cols_sel: Iterable, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + """ Select columns by `Iterable` type key. """ + pass + + @abstractmethod + def _select_cols_else( + self, cols_sel: Any, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + """ Select columns by other type key. """ + pass + + def __getitem__(self, key) -> Union["Series", "DataFrame"]: + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.series import Series, first_series + + if self._is_series: + if isinstance(key, Series) and not same_anchor(key, self._kdf_or_kser): + kdf = self._kdf_or_kser.to_frame() + temp_col = verify_temp_column_name(kdf, "__temp_col__") + + kdf[temp_col] = key + return type(self)(kdf[self._kdf_or_kser.name])[kdf[temp_col]] + + cond, limit, remaining_index = self._select_rows(key) + if cond is None and limit is None: + return self._kdf_or_kser + + column_label = self._kdf_or_kser._column_label + column_labels = [column_label] + data_spark_columns = [self._internal.spark_column_for(column_label)] + data_dtypes = [self._internal.dtype_for(column_label)] + returns_series = True + series_name = self._kdf_or_kser.name + else: + assert self._is_df + if isinstance(key, tuple): + if len(key) != 2: + raise SparkPandasIndexingError("Only accepts pairs of candidates") + rows_sel, cols_sel = key + else: + rows_sel = key + cols_sel = None + + if isinstance(rows_sel, Series) and not same_anchor(rows_sel, self._kdf_or_kser): + kdf = self._kdf_or_kser.copy() + temp_col = verify_temp_column_name(kdf, "__temp_col__") + + kdf[temp_col] = rows_sel + return type(self)(kdf)[kdf[temp_col], cols_sel][list(self._kdf_or_kser.columns)] + + cond, limit, remaining_index = self._select_rows(rows_sel) + ( + column_labels, + data_spark_columns, + data_dtypes, + returns_series, + series_name, + ) = self._select_cols(cols_sel) + + if cond is None and limit is None and returns_series: + kser = self._kdf_or_kser._kser_for(column_labels[0]) + if series_name is not None and series_name != kser.name: + kser = kser.rename(series_name) + return kser + + if remaining_index is not None: + index_spark_columns = self._internal.index_spark_columns[-remaining_index:] + index_names = self._internal.index_names[-remaining_index:] + index_dtypes = self._internal.index_dtypes[-remaining_index:] + else: + index_spark_columns = self._internal.index_spark_columns + index_names = self._internal.index_names + index_dtypes = self._internal.index_dtypes + + if len(column_labels) > 0: + column_labels = column_labels.copy() + column_labels_level = max( + len(label) if label is not None else 1 for label in column_labels + ) + none_column = 0 + for i, label in enumerate(column_labels): + if label is None: + label = (none_column,) + none_column += 1 + if len(label) < column_labels_level: + label = tuple(list(label) + ([""]) * (column_labels_level - len(label))) + column_labels[i] = label + + if i == 0 and none_column == 1: + column_labels = [None] + + column_label_names = self._internal.column_label_names[-column_labels_level:] + else: + column_label_names = self._internal.column_label_names + + try: + sdf = self._internal.spark_frame + + if cond is not None: + index_columns = sdf.select(index_spark_columns).columns + data_columns = sdf.select(data_spark_columns).columns + sdf = sdf.filter(cond).select(index_spark_columns + data_spark_columns) + index_spark_columns = [scol_for(sdf, col) for col in index_columns] + data_spark_columns = [scol_for(sdf, col) for col in data_columns] + + if limit is not None: + if limit >= 0: + sdf = sdf.limit(limit) + else: + sdf = sdf.limit(sdf.count() + limit) + sdf = sdf.drop(NATURAL_ORDER_COLUMN_NAME) + except AnalysisException: + raise KeyError( + "[{}] don't exist in columns".format( + [col._jc.toString() for col in data_spark_columns] + ) + ) + + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=index_spark_columns, + index_names=index_names, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=data_spark_columns, + data_dtypes=data_dtypes, + column_label_names=column_label_names, + ) + kdf = DataFrame(internal) + + if returns_series: + kdf_or_kser = first_series(kdf) + if series_name is not None and series_name != kdf_or_kser.name: + kdf_or_kser = kdf_or_kser.rename(series_name) + else: + kdf_or_kser = kdf + + if remaining_index is not None and remaining_index == 0: + pdf_or_pser = kdf_or_kser.head(2).to_pandas() + length = len(pdf_or_pser) + if length == 0: + raise KeyError(name_like_string(key)) + elif length == 1: + return pdf_or_pser.iloc[0] + else: + return kdf_or_kser + else: + return kdf_or_kser + + def __setitem__(self, key, value): + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.series import Series, first_series + + if self._is_series: + if ( + isinstance(key, Series) + and (isinstance(self, iLocIndexer) or not same_anchor(key, self._kdf_or_kser)) + ) or ( + isinstance(value, Series) + and (isinstance(self, iLocIndexer) or not same_anchor(value, self._kdf_or_kser)) + ): + if self._kdf_or_kser.name is None: + kdf = self._kdf_or_kser.to_frame() + column_label = kdf._internal.column_labels[0] + else: + kdf = self._kdf_or_kser._kdf.copy() + column_label = self._kdf_or_kser._column_label + temp_natural_order = verify_temp_column_name(kdf, "__temp_natural_order__") + temp_key_col = verify_temp_column_name(kdf, "__temp_key_col__") + temp_value_col = verify_temp_column_name(kdf, "__temp_value_col__") + + kdf[temp_natural_order] = F.monotonically_increasing_id() + if isinstance(key, Series): + kdf[temp_key_col] = key + if isinstance(value, Series): + kdf[temp_value_col] = value + kdf = kdf.sort_values(temp_natural_order).drop(temp_natural_order) + + kser = kdf._kser_for(column_label) + if isinstance(key, Series): + key = F.col( + "`{}`".format(kdf[temp_key_col]._internal.data_spark_column_names[0]) + ) + if isinstance(value, Series): + value = F.col( + "`{}`".format(kdf[temp_value_col]._internal.data_spark_column_names[0]) + ) + + type(self)(kser)[key] = value + + if self._kdf_or_kser.name is None: + kser = kser.rename() + + self._kdf_or_kser._kdf._update_internal_frame( + kser._kdf[ + self._kdf_or_kser._kdf._internal.column_labels + ]._internal.resolved_copy, + requires_same_anchor=False, + ) + return + + if isinstance(value, DataFrame): + raise ValueError("Incompatible indexer with DataFrame") + + cond, limit, remaining_index = self._select_rows(key) + if cond is None: + cond = F.lit(True) + if limit is not None: + cond = cond & (self._internal.spark_frame[self._sequence_col] < F.lit(limit)) + + if isinstance(value, (Series, spark.Column)): + if remaining_index is not None and remaining_index == 0: + raise ValueError( + "No axis named {} for object type {}".format(key, type(value).__name__) + ) + if isinstance(value, Series): + value = value.spark.column + else: + value = F.lit(value) + scol = ( + F.when(cond, value) + .otherwise(self._internal.spark_column_for(self._kdf_or_kser._column_label)) + .alias(name_like_string(self._kdf_or_kser.name or SPARK_DEFAULT_SERIES_NAME)) + ) + + internal = self._internal.with_new_spark_column( + self._kdf_or_kser._column_label, scol # TODO: dtype? + ) + self._kdf_or_kser._kdf._update_internal_frame(internal, requires_same_anchor=False) + else: + assert self._is_df + + if isinstance(key, tuple): + if len(key) != 2: + raise SparkPandasIndexingError("Only accepts pairs of candidates") + rows_sel, cols_sel = key + else: + rows_sel = key + cols_sel = None + + if isinstance(value, DataFrame): + if len(value.columns) == 1: + value = first_series(value) + else: + raise ValueError("Only a dataframe with one column can be assigned") + + if ( + isinstance(rows_sel, Series) + and (isinstance(self, iLocIndexer) or not same_anchor(rows_sel, self._kdf_or_kser)) + ) or ( + isinstance(value, Series) + and (isinstance(self, iLocIndexer) or not same_anchor(value, self._kdf_or_kser)) + ): + kdf = self._kdf_or_kser.copy() + temp_natural_order = verify_temp_column_name(kdf, "__temp_natural_order__") + temp_key_col = verify_temp_column_name(kdf, "__temp_key_col__") + temp_value_col = verify_temp_column_name(kdf, "__temp_value_col__") + + kdf[temp_natural_order] = F.monotonically_increasing_id() + if isinstance(rows_sel, Series): + kdf[temp_key_col] = rows_sel + if isinstance(value, Series): + kdf[temp_value_col] = value + kdf = kdf.sort_values(temp_natural_order).drop(temp_natural_order) + + if isinstance(rows_sel, Series): + rows_sel = F.col( + "`{}`".format(kdf[temp_key_col]._internal.data_spark_column_names[0]) + ) + if isinstance(value, Series): + value = F.col( + "`{}`".format(kdf[temp_value_col]._internal.data_spark_column_names[0]) + ) + + type(self)(kdf)[rows_sel, cols_sel] = value + + self._kdf_or_kser._update_internal_frame( + kdf[list(self._kdf_or_kser.columns)]._internal.resolved_copy, + requires_same_anchor=False, + ) + return + + cond, limit, remaining_index = self._select_rows(rows_sel) + missing_keys = [] + _, data_spark_columns, _, _, _ = self._select_cols(cols_sel, missing_keys=missing_keys) + + if cond is None: + cond = F.lit(True) + if limit is not None: + cond = cond & (self._internal.spark_frame[self._sequence_col] < F.lit(limit)) + + if isinstance(value, (Series, spark.Column)): + if remaining_index is not None and remaining_index == 0: + raise ValueError("Incompatible indexer with Series") + if len(data_spark_columns) > 1: + raise ValueError("shape mismatch") + if isinstance(value, Series): + value = value.spark.column + else: + value = F.lit(value) + + new_data_spark_columns = [] + new_dtypes = [] + for new_scol, spark_column_name, new_dtype in zip( + self._internal.data_spark_columns, + self._internal.data_spark_column_names, + self._internal.data_dtypes, + ): + for scol in data_spark_columns: + if new_scol._jc.equals(scol._jc): + new_scol = F.when(cond, value).otherwise(scol).alias(spark_column_name) + new_dtype = spark_type_to_pandas_dtype( + self._internal.spark_frame.select(new_scol).schema[0].dataType, + use_extension_dtypes=isinstance(new_dtype, extension_dtypes), + ) + break + new_data_spark_columns.append(new_scol) + new_dtypes.append(new_dtype) + + column_labels = self._internal.column_labels.copy() + for label in missing_keys: + if not is_name_like_tuple(label): + label = (label,) + if len(label) < self._internal.column_labels_level: + label = tuple( + list(label) + ([""] * (self._internal.column_labels_level - len(label))) + ) + elif len(label) > self._internal.column_labels_level: + raise KeyError( + "Key length ({}) exceeds index depth ({})".format( + len(label), self._internal.column_labels_level + ) + ) + column_labels.append(label) + new_data_spark_columns.append(F.when(cond, value).alias(name_like_string(label))) + new_dtypes.append(None) + + internal = self._internal.with_new_columns( + new_data_spark_columns, column_labels=column_labels, data_dtypes=new_dtypes + ) + self._kdf_or_kser._update_internal_frame(internal, requires_same_anchor=False) + + +class LocIndexer(LocIndexerLike): + """ + Access a group of rows and columns by label(s) or a boolean Series. + + ``.loc[]`` is primarily label based, but may also be used with a + conditional boolean Series derived from the DataFrame or Series. + + Allowed inputs are: + + - A single label, e.g. ``5`` or ``'a'``, (note that ``5`` is + interpreted as a *label* of the index, and **never** as an + integer position along the index) for column selection. + + - A list or array of labels, e.g. ``['a', 'b', 'c']``. + + - A slice object with labels, e.g. ``'a':'f'``. + + - A conditional boolean Series derived from the DataFrame or Series + + - A boolean array of the same length as the column axis being sliced, + e.g. ``[True, False, True]``. + + - An alignable boolean pandas Series to the column axis being sliced. + The index of the key will be aligned before masking. + + Not allowed inputs which pandas allows are: + + - A boolean array of the same length as the row axis being sliced, + e.g. ``[True, False, True]``. + - A ``callable`` function with one argument (the calling Series, DataFrame + or Panel) and that returns valid output for indexing (one of the above) + + .. note:: MultiIndex is not supported yet. + + .. note:: Note that contrary to usual python slices, **both** the + start and the stop are included, and the step of the slice is not allowed. + + .. note:: With a list or array of labels for row selection, + Koalas behaves as a filter without reordering by the labels. + + See Also + -------- + Series.loc : Access group of values using labels. + + Examples + -------- + **Getting values** + + >>> df = pp.DataFrame([[1, 2], [4, 5], [7, 8]], + ... index=['cobra', 'viper', 'sidewinder'], + ... columns=['max_speed', 'shield']) + >>> df + max_speed shield + cobra 1 2 + viper 4 5 + sidewinder 7 8 + + Single label. Note this returns the row as a Series. + + >>> df.loc['viper'] + max_speed 4 + shield 5 + Name: viper, dtype: int64 + + List of labels. Note using ``[[]]`` returns a DataFrame. + Also note that Koalas behaves just a filter without reordering by the labels. + + >>> df.loc[['viper', 'sidewinder']] + max_speed shield + viper 4 5 + sidewinder 7 8 + + >>> df.loc[['sidewinder', 'viper']] + max_speed shield + viper 4 5 + sidewinder 7 8 + + Single label for column. + + >>> df.loc['cobra', 'shield'] + 2 + + List of labels for row. + + >>> df.loc[['cobra'], 'shield'] + cobra 2 + Name: shield, dtype: int64 + + List of labels for column. + + >>> df.loc['cobra', ['shield']] + shield 2 + Name: cobra, dtype: int64 + + List of labels for both row and column. + + >>> df.loc[['cobra'], ['shield']] + shield + cobra 2 + + Slice with labels for row and single label for column. As mentioned + above, note that both the start and stop of the slice are included. + + >>> df.loc['cobra':'viper', 'max_speed'] + cobra 1 + viper 4 + Name: max_speed, dtype: int64 + + Conditional that returns a boolean Series + + >>> df.loc[df['shield'] > 6] + max_speed shield + sidewinder 7 8 + + Conditional that returns a boolean Series with column labels specified + + >>> df.loc[df['shield'] > 6, ['max_speed']] + max_speed + sidewinder 7 + + A boolean array of the same length as the column axis being sliced. + + >>> df.loc[:, [False, True]] + shield + cobra 2 + viper 5 + sidewinder 8 + + An alignable boolean Series to the column axis being sliced. + + >>> df.loc[:, pd.Series([False, True], index=['max_speed', 'shield'])] + shield + cobra 2 + viper 5 + sidewinder 8 + + **Setting values** + + Setting value for all items matching the list of labels. + + >>> df.loc[['viper', 'sidewinder'], ['shield']] = 50 + >>> df + max_speed shield + cobra 1 2 + viper 4 50 + sidewinder 7 50 + + Setting value for an entire row + + >>> df.loc['cobra'] = 10 + >>> df + max_speed shield + cobra 10 10 + viper 4 50 + sidewinder 7 50 + + Set value for an entire column + + >>> df.loc[:, 'max_speed'] = 30 + >>> df + max_speed shield + cobra 30 10 + viper 30 50 + sidewinder 30 50 + + Set value for an entire list of columns + + >>> df.loc[:, ['max_speed', 'shield']] = 100 + >>> df + max_speed shield + cobra 100 100 + viper 100 100 + sidewinder 100 100 + + Set value with Series + + >>> df.loc[:, 'shield'] = df['shield'] * 2 + >>> df + max_speed shield + cobra 100 200 + viper 100 200 + sidewinder 100 200 + + **Getting values on a DataFrame with an index that has integer labels** + + Another example using integers for the index + + >>> df = pp.DataFrame([[1, 2], [4, 5], [7, 8]], + ... index=[7, 8, 9], + ... columns=['max_speed', 'shield']) + >>> df + max_speed shield + 7 1 2 + 8 4 5 + 9 7 8 + + Slice with integer labels for rows. As mentioned above, note that both + the start and stop of the slice are included. + + >>> df.loc[7:9] + max_speed shield + 7 1 2 + 8 4 5 + 9 7 8 + """ + + @staticmethod + def _NotImplemented(description): + return SparkPandasNotImplementedError( + description=description, + pandas_function=".loc[..., ...]", + spark_target_function="select, where", + ) + + def _select_rows_by_series( + self, rows_sel: "Series" + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + assert isinstance(rows_sel.spark.data_type, BooleanType), rows_sel.spark.data_type + return rows_sel.spark.column, None, None + + def _select_rows_by_spark_column( + self, rows_sel: spark.Column + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + spark_type = self._internal.spark_frame.select(rows_sel).schema[0].dataType + assert isinstance(spark_type, BooleanType), spark_type + return rows_sel, None, None + + def _select_rows_by_slice( + self, rows_sel: slice + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + from pyspark.pandas.indexes import MultiIndex + + if rows_sel.step is not None: + raise LocIndexer._NotImplemented("Cannot use step with Spark.") + elif self._internal.index_level == 1: + sdf = self._internal.spark_frame + index = self._kdf_or_kser.index + index_column = index.to_series() + index_data_type = index_column.spark.data_type + start = rows_sel.start + stop = rows_sel.stop + + # get natural order from '__natural_order__' from start to stop + # to keep natural order. + start_and_stop = ( + sdf.select(index_column.spark.column, NATURAL_ORDER_COLUMN_NAME) + .where( + (index_column.spark.column == F.lit(start).cast(index_data_type)) + | (index_column.spark.column == F.lit(stop).cast(index_data_type)) + ) + .collect() + ) + + start = [row[1] for row in start_and_stop if row[0] == start] + start = start[0] if len(start) > 0 else None + + stop = [row[1] for row in start_and_stop if row[0] == stop] + stop = stop[-1] if len(stop) > 0 else None + + cond = [] + if start is not None: + cond.append(F.col(NATURAL_ORDER_COLUMN_NAME) >= F.lit(start).cast(LongType())) + if stop is not None: + cond.append(F.col(NATURAL_ORDER_COLUMN_NAME) <= F.lit(stop).cast(LongType())) + + # if index order is not monotonic increasing or decreasing + # and specified values don't exist in index, raise KeyError + if (start is None and rows_sel.start is not None) or ( + stop is None and rows_sel.stop is not None + ): + + inc = index_column.is_monotonic_increasing + if inc is False: + dec = index_column.is_monotonic_decreasing + + if start is None and rows_sel.start is not None: + start = rows_sel.start + if inc is not False: + cond.append(index_column.spark.column >= F.lit(start).cast(index_data_type)) + elif dec is not False: + cond.append(index_column.spark.column <= F.lit(start).cast(index_data_type)) + else: + raise KeyError(rows_sel.start) + if stop is None and rows_sel.stop is not None: + stop = rows_sel.stop + if inc is not False: + cond.append(index_column.spark.column <= F.lit(stop).cast(index_data_type)) + elif dec is not False: + cond.append(index_column.spark.column >= F.lit(stop).cast(index_data_type)) + else: + raise KeyError(rows_sel.stop) + + return reduce(lambda x, y: x & y, cond), None, None + else: + index = self._kdf_or_kser.index + index_data_type = [f.dataType for f in index.to_series().spark.data_type] + + start = rows_sel.start + if start is not None: + if not isinstance(start, tuple): + start = (start,) + if len(start) == 0: + start = None + stop = rows_sel.stop + if stop is not None: + if not isinstance(stop, tuple): + stop = (stop,) + if len(stop) == 0: + stop = None + + depth = max( + len(start) if start is not None else 0, len(stop) if stop is not None else 0 + ) + if depth == 0: + return None, None, None + elif ( + depth > self._internal.index_level + or not index.droplevel(list(range(self._internal.index_level)[depth:])).is_monotonic + ): + raise KeyError( + "Key length ({}) was greater than MultiIndex sort depth".format(depth) + ) + + conds = [] # type: List[spark.Column] + if start is not None: + cond = F.lit(True) + for scol, value, dt in list( + zip(self._internal.index_spark_columns, start, index_data_type) + )[::-1]: + compare = MultiIndex._comparator_for_monotonic_increasing(dt) + cond = F.when(scol.eqNullSafe(F.lit(value).cast(dt)), cond).otherwise( + compare(scol, F.lit(value).cast(dt), spark.Column.__gt__) + ) + conds.append(cond) + if stop is not None: + cond = F.lit(True) + for scol, value, dt in list( + zip(self._internal.index_spark_columns, stop, index_data_type) + )[::-1]: + compare = MultiIndex._comparator_for_monotonic_increasing(dt) + cond = F.when(scol.eqNullSafe(F.lit(value).cast(dt)), cond).otherwise( + compare(scol, F.lit(value).cast(dt), spark.Column.__lt__) + ) + conds.append(cond) + + return reduce(lambda x, y: x & y, conds), None, None + + def _select_rows_by_iterable( + self, rows_sel: Iterable + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + rows_sel = list(rows_sel) + if len(rows_sel) == 0: + return F.lit(False), None, None + elif self._internal.index_level == 1: + index_column = self._kdf_or_kser.index.to_series() + index_data_type = index_column.spark.data_type + if len(rows_sel) == 1: + return ( + index_column.spark.column == F.lit(rows_sel[0]).cast(index_data_type), + None, + None, + ) + else: + return ( + index_column.spark.column.isin( + [F.lit(r).cast(index_data_type) for r in rows_sel] + ), + None, + None, + ) + else: + raise LocIndexer._NotImplemented("Cannot select with MultiIndex with Spark.") + + def _select_rows_else( + self, rows_sel: Any + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + if not isinstance(rows_sel, tuple): + rows_sel = (rows_sel,) + if len(rows_sel) > self._internal.index_level: + raise SparkPandasIndexingError("Too many indexers") + + rows = [scol == value for scol, value in zip(self._internal.index_spark_columns, rows_sel)] + return ( + reduce(lambda x, y: x & y, rows), + None, + self._internal.index_level - len(rows_sel), + ) + + def _get_from_multiindex_column( + self, key, missing_keys, labels=None, recursed=0 + ) -> Tuple[List[Tuple], Optional[List[spark.Column]], Any, bool, Optional[Tuple]]: + """ Select columns from multi-index columns. """ + assert isinstance(key, tuple) + if labels is None: + labels = [(label, label) for label in self._internal.column_labels] + for k in key: + labels = [ + (label, None if lbl is None else lbl[1:]) + for label, lbl in labels + if (lbl is None and k is None) or (lbl is not None and lbl[0] == k) + ] + if len(labels) == 0: + if missing_keys is None: + raise KeyError(k) + else: + missing_keys.append(key) + return [], [], [], False, None + + if all(lbl is not None and len(lbl) > 0 and lbl[0] == "" for _, lbl in labels): + # If the head is '', drill down recursively. + labels = [(label, tuple([str(key), *lbl[1:]])) for i, (label, lbl) in enumerate(labels)] + return self._get_from_multiindex_column((str(key),), missing_keys, labels, recursed + 1) + else: + returns_series = all(lbl is None or len(lbl) == 0 for _, lbl in labels) + if returns_series: + labels = set(label for label, _ in labels) + assert len(labels) == 1 + label = list(labels)[0] + column_labels = [label] + data_spark_columns = [self._internal.spark_column_for(label)] + data_dtypes = [self._internal.dtype_for(label)] + if label is None: + series_name = None + else: + if recursed > 0: + label = label[:-recursed] + series_name = label if len(label) > 1 else label[0] + else: + column_labels = [ + None if lbl is None or lbl == (None,) else lbl for _, lbl in labels + ] + data_spark_columns = [self._internal.spark_column_for(label) for label, _ in labels] + data_dtypes = [self._internal.dtype_for(label) for label, _ in labels] + series_name = None + + return column_labels, data_spark_columns, data_dtypes, returns_series, series_name + + def _select_cols_by_series( + self, cols_sel: "Series", missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + column_labels = [cols_sel._column_label] + data_spark_columns = [cols_sel.spark.column] + data_dtypes = [cols_sel.dtype] + return column_labels, data_spark_columns, data_dtypes, True, None + + def _select_cols_by_spark_column( + self, cols_sel: spark.Column, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + column_labels = [ + (self._internal.spark_frame.select(cols_sel).columns[0],) + ] # type: List[Tuple] + data_spark_columns = [cols_sel] + return column_labels, data_spark_columns, None, True, None + + def _select_cols_by_slice( + self, cols_sel: slice, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + start, stop = self._kdf_or_kser.columns.slice_locs(start=cols_sel.start, end=cols_sel.stop) + column_labels = self._internal.column_labels[start:stop] + data_spark_columns = self._internal.data_spark_columns[start:stop] + data_dtypes = self._internal.data_dtypes[start:stop] + return column_labels, data_spark_columns, data_dtypes, False, None + + def _select_cols_by_iterable( + self, cols_sel: Iterable, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + from pyspark.pandas.series import Series + + if all(isinstance(key, Series) for key in cols_sel): + column_labels = [key._column_label for key in cols_sel] + data_spark_columns = [key.spark.column for key in cols_sel] + data_dtypes = [key.dtype for key in cols_sel] + elif all(isinstance(key, spark.Column) for key in cols_sel): + column_labels = [ + (self._internal.spark_frame.select(col).columns[0],) for col in cols_sel + ] + data_spark_columns = list(cols_sel) + data_dtypes = None + elif all(isinstance(key, bool) for key in cols_sel) or all( + isinstance(key, np.bool_) for key in cols_sel + ): + if len(cast(Sized, cols_sel)) != len(self._internal.column_labels): + raise IndexError( + "Boolean index has wrong length: %s instead of %s" + % (len(cast(Sized, cols_sel)), len(self._internal.column_labels)) + ) + if isinstance(cols_sel, pd.Series): + if not cols_sel.index.sort_values().equals(self._kdf.columns.sort_values()): + raise SparkPandasIndexingError( + "Unalignable boolean Series provided as indexer " + "(index of the boolean Series and of the indexed object do not match)" + ) + else: + column_labels = [ + column_label + for column_label in self._internal.column_labels + if cols_sel[column_label if len(column_label) > 1 else column_label[0]] + ] + data_spark_columns = [ + self._internal.spark_column_for(column_label) + for column_label in column_labels + ] + data_dtypes = [ + self._internal.dtype_for(column_label) for column_label in column_labels + ] + else: + column_labels = [ + self._internal.column_labels[i] for i, col in enumerate(cols_sel) if col + ] + data_spark_columns = [ + self._internal.data_spark_columns[i] for i, col in enumerate(cols_sel) if col + ] + data_dtypes = [ + self._internal.data_dtypes[i] for i, col in enumerate(cols_sel) if col + ] + elif any(isinstance(key, tuple) for key in cols_sel) and any( + not is_name_like_tuple(key) for key in cols_sel + ): + raise TypeError( + "Expected tuple, got {}".format( + type(set(key for key in cols_sel if not is_name_like_tuple(key)).pop()) + ) + ) + else: + if missing_keys is None and all(isinstance(key, tuple) for key in cols_sel): + level = self._internal.column_labels_level + if any(len(key) != level for key in cols_sel): + raise ValueError("All the key level should be the same as column index level.") + + column_labels = [] + data_spark_columns = [] + data_dtypes = [] + for key in cols_sel: + found = False + for label in self._internal.column_labels: + if label == key or label[0] == key: + column_labels.append(label) + data_spark_columns.append(self._internal.spark_column_for(label)) + data_dtypes.append(self._internal.dtype_for(label)) + found = True + if not found: + if missing_keys is None: + raise KeyError("['{}'] not in index".format(name_like_string(key))) + else: + missing_keys.append(key) + + return column_labels, data_spark_columns, data_dtypes, False, None + + def _select_cols_else( + self, cols_sel: Any, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + if not is_name_like_tuple(cols_sel): + cols_sel = (cols_sel,) + return self._get_from_multiindex_column(cols_sel, missing_keys) + + +class iLocIndexer(LocIndexerLike): + """ + Purely integer-location based indexing for selection by position. + + ``.iloc[]`` is primarily integer position based (from ``0`` to + ``length-1`` of the axis), but may also be used with a conditional boolean Series. + + Allowed inputs are: + + - An integer for column selection, e.g. ``5``. + - A list or array of integers for row selection with distinct index values, + e.g. ``[3, 4, 0]`` + - A list or array of integers for column selection, e.g. ``[4, 3, 0]``. + - A boolean array for column selection. + - A slice object with ints for row and column selection, e.g. ``1:7``. + + Not allowed inputs which pandas allows are: + + - A list or array of integers for row selection with duplicated indexes, + e.g. ``[4, 4, 0]``. + - A boolean array for row selection. + - A ``callable`` function with one argument (the calling Series, DataFrame + or Panel) and that returns valid output for indexing (one of the above). + This is useful in method chains, when you don't have a reference to the + calling object, but would like to base your selection on some value. + + ``.iloc`` will raise ``IndexError`` if a requested indexer is + out-of-bounds, except *slice* indexers which allow out-of-bounds + indexing (this conforms with python/numpy *slice* semantics). + + See Also + -------- + DataFrame.loc : Purely label-location based indexer for selection by label. + Series.iloc : Purely integer-location based indexing for + selection by position. + + Examples + -------- + + >>> mydict = [{'a': 1, 'b': 2, 'c': 3, 'd': 4}, + ... {'a': 100, 'b': 200, 'c': 300, 'd': 400}, + ... {'a': 1000, 'b': 2000, 'c': 3000, 'd': 4000 }] + >>> df = pp.DataFrame(mydict, columns=['a', 'b', 'c', 'd']) + >>> df + a b c d + 0 1 2 3 4 + 1 100 200 300 400 + 2 1000 2000 3000 4000 + + **Indexing just the rows** + + A scalar integer for row selection. + + >>> df.iloc[1] + a 100 + b 200 + c 300 + d 400 + Name: 1, dtype: int64 + + >>> df.iloc[[0]] + a b c d + 0 1 2 3 4 + + With a `slice` object. + + >>> df.iloc[:3] + a b c d + 0 1 2 3 4 + 1 100 200 300 400 + 2 1000 2000 3000 4000 + + **Indexing both axes** + + You can mix the indexer types for the index and columns. Use ``:`` to + select the entire axis. + + With scalar integers. + + >>> df.iloc[:1, 1] + 0 2 + Name: b, dtype: int64 + + With lists of integers. + + >>> df.iloc[:2, [1, 3]] + b d + 0 2 4 + 1 200 400 + + With `slice` objects. + + >>> df.iloc[:2, 0:3] + a b c + 0 1 2 3 + 1 100 200 300 + + With a boolean array whose length matches the columns. + + >>> df.iloc[:, [True, False, True, False]] + a c + 0 1 3 + 1 100 300 + 2 1000 3000 + + **Setting values** + + Setting value for all items matching the list of labels. + + >>> df.iloc[[1, 2], [1]] = 50 + >>> df + a b c d + 0 1 2 3 4 + 1 100 50 300 400 + 2 1000 50 3000 4000 + + Setting value for an entire row + + >>> df.iloc[0] = 10 + >>> df + a b c d + 0 10 10 10 10 + 1 100 50 300 400 + 2 1000 50 3000 4000 + + Set value for an entire column + + >>> df.iloc[:, 2] = 30 + >>> df + a b c d + 0 10 10 30 10 + 1 100 50 30 400 + 2 1000 50 30 4000 + + Set value for an entire list of columns + + >>> df.iloc[:, [2, 3]] = 100 + >>> df + a b c d + 0 10 10 100 100 + 1 100 50 100 100 + 2 1000 50 100 100 + + Set value with Series + + >>> df.iloc[:, 3] = df.iloc[:, 3] * 2 + >>> df + a b c d + 0 10 10 100 200 + 1 100 50 100 200 + 2 1000 50 100 200 + """ + + @staticmethod + def _NotImplemented(description): + return SparkPandasNotImplementedError( + description=description, + pandas_function=".iloc[..., ...]", + spark_target_function="select, where", + ) + + @lazy_property + def _internal(self): + # Use resolved_copy to fix the natural order. + internal = super()._internal.resolved_copy + sdf = InternalFrame.attach_distributed_sequence_column( + internal.spark_frame, column_name=self._sequence_col + ) + return internal.with_new_sdf(spark_frame=sdf.orderBy(NATURAL_ORDER_COLUMN_NAME)) + + @lazy_property + def _sequence_col(self): + # Use resolved_copy to fix the natural order. + internal = super()._internal.resolved_copy + return verify_temp_column_name(internal.spark_frame, "__distributed_sequence_column__") + + def _select_rows_by_series( + self, rows_sel: "Series" + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + raise iLocIndexer._NotImplemented( + ".iloc requires numeric slice, conditional " + "boolean Index or a sequence of positions as int, " + "got {}".format(type(rows_sel)) + ) + + def _select_rows_by_spark_column( + self, rows_sel: spark.column + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + raise iLocIndexer._NotImplemented( + ".iloc requires numeric slice, conditional " + "boolean Index or a sequence of positions as int, " + "got {}".format(type(rows_sel)) + ) + + def _select_rows_by_slice( + self, rows_sel: slice + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + def verify_type(i): + if not isinstance(i, int): + raise TypeError( + "cannot do slice indexing with these indexers [{}] of {}".format(i, type(i)) + ) + + has_negative = False + start = rows_sel.start + if start is not None: + verify_type(start) + if start == 0: + start = None + elif start < 0: + has_negative = True + stop = rows_sel.stop + if stop is not None: + verify_type(stop) + if stop < 0: + has_negative = True + + step = rows_sel.step + if step is not None: + verify_type(step) + if step == 0: + raise ValueError("slice step cannot be zero") + else: + step = 1 + + if start is None and step == 1: + return None, stop, None + + sdf = self._internal.spark_frame + sequence_scol = sdf[self._sequence_col] + + if has_negative or (step < 0 and start is None): + cnt = sdf.count() + + cond = [] + if start is not None: + if start < 0: + start = start + cnt + if step >= 0: + cond.append(sequence_scol >= F.lit(start).cast(LongType())) + else: + cond.append(sequence_scol <= F.lit(start).cast(LongType())) + if stop is not None: + if stop < 0: + stop = stop + cnt + if step >= 0: + cond.append(sequence_scol < F.lit(stop).cast(LongType())) + else: + cond.append(sequence_scol > F.lit(stop).cast(LongType())) + if step != 1: + if step > 0: + start = start or 0 + else: + start = start or (cnt - 1) + cond.append(((sequence_scol - start) % F.lit(step).cast(LongType())) == F.lit(0)) + + return reduce(lambda x, y: x & y, cond), None, None + + def _select_rows_by_iterable( + self, rows_sel: Iterable + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + sdf = self._internal.spark_frame + + if any(isinstance(key, (int, np.int, np.int64, np.int32)) and key < 0 for key in rows_sel): + offset = sdf.count() + else: + offset = 0 + + new_rows_sel = [] + for key in list(rows_sel): + if not isinstance(key, (int, np.int, np.int64, np.int32)): + raise TypeError( + "cannot do positional indexing with these indexers [{}] of {}".format( + key, type(key) + ) + ) + if key < 0: + key = key + offset + new_rows_sel.append(key) + + if len(new_rows_sel) != len(set(new_rows_sel)): + raise NotImplementedError( + "Duplicated row selection is not currently supported; " + "however, normalised index was [%s]" % new_rows_sel + ) + + sequence_scol = sdf[self._sequence_col] + cond = [] + for key in new_rows_sel: + cond.append(sequence_scol == F.lit(int(key)).cast(LongType())) + + if len(cond) == 0: + cond = [F.lit(False)] + return reduce(lambda x, y: x | y, cond), None, None + + def _select_rows_else( + self, rows_sel: Any + ) -> Tuple[Optional[spark.Column], Optional[int], Optional[int]]: + if isinstance(rows_sel, int): + sdf = self._internal.spark_frame + return (sdf[self._sequence_col] == rows_sel), None, 0 + elif isinstance(rows_sel, tuple): + raise SparkPandasIndexingError("Too many indexers") + else: + raise iLocIndexer._NotImplemented( + ".iloc requires numeric slice, conditional " + "boolean Index or a sequence of positions as int, " + "got {}".format(type(rows_sel)) + ) + + def _select_cols_by_series( + self, cols_sel: "Series", missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + raise ValueError( + "Location based indexing can only have [integer, integer slice, " + "listlike of integers, boolean array] types, got {}".format(cols_sel) + ) + + def _select_cols_by_spark_column( + self, cols_sel: spark.Column, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + raise ValueError( + "Location based indexing can only have [integer, integer slice, " + "listlike of integers, boolean array] types, got {}".format(cols_sel) + ) + + def _select_cols_by_slice( + self, cols_sel: slice, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + if all( + s is None or isinstance(s, int) for s in (cols_sel.start, cols_sel.stop, cols_sel.step) + ): + column_labels = self._internal.column_labels[cols_sel] + data_spark_columns = self._internal.data_spark_columns[cols_sel] + data_dtypes = self._internal.data_dtypes[cols_sel] + return column_labels, data_spark_columns, data_dtypes, False, None + else: + not_none = ( + cols_sel.start + if cols_sel.start is not None + else cols_sel.stop + if cols_sel.stop is not None + else cols_sel.step + ) + raise TypeError( + "cannot do slice indexing with these indexers {} of {}".format( + not_none, type(not_none) + ) + ) + + def _select_cols_by_iterable( + self, cols_sel: Iterable, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + if all(isinstance(s, bool) for s in cols_sel): + cols_sel = [i for i, s in enumerate(cols_sel) if s] + if all(isinstance(s, int) for s in cols_sel): + column_labels = [self._internal.column_labels[s] for s in cols_sel] + data_spark_columns = [self._internal.data_spark_columns[s] for s in cols_sel] + data_dtypes = [self._internal.data_dtypes[s] for s in cols_sel] + return column_labels, data_spark_columns, data_dtypes, False, None + else: + raise TypeError("cannot perform reduce with flexible type") + + def _select_cols_else( + self, cols_sel: Any, missing_keys: Optional[List[Tuple]] + ) -> Tuple[ + List[Tuple], Optional[List[spark.Column]], Optional[List[Dtype]], bool, Optional[Tuple] + ]: + if isinstance(cols_sel, int): + if cols_sel > len(self._internal.column_labels): + raise KeyError(cols_sel) + column_labels = [self._internal.column_labels[cols_sel]] + data_spark_columns = [self._internal.data_spark_columns[cols_sel]] + data_dtypes = [self._internal.data_dtypes[cols_sel]] + return column_labels, data_spark_columns, data_dtypes, True, None + else: + raise ValueError( + "Location based indexing can only have [integer, integer slice, " + "listlike of integers, boolean array] types, got {}".format(cols_sel) + ) + + def __setitem__(self, key, value): + super().__setitem__(key, value) + # Update again with resolved_copy to drop extra columns. + self._kdf._update_internal_frame( + self._kdf._internal.resolved_copy, requires_same_anchor=False + ) + + # Clean up implicitly cached properties to be able to reuse the indexer. + del self._internal + del self._sequence_col diff --git a/python/pyspark/pandas/internal.py b/python/pyspark/pandas/internal.py new file mode 100644 index 0000000000..94abe97c2c --- /dev/null +++ b/python/pyspark/pandas/internal.py @@ -0,0 +1,1438 @@ +# +# 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. +# + +""" +An internal immutable DataFrame with some metadata to manage indexes. +""" +from distutils.version import LooseVersion +import re +from typing import Dict, List, Optional, Tuple, Union, TYPE_CHECKING +from itertools import accumulate +import py4j + +import numpy as np +import pandas as pd +from pandas.api.types import CategoricalDtype, is_datetime64_dtype, is_datetime64tz_dtype +import pyspark +from pyspark import sql as spark +from pyspark._globals import _NoValue, _NoValueType +from pyspark.sql import functions as F, Window +from pyspark.sql.functions import PandasUDFType, pandas_udf +from pyspark.sql.types import BooleanType, DataType, IntegralType, StructField, StructType, LongType + +try: + from pyspark.sql.types import to_arrow_type +except ImportError: + from pyspark.sql.pandas.types import to_arrow_type # noqa: F401 + +# For running doctests and reference resolution in PyCharm. +from pyspark import pandas as pp # noqa: F401 + +if TYPE_CHECKING: + # This is required in old Python 3.5 to prevent circular reference. + from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943) +from pyspark.pandas.config import get_option +from pyspark.pandas.typedef import ( + Dtype, + as_spark_type, + extension_dtypes, + infer_pd_series_spark_type, + spark_type_to_pandas_dtype, +) +from pyspark.pandas.utils import ( + column_labels_level, + default_session, + is_name_like_tuple, + is_testing, + lazy_property, + name_like_string, + scol_for, + verify_temp_column_name, +) + + +# A function to turn given numbers to Spark columns that represent Koalas index. +SPARK_INDEX_NAME_FORMAT = "__index_level_{}__".format +SPARK_DEFAULT_INDEX_NAME = SPARK_INDEX_NAME_FORMAT(0) +# A pattern to check if the name of a Spark column is a Koalas index name or not. +SPARK_INDEX_NAME_PATTERN = re.compile(r"__index_level_[0-9]+__") + +NATURAL_ORDER_COLUMN_NAME = "__natural_order__" + +HIDDEN_COLUMNS = {NATURAL_ORDER_COLUMN_NAME} + +DEFAULT_SERIES_NAME = 0 +SPARK_DEFAULT_SERIES_NAME = str(DEFAULT_SERIES_NAME) + + +class InternalFrame(object): + """ + The internal immutable DataFrame which manages Spark DataFrame and column names and index + information. + + .. note:: this is an internal class. It is not supposed to be exposed to users and users + should not directly access to it. + + The internal immutable DataFrame represents the index information for a DataFrame it belongs to. + For instance, if we have a Koalas DataFrame as below, pandas DataFrame does not store the index + as columns. + + >>> kdf = pp.DataFrame({ + ... 'A': [1, 2, 3, 4], + ... 'B': [5, 6, 7, 8], + ... 'C': [9, 10, 11, 12], + ... 'D': [13, 14, 15, 16], + ... 'E': [17, 18, 19, 20]}, columns = ['A', 'B', 'C', 'D', 'E']) + >>> kdf # doctest: +NORMALIZE_WHITESPACE + A B C D E + 0 1 5 9 13 17 + 1 2 6 10 14 18 + 2 3 7 11 15 19 + 3 4 8 12 16 20 + + However, all columns including index column are also stored in Spark DataFrame internally + as below. + + >>> kdf._internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE + +-----------------+---+---+---+---+---+ + |__index_level_0__| A| B| C| D| E| + +-----------------+---+---+---+---+---+ + | 0| 1| 5| 9| 13| 17| + | 1| 2| 6| 10| 14| 18| + | 2| 3| 7| 11| 15| 19| + | 3| 4| 8| 12| 16| 20| + +-----------------+---+---+---+---+---+ + + In order to fill this gap, the current metadata is used by mapping Spark's internal column + to Koalas' index. See the method below: + + * `spark_frame` represents the internal Spark DataFrame + + * `data_spark_column_names` represents non-indexing Spark column names + + * `data_spark_columns` represents non-indexing Spark columns + + * `data_dtypes` represents external non-indexing dtypes + + * `index_spark_column_names` represents internal index Spark column names + + * `index_spark_columns` represents internal index Spark columns + + * `index_dtypes` represents external index dtypes + + * `spark_column_names` represents all columns + + * `index_names` represents the external index name as a label + + * `to_internal_spark_frame` represents Spark DataFrame derived by the metadata. Includes index. + + * `to_pandas_frame` represents pandas DataFrame derived by the metadata + + >>> internal = kdf._internal + >>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS + +-----------------+---+---+---+---+---+-----------------+ + |__index_level_0__| A| B| C| D| E|__natural_order__| + +-----------------+---+---+---+---+---+-----------------+ + | 0| 1| 5| 9| 13| 17| ...| + | 1| 2| 6| 10| 14| 18| ...| + | 2| 3| 7| 11| 15| 19| ...| + | 3| 4| 8| 12| 16| 20| ...| + +-----------------+---+---+---+---+---+-----------------+ + >>> internal.data_spark_column_names + ['A', 'B', 'C', 'D', 'E'] + >>> internal.index_spark_column_names + ['__index_level_0__'] + >>> internal.spark_column_names + ['__index_level_0__', 'A', 'B', 'C', 'D', 'E'] + >>> internal.index_names + [None] + >>> internal.data_dtypes + [dtype('int64'), dtype('int64'), dtype('int64'), dtype('int64'), dtype('int64')] + >>> internal.index_dtypes + [dtype('int64')] + >>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE + +-----------------+---+---+---+---+---+ + |__index_level_0__| A| B| C| D| E| + +-----------------+---+---+---+---+---+ + | 0| 1| 5| 9| 13| 17| + | 1| 2| 6| 10| 14| 18| + | 2| 3| 7| 11| 15| 19| + | 3| 4| 8| 12| 16| 20| + +-----------------+---+---+---+---+---+ + >>> internal.to_pandas_frame + A B C D E + 0 1 5 9 13 17 + 1 2 6 10 14 18 + 2 3 7 11 15 19 + 3 4 8 12 16 20 + + In case that index is set to one of the existing column as below: + + >>> kdf1 = kdf.set_index("A") + >>> kdf1 # doctest: +NORMALIZE_WHITESPACE + B C D E + A + 1 5 9 13 17 + 2 6 10 14 18 + 3 7 11 15 19 + 4 8 12 16 20 + + >>> kdf1._internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE + +---+---+---+---+---+ + | A| B| C| D| E| + +---+---+---+---+---+ + | 1| 5| 9| 13| 17| + | 2| 6| 10| 14| 18| + | 3| 7| 11| 15| 19| + | 4| 8| 12| 16| 20| + +---+---+---+---+---+ + + >>> internal = kdf1._internal + >>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS + +-----------------+---+---+---+---+---+-----------------+ + |__index_level_0__| A| B| C| D| E|__natural_order__| + +-----------------+---+---+---+---+---+-----------------+ + | 0| 1| 5| 9| 13| 17| ...| + | 1| 2| 6| 10| 14| 18| ...| + | 2| 3| 7| 11| 15| 19| ...| + | 3| 4| 8| 12| 16| 20| ...| + +-----------------+---+---+---+---+---+-----------------+ + >>> internal.data_spark_column_names + ['B', 'C', 'D', 'E'] + >>> internal.index_spark_column_names + ['A'] + >>> internal.spark_column_names + ['A', 'B', 'C', 'D', 'E'] + >>> internal.index_names + [('A',)] + >>> internal.data_dtypes + [dtype('int64'), dtype('int64'), dtype('int64'), dtype('int64')] + >>> internal.index_dtypes + [dtype('int64')] + >>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE + +---+---+---+---+---+ + | A| B| C| D| E| + +---+---+---+---+---+ + | 1| 5| 9| 13| 17| + | 2| 6| 10| 14| 18| + | 3| 7| 11| 15| 19| + | 4| 8| 12| 16| 20| + +---+---+---+---+---+ + >>> internal.to_pandas_frame # doctest: +NORMALIZE_WHITESPACE + B C D E + A + 1 5 9 13 17 + 2 6 10 14 18 + 3 7 11 15 19 + 4 8 12 16 20 + + In case that index becomes a multi index as below: + + >>> kdf2 = kdf.set_index("A", append=True) + >>> kdf2 # doctest: +NORMALIZE_WHITESPACE + B C D E + A + 0 1 5 9 13 17 + 1 2 6 10 14 18 + 2 3 7 11 15 19 + 3 4 8 12 16 20 + + >>> kdf2._internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE + +-----------------+---+---+---+---+---+ + |__index_level_0__| A| B| C| D| E| + +-----------------+---+---+---+---+---+ + | 0| 1| 5| 9| 13| 17| + | 1| 2| 6| 10| 14| 18| + | 2| 3| 7| 11| 15| 19| + | 3| 4| 8| 12| 16| 20| + +-----------------+---+---+---+---+---+ + + >>> internal = kdf2._internal + >>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS + +-----------------+---+---+---+---+---+-----------------+ + |__index_level_0__| A| B| C| D| E|__natural_order__| + +-----------------+---+---+---+---+---+-----------------+ + | 0| 1| 5| 9| 13| 17| ...| + | 1| 2| 6| 10| 14| 18| ...| + | 2| 3| 7| 11| 15| 19| ...| + | 3| 4| 8| 12| 16| 20| ...| + +-----------------+---+---+---+---+---+-----------------+ + >>> internal.data_spark_column_names + ['B', 'C', 'D', 'E'] + >>> internal.index_spark_column_names + ['__index_level_0__', 'A'] + >>> internal.spark_column_names + ['__index_level_0__', 'A', 'B', 'C', 'D', 'E'] + >>> internal.index_names + [None, ('A',)] + >>> internal.data_dtypes + [dtype('int64'), dtype('int64'), dtype('int64'), dtype('int64')] + >>> internal.index_dtypes + [dtype('int64'), dtype('int64')] + >>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE + +-----------------+---+---+---+---+---+ + |__index_level_0__| A| B| C| D| E| + +-----------------+---+---+---+---+---+ + | 0| 1| 5| 9| 13| 17| + | 1| 2| 6| 10| 14| 18| + | 2| 3| 7| 11| 15| 19| + | 3| 4| 8| 12| 16| 20| + +-----------------+---+---+---+---+---+ + >>> internal.to_pandas_frame # doctest: +NORMALIZE_WHITESPACE + B C D E + A + 0 1 5 9 13 17 + 1 2 6 10 14 18 + 2 3 7 11 15 19 + 3 4 8 12 16 20 + + For multi-level columns, it also holds column_labels + + >>> columns = pd.MultiIndex.from_tuples([('X', 'A'), ('X', 'B'), + ... ('Y', 'C'), ('Y', 'D')]) + >>> kdf3 = pp.DataFrame([ + ... [1, 2, 3, 4], + ... [5, 6, 7, 8], + ... [9, 10, 11, 12], + ... [13, 14, 15, 16], + ... [17, 18, 19, 20]], columns = columns) + >>> kdf3 # doctest: +NORMALIZE_WHITESPACE + X Y + A B C D + 0 1 2 3 4 + 1 5 6 7 8 + 2 9 10 11 12 + 3 13 14 15 16 + 4 17 18 19 20 + + >>> internal = kdf3._internal + >>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS + +-----------------+------+------+------+------+-----------------+ + |__index_level_0__|(X, A)|(X, B)|(Y, C)|(Y, D)|__natural_order__| + +-----------------+------+------+------+------+-----------------+ + | 0| 1| 2| 3| 4| ...| + | 1| 5| 6| 7| 8| ...| + | 2| 9| 10| 11| 12| ...| + | 3| 13| 14| 15| 16| ...| + | 4| 17| 18| 19| 20| ...| + +-----------------+------+------+------+------+-----------------+ + >>> internal.data_spark_column_names + ['(X, A)', '(X, B)', '(Y, C)', '(Y, D)'] + >>> internal.column_labels + [('X', 'A'), ('X', 'B'), ('Y', 'C'), ('Y', 'D')] + + For Series, it also holds scol to represent the column. + + >>> kseries = kdf1.B + >>> kseries + A + 1 5 + 2 6 + 3 7 + 4 8 + Name: B, dtype: int64 + + >>> internal = kseries._internal + >>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS + +-----------------+---+---+---+---+---+-----------------+ + |__index_level_0__| A| B| C| D| E|__natural_order__| + +-----------------+---+---+---+---+---+-----------------+ + | 0| 1| 5| 9| 13| 17| ...| + | 1| 2| 6| 10| 14| 18| ...| + | 2| 3| 7| 11| 15| 19| ...| + | 3| 4| 8| 12| 16| 20| ...| + +-----------------+---+---+---+---+---+-----------------+ + >>> internal.data_spark_column_names + ['B'] + >>> internal.index_spark_column_names + ['A'] + >>> internal.spark_column_names + ['A', 'B'] + >>> internal.index_names + [('A',)] + >>> internal.data_dtypes + [dtype('int64')] + >>> internal.index_dtypes + [dtype('int64')] + >>> internal.to_internal_spark_frame.show() # doctest: +NORMALIZE_WHITESPACE + +---+---+ + | A| B| + +---+---+ + | 1| 5| + | 2| 6| + | 3| 7| + | 4| 8| + +---+---+ + >>> internal.to_pandas_frame # doctest: +NORMALIZE_WHITESPACE + B + A + 1 5 + 2 6 + 3 7 + 4 8 + """ + + def __init__( + self, + spark_frame: spark.DataFrame, + index_spark_columns: Optional[List[spark.Column]], + index_names: Optional[List[Optional[Tuple]]] = None, + index_dtypes: Optional[List[Dtype]] = None, + column_labels: Optional[List[Tuple]] = None, + data_spark_columns: Optional[List[spark.Column]] = None, + data_dtypes: Optional[List[Dtype]] = None, + column_label_names: Optional[List[Optional[Tuple]]] = None, + ) -> None: + """ + Create a new internal immutable DataFrame to manage Spark DataFrame, column fields and + index fields and names. + + :param spark_frame: Spark DataFrame to be managed. + :param index_spark_columns: list of Spark Column + Spark Columns for the index. + :param index_names: list of tuples + the index names. + :param index_dtypes: list of dtypes + the index dtypes. + :param column_labels: list of tuples with the same length + The multi-level values in the tuples. + :param data_spark_columns: list of Spark Column + Spark Columns to appear as columns. If this is None, calculated + from spark_frame. + :param data_dtypes: list of dtypes. + the data dtypes. + :param column_label_names: Names for each of the column index levels. + + See the examples below to refer what each parameter means. + + >>> column_labels = pd.MultiIndex.from_tuples( + ... [('a', 'x'), ('a', 'y'), ('b', 'z')], names=["column_labels_a", "column_labels_b"]) + >>> row_index = pd.MultiIndex.from_tuples( + ... [('foo', 'bar'), ('foo', 'bar'), ('zoo', 'bar')], + ... names=["row_index_a", "row_index_b"]) + >>> kdf = pp.DataFrame( + ... [[1, 2, 3], [4, 5, 6], [7, 8, 9]], index=row_index, columns=column_labels) + >>> kdf.set_index(('a', 'x'), append=True, inplace=True) + >>> kdf # doctest: +NORMALIZE_WHITESPACE + column_labels_a a b + column_labels_b y z + row_index_a row_index_b (a, x) + foo bar 1 2 3 + 4 5 6 + zoo bar 7 8 9 + + >>> internal = kdf._internal + + >>> internal.spark_frame.show() # doctest: +NORMALIZE_WHITESPACE +ELLIPSIS + +-----------------+-----------------+------+------+------+... + |__index_level_0__|__index_level_1__|(a, x)|(a, y)|(b, z)|... + +-----------------+-----------------+------+------+------+... + | foo| bar| 1| 2| 3|... + | foo| bar| 4| 5| 6|... + | zoo| bar| 7| 8| 9|... + +-----------------+-----------------+------+------+------+... + + >>> internal.index_spark_columns # doctest: +SKIP + [Column<'__index_level_0__'>, Column<'__index_level_1__'>, Column<'(a, x)'>] + + >>> internal.index_names + [('row_index_a',), ('row_index_b',), ('a', 'x')] + + >>> internal.index_dtypes + [dtype('O'), dtype('O'), dtype('int64')] + + >>> internal.column_labels + [('a', 'y'), ('b', 'z')] + + >>> internal.data_spark_columns # doctest: +SKIP + [Column<'(a, y)'>, Column<'(b, z)'>] + + >>> internal.data_dtypes + [dtype('int64'), dtype('int64')] + + >>> internal.column_label_names + [('column_labels_a',), ('column_labels_b',)] + """ + + assert isinstance(spark_frame, spark.DataFrame) + assert not spark_frame.isStreaming, "Koalas does not support Structured Streaming." + + if not index_spark_columns: + if data_spark_columns is not None: + if column_labels is not None: + data_spark_columns = [ + scol.alias(name_like_string(label)) + for scol, label in zip(data_spark_columns, column_labels) + ] + spark_frame = spark_frame.select(data_spark_columns) + + assert not any(SPARK_INDEX_NAME_PATTERN.match(name) for name in spark_frame.columns), ( + "Index columns should not appear in columns of the Spark DataFrame. Avoid " + "index column names [%s]." % SPARK_INDEX_NAME_PATTERN + ) + + # Create default index. + spark_frame = InternalFrame.attach_default_index(spark_frame) + index_spark_columns = [scol_for(spark_frame, SPARK_DEFAULT_INDEX_NAME)] + + if data_spark_columns is not None: + data_spark_columns = [ + scol_for(spark_frame, col) + for col in spark_frame.columns + if col != SPARK_DEFAULT_INDEX_NAME + ] + + if NATURAL_ORDER_COLUMN_NAME not in spark_frame.columns: + spark_frame = spark_frame.withColumn( + NATURAL_ORDER_COLUMN_NAME, F.monotonically_increasing_id() + ) + + self._sdf = spark_frame # type: spark.DataFrame + + # index_spark_columns + assert all( + isinstance(index_scol, spark.Column) for index_scol in index_spark_columns + ), index_spark_columns + + self._index_spark_columns = index_spark_columns # type: List[spark.Column] + + # index_names + if not index_names: + index_names = [None] * len(index_spark_columns) + + assert len(index_spark_columns) == len(index_names), ( + len(index_spark_columns), + len(index_names), + ) + assert all( + is_name_like_tuple(index_name, check_type=True) for index_name in index_names + ), index_names + + self._index_names = index_names # type: List[Optional[Tuple]] + + # index_dtypes + if not index_dtypes: + index_dtypes = [None] * len(index_spark_columns) + + assert len(index_spark_columns) == len(index_dtypes), ( + len(index_spark_columns), + len(index_dtypes), + ) + + index_dtypes = [ + spark_type_to_pandas_dtype(spark_frame.select(scol).schema[0].dataType) + if dtype is None or dtype == np.dtype("object") + else dtype + for dtype, scol in zip(index_dtypes, index_spark_columns) + ] + + assert all( + isinstance(dtype, Dtype.__args__) # type: ignore + and (dtype == np.dtype("object") or as_spark_type(dtype, raise_error=False) is not None) + for dtype in index_dtypes + ), index_dtypes + + self._index_dtypes = index_dtypes + + # data_spark-columns + if data_spark_columns is None: + data_spark_columns = [ + scol_for(spark_frame, col) + for col in spark_frame.columns + if all( + not scol_for(spark_frame, col)._jc.equals(index_scol._jc) + for index_scol in index_spark_columns + ) + and col not in HIDDEN_COLUMNS + ] + self._data_spark_columns = data_spark_columns + else: + assert all(isinstance(scol, spark.Column) for scol in data_spark_columns) + self._data_spark_columns = data_spark_columns + + # column_labels + if column_labels is None: + self._column_labels = [ + (col,) for col in spark_frame.select(self._data_spark_columns).columns + ] # type: List[Tuple] + else: + assert len(column_labels) == len(self._data_spark_columns), ( + len(column_labels), + len(self._data_spark_columns), + ) + if len(column_labels) == 1: + column_label = column_labels[0] + assert is_name_like_tuple(column_label, check_type=True), column_label + else: + assert all( + is_name_like_tuple(column_label, check_type=True) + for column_label in column_labels + ), column_labels + assert len(set(len(label) for label in column_labels)) <= 1, column_labels + self._column_labels = column_labels + + # data_dtypes + if not data_dtypes: + data_dtypes = [None] * len(data_spark_columns) + + assert len(data_spark_columns) == len(data_dtypes), ( + len(data_spark_columns), + len(data_dtypes), + ) + + data_dtypes = [ + spark_type_to_pandas_dtype(spark_frame.select(scol).schema[0].dataType) + if dtype is None or dtype == np.dtype("object") + else dtype + for dtype, scol in zip(data_dtypes, data_spark_columns) + ] + + assert all( + isinstance(dtype, Dtype.__args__) # type: ignore + and (dtype == np.dtype("object") or as_spark_type(dtype, raise_error=False) is not None) + for dtype in data_dtypes + ), data_dtypes + + self._data_dtypes = data_dtypes + + # column_label_names + if column_label_names is None: + self._column_label_names = [None] * column_labels_level( + self._column_labels + ) # type: List[Optional[Tuple]] + else: + if len(self._column_labels) > 0: + assert len(column_label_names) == column_labels_level(self._column_labels), ( + len(column_label_names), + column_labels_level(self._column_labels), + ) + else: + assert len(column_label_names) > 0, len(column_label_names) + assert all( + is_name_like_tuple(column_label_name, check_type=True) + for column_label_name in column_label_names + ), column_label_names + self._column_label_names = column_label_names + + @staticmethod + def attach_default_index(sdf, default_index_type=None): + """ + This method attaches a default index to Spark DataFrame. Spark does not have the index + notion so corresponding column should be generated. + There are several types of default index can be configured by `compute.default_index_type`. + + >>> spark_frame = pp.range(10).to_spark() + >>> spark_frame + DataFrame[id: bigint] + + It adds the default index column '__index_level_0__'. + + >>> spark_frame = InternalFrame.attach_default_index(spark_frame) + >>> spark_frame + DataFrame[__index_level_0__: bigint, id: bigint] + + It throws an exception if the given column name already exists. + + >>> InternalFrame.attach_default_index(spark_frame) + ... # doctest: +ELLIPSIS + Traceback (most recent call last): + ... + AssertionError: '__index_level_0__' already exists... + """ + index_column = SPARK_DEFAULT_INDEX_NAME + assert ( + index_column not in sdf.columns + ), "'%s' already exists in the Spark column names '%s'" % (index_column, sdf.columns) + + if default_index_type is None: + default_index_type = get_option("compute.default_index_type") + + if default_index_type == "sequence": + return InternalFrame.attach_sequence_column(sdf, column_name=index_column) + elif default_index_type == "distributed-sequence": + return InternalFrame.attach_distributed_sequence_column(sdf, column_name=index_column) + elif default_index_type == "distributed": + return InternalFrame.attach_distributed_column(sdf, column_name=index_column) + else: + raise ValueError( + "'compute.default_index_type' should be one of 'sequence'," + " 'distributed-sequence' and 'distributed'" + ) + + @staticmethod + def attach_sequence_column(sdf, column_name): + scols = [scol_for(sdf, column) for column in sdf.columns] + sequential_index = ( + F.row_number().over(Window.orderBy(F.monotonically_increasing_id())).cast("long") - 1 + ) + return sdf.select(sequential_index.alias(column_name), *scols) + + @staticmethod + def attach_distributed_column(sdf, column_name): + scols = [scol_for(sdf, column) for column in sdf.columns] + return sdf.select(F.monotonically_increasing_id().alias(column_name), *scols) + + @staticmethod + def attach_distributed_sequence_column(sdf, column_name): + """ + This method attaches a Spark column that has a sequence in a distributed manner. + This is equivalent to the column assigned when default index type 'distributed-sequence'. + + >>> sdf = pp.DataFrame(['a', 'b', 'c']).to_spark() + >>> sdf = InternalFrame.attach_distributed_sequence_column(sdf, column_name="sequence") + >>> sdf.show() # doctest: +NORMALIZE_WHITESPACE + +--------+---+ + |sequence| 0| + +--------+---+ + | 0| a| + | 1| b| + | 2| c| + +--------+---+ + """ + if len(sdf.columns) > 0: + try: + jdf = sdf._jdf.toDF() + + sql_ctx = sdf.sql_ctx + encoders = sql_ctx._jvm.org.apache.spark.sql.Encoders + encoder = encoders.tuple(jdf.exprEnc(), encoders.scalaLong()) + + jrdd = jdf.localCheckpoint(False).rdd().zipWithIndex() + + df = spark.DataFrame( + sql_ctx.sparkSession._jsparkSession.createDataset(jrdd, encoder).toDF(), sql_ctx + ) + columns = df.columns + return df.selectExpr( + "`{}` as `{}`".format(columns[1], column_name), "`{}`.*".format(columns[0]) + ) + except py4j.protocol.Py4JError: + if is_testing(): + raise + return InternalFrame._attach_distributed_sequence_column(sdf, column_name) + else: + cnt = sdf.count() + if cnt > 0: + return default_session().range(cnt).toDF(column_name) + else: + return default_session().createDataFrame( + [], schema=StructType().add(column_name, data_type=LongType(), nullable=False) + ) + + @staticmethod + def _attach_distributed_sequence_column(sdf, column_name): + """ + >>> sdf = pp.DataFrame(['a', 'b', 'c']).to_spark() + >>> sdf = InternalFrame._attach_distributed_sequence_column(sdf, column_name="sequence") + >>> sdf.sort("sequence").show() # doctest: +NORMALIZE_WHITESPACE + +--------+---+ + |sequence| 0| + +--------+---+ + | 0| a| + | 1| b| + | 2| c| + +--------+---+ + """ + scols = [scol_for(sdf, column) for column in sdf.columns] + + spark_partition_column = verify_temp_column_name(sdf, "__spark_partition_id__") + offset_column = verify_temp_column_name(sdf, "__offset__") + row_number_column = verify_temp_column_name(sdf, "__row_number__") + + # 1. Calculates counts per each partition ID. `counts` here is, for instance, + # { + # 1: 83, + # 6: 83, + # 3: 83, + # ... + # } + sdf = sdf.withColumn(spark_partition_column, F.spark_partition_id()) + + # Checkpoint the DataFrame to fix the partition ID. + sdf = sdf.localCheckpoint(eager=False) + + counts = map( + lambda x: (x["key"], x["count"]), + sdf.groupby(sdf[spark_partition_column].alias("key")).count().collect(), + ) + + # 2. Calculates cumulative sum in an order of partition id. + # Note that it does not matter if partition id guarantees its order or not. + # We just need a one-by-one sequential id. + + # sort by partition key. + sorted_counts = sorted(counts, key=lambda x: x[0]) + # get cumulative sum in an order of partition key. + cumulative_counts = [0] + list(accumulate(map(lambda count: count[1], sorted_counts))) + # zip it with partition key. + sums = dict(zip(map(lambda count: count[0], sorted_counts), cumulative_counts)) + + # 3. Attach offset for each partition. + @pandas_udf(LongType(), PandasUDFType.SCALAR) + def offset(id): + current_partition_offset = sums[id.iloc[0]] + return pd.Series(current_partition_offset).repeat(len(id)) + + sdf = sdf.withColumn(offset_column, offset(spark_partition_column)) + + # 4. Calculate row_number in each partition. + w = Window.partitionBy(spark_partition_column).orderBy(F.monotonically_increasing_id()) + row_number = F.row_number().over(w) + sdf = sdf.withColumn(row_number_column, row_number) + + # 5. Calculate the index. + return sdf.select( + (sdf[offset_column] + sdf[row_number_column] - 1).alias(column_name), *scols + ) + + def spark_column_for(self, label: Tuple) -> spark.Column: + """ Return Spark Column for the given column label. """ + column_labels_to_scol = dict(zip(self.column_labels, self.data_spark_columns)) + if label in column_labels_to_scol: + return column_labels_to_scol[label] + else: + raise KeyError(name_like_string(label)) + + def spark_column_name_for(self, label_or_scol: Union[Tuple, spark.Column]) -> str: + """ Return the actual Spark column name for the given column label. """ + if isinstance(label_or_scol, spark.Column): + scol = label_or_scol + else: + scol = self.spark_column_for(label_or_scol) + return self.spark_frame.select(scol).columns[0] + + def spark_type_for(self, label_or_scol: Union[Tuple, spark.Column]) -> DataType: + """ Return DataType for the given column label. """ + if isinstance(label_or_scol, spark.Column): + scol = label_or_scol + else: + scol = self.spark_column_for(label_or_scol) + return self.spark_frame.select(scol).schema[0].dataType + + def spark_column_nullable_for(self, label_or_scol: Union[Tuple, spark.Column]) -> bool: + """ Return nullability for the given column label. """ + if isinstance(label_or_scol, spark.Column): + scol = label_or_scol + else: + scol = self.spark_column_for(label_or_scol) + return self.spark_frame.select(scol).schema[0].nullable + + def dtype_for(self, label: Tuple) -> Dtype: + """ Return dtype for the given column label. """ + column_labels_to_dtype = dict(zip(self.column_labels, self.data_dtypes)) + if label in column_labels_to_dtype: + return column_labels_to_dtype[label] + else: + raise KeyError(name_like_string(label)) + + @property + def spark_frame(self) -> spark.DataFrame: + """ Return the managed Spark DataFrame. """ + return self._sdf + + @lazy_property + def data_spark_column_names(self) -> List[str]: + """ Return the managed column field names. """ + return self.spark_frame.select(self.data_spark_columns).columns + + @property + def data_spark_columns(self) -> List[spark.Column]: + """ Return Spark Columns for the managed data columns. """ + return self._data_spark_columns + + @property + def index_spark_column_names(self) -> List[str]: + """ Return the managed index field names. """ + return self.spark_frame.select(self.index_spark_columns).columns + + @property + def index_spark_columns(self) -> List[spark.Column]: + """ Return Spark Columns for the managed index columns. """ + return self._index_spark_columns + + @lazy_property + def spark_column_names(self) -> List[str]: + """ Return all the field names including index field names. """ + return self.spark_frame.select(self.spark_columns).columns + + @lazy_property + def spark_columns(self) -> List[spark.Column]: + """ Return Spark Columns for the managed columns including index columns. """ + index_spark_columns = self.index_spark_columns + return index_spark_columns + [ + spark_column + for spark_column in self.data_spark_columns + if all(not spark_column._jc.equals(scol._jc) for scol in index_spark_columns) + ] + + @property + def index_names(self) -> List[Optional[Tuple]]: + """ Return the managed index names. """ + return self._index_names + + @lazy_property + def index_level(self) -> int: + """ Return the level of the index. """ + return len(self._index_names) + + @property + def column_labels(self) -> List[Tuple]: + """ Return the managed column index. """ + return self._column_labels + + @lazy_property + def column_labels_level(self) -> int: + """ Return the level of the column index. """ + return len(self._column_label_names) + + @property + def column_label_names(self) -> List[Optional[Tuple]]: + """ Return names of the index levels. """ + return self._column_label_names + + @property + def index_dtypes(self) -> List[Dtype]: + """ Return dtypes for the managed index columns. """ + return self._index_dtypes + + @property + def data_dtypes(self) -> List[Dtype]: + """ Return dtypes for the managed columns. """ + return self._data_dtypes + + @lazy_property + def to_internal_spark_frame(self) -> spark.DataFrame: + """ + Return as Spark DataFrame. This contains index columns as well + and should be only used for internal purposes. + """ + index_spark_columns = self.index_spark_columns + data_columns = [] + for spark_column in self.data_spark_columns: + if all(not spark_column._jc.equals(scol._jc) for scol in index_spark_columns): + data_columns.append(spark_column) + return self.spark_frame.select(index_spark_columns + data_columns) + + @lazy_property + def to_pandas_frame(self) -> pd.DataFrame: + """ Return as pandas DataFrame. """ + sdf = self.to_internal_spark_frame + pdf = sdf.toPandas() + if len(pdf) == 0 and len(sdf.schema) > 0: + pdf = pdf.astype( + {field.name: spark_type_to_pandas_dtype(field.dataType) for field in sdf.schema} + ) + elif LooseVersion(pyspark.__version__) < LooseVersion("3.0"): + for field in sdf.schema: + if field.nullable and pdf[field.name].isnull().all(): + if isinstance(field.dataType, BooleanType): + pdf[field.name] = pdf[field.name].astype(np.object) + elif isinstance(field.dataType, IntegralType): + pdf[field.name] = pdf[field.name].astype(np.float64) + else: + pdf[field.name] = pdf[field.name].astype( + spark_type_to_pandas_dtype(field.dataType) + ) + + return InternalFrame.restore_index(pdf, **self.arguments_for_restore_index) + + @lazy_property + def arguments_for_restore_index(self) -> Dict: + """ Create arguments for `restore_index`. """ + column_names = [] + ext_dtypes = { + col: dtype + for col, dtype in zip(self.index_spark_column_names, self.index_dtypes) + if isinstance(dtype, extension_dtypes) + } + categorical_dtypes = { + col: dtype + for col, dtype in zip(self.index_spark_column_names, self.index_dtypes) + if isinstance(dtype, CategoricalDtype) + } + for spark_column, column_name, dtype in zip( + self.data_spark_columns, self.data_spark_column_names, self.data_dtypes + ): + for index_spark_column_name, index_spark_column in zip( + self.index_spark_column_names, self.index_spark_columns + ): + if spark_column._jc.equals(index_spark_column._jc): + column_names.append(index_spark_column_name) + break + else: + column_names.append(column_name) + if isinstance(dtype, extension_dtypes): + ext_dtypes[column_name] = dtype + elif isinstance(dtype, CategoricalDtype): + categorical_dtypes[column_name] = dtype + + return dict( + index_columns=self.index_spark_column_names, + index_names=self.index_names, + data_columns=column_names, + column_labels=self.column_labels, + column_label_names=self.column_label_names, + ext_dtypes=ext_dtypes, + categorical_dtypes=categorical_dtypes, + ) + + @staticmethod + def restore_index( + pdf: pd.DataFrame, + *, + index_columns: List[str], + index_names: List[Tuple], + data_columns: List[str], + column_labels: List[Tuple], + column_label_names: List[Tuple], + ext_dtypes: Dict[str, Dtype] = None, + categorical_dtypes: Dict[str, CategoricalDtype] = None + ) -> pd.DataFrame: + """ + Restore pandas DataFrame indices using the metadata. + + :param pdf: the pandas DataFrame to be processed. + :param index_columns: the original column names for index columns. + :param index_names: the index names after restored. + :param data_columns: the original column names for data columns. + :param column_labels: the column labels after restored. + :param column_label_names: the column label names after restored. + :param ext_dtypes: the map from the original column names to extension data types. + :param categorical_dtypes: the map from the original column names to categorical types. + :return: the restored pandas DataFrame + + >>> pdf = pd.DataFrame({"index": [10, 20, 30], "a": ['a', 'b', 'c'], "b": [0, 2, 1]}) + >>> InternalFrame.restore_index( + ... pdf, + ... index_columns=["index"], + ... index_names=[("idx",)], + ... data_columns=["a", "b", "index"], + ... column_labels=[("x",), ("y",), ("z",)], + ... column_label_names=[("lv1",)], + ... ext_dtypes=None, + ... categorical_dtypes={"b": CategoricalDtype(categories=["i", "j", "k"])} + ... ) # doctest: +NORMALIZE_WHITESPACE + lv1 x y z + idx + 10 a i 10 + 20 b k 20 + 30 c j 30 + """ + if ext_dtypes is not None and len(ext_dtypes) > 0: + pdf = pdf.astype(ext_dtypes, copy=True) + + if categorical_dtypes is not None: + for col, dtype in categorical_dtypes.items(): + pdf[col] = pd.Categorical.from_codes( + pdf[col], categories=dtype.categories, ordered=dtype.ordered + ) + + append = False + for index_field in index_columns: + drop = index_field not in data_columns + pdf = pdf.set_index(index_field, drop=drop, append=append) + append = True + pdf = pdf[data_columns] + + pdf.index.names = [ + name if name is None or len(name) > 1 else name[0] for name in index_names + ] + + names = [name if name is None or len(name) > 1 else name[0] for name in column_label_names] + if len(column_label_names) > 1: + pdf.columns = pd.MultiIndex.from_tuples(column_labels, names=names) + else: + pdf.columns = pd.Index( + [None if label is None else label[0] for label in column_labels], name=names[0], + ) + + return pdf + + @lazy_property + def resolved_copy(self) -> "InternalFrame": + """ Copy the immutable InternalFrame with the updates resolved. """ + sdf = self.spark_frame.select(self.spark_columns + list(HIDDEN_COLUMNS)) + return self.copy( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in self.index_spark_column_names], + data_spark_columns=[scol_for(sdf, col) for col in self.data_spark_column_names], + ) + + def with_new_sdf( + self, + spark_frame: spark.DataFrame, + *, + index_dtypes: Optional[List[Dtype]] = None, + data_columns: Optional[List[str]] = None, + data_dtypes: Optional[List[Dtype]] = None + ) -> "InternalFrame": + """ Copy the immutable InternalFrame with the updates by the specified Spark DataFrame. + + :param spark_frame: the new Spark DataFrame + :param index_dtypes: the index dtypes. If None, the original dtyeps are used. + :param data_columns: the new column names. If None, the original one is used. + :param data_dtypes: the data dtypes. If None, the original dtyeps are used. + :return: the copied InternalFrame. + """ + if index_dtypes is None: + index_dtypes = self.index_dtypes + else: + assert len(index_dtypes) == len(self.index_dtypes), ( + len(index_dtypes), + len(self.index_dtypes), + ) + + if data_columns is None: + data_columns = self.data_spark_column_names + else: + assert len(data_columns) == len(self.column_labels), ( + len(data_columns), + len(self.column_labels), + ) + + if data_dtypes is None: + data_dtypes = self.data_dtypes + else: + assert len(data_dtypes) == len(self.column_labels), ( + len(data_dtypes), + len(self.column_labels), + ) + + sdf = spark_frame.drop(NATURAL_ORDER_COLUMN_NAME) + return self.copy( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in self.index_spark_column_names], + index_dtypes=index_dtypes, + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + data_dtypes=data_dtypes, + ) + + def with_new_columns( + self, + scols_or_ksers: List[Union[spark.Column, "Series"]], + *, + column_labels: Optional[List[Tuple]] = None, + data_dtypes: Optional[List[Dtype]] = None, + column_label_names: Optional[Union[List[Optional[Tuple]], _NoValueType]] = _NoValue, + keep_order: bool = True + ) -> "InternalFrame": + """ + Copy the immutable InternalFrame with the updates by the specified Spark Columns or Series. + + :param scols_or_ksers: the new Spark Columns or Series. + :param column_labels: the new column index. + If None, the column_labels of the corresponding `scols_or_ksers` is used if it is + Series; otherwise the original one is used. + :param data_dtypes: the new dtypes. + If None, the dtypes of the corresponding `scols_or_ksers` is used if it is Series; + otherwise the dtypes will be inferred from the corresponding `scols_or_ksers`. + :param column_label_names: the new names of the column index levels. + :return: the copied InternalFrame. + """ + from pyspark.pandas.series import Series + + if column_labels is None: + if all(isinstance(scol_or_kser, Series) for scol_or_kser in scols_or_ksers): + column_labels = [kser._column_label for kser in scols_or_ksers] + else: + assert len(scols_or_ksers) == len(self.column_labels), ( + len(scols_or_ksers), + len(self.column_labels), + ) + column_labels = [] + for scol_or_kser, label in zip(scols_or_ksers, self.column_labels): + if isinstance(scol_or_kser, Series): + column_labels.append(scol_or_kser._column_label) + else: + column_labels.append(label) + else: + assert len(scols_or_ksers) == len(column_labels), ( + len(scols_or_ksers), + len(column_labels), + ) + + data_spark_columns = [] + for scol_or_kser in scols_or_ksers: + if isinstance(scol_or_kser, Series): + scol = scol_or_kser.spark.column + else: + scol = scol_or_kser + data_spark_columns.append(scol) + + if data_dtypes is None: + data_dtypes = [] + for scol_or_kser in scols_or_ksers: + if isinstance(scol_or_kser, Series): + data_dtypes.append(scol_or_kser.dtype) + else: + data_dtypes.append(None) + else: + assert len(scols_or_ksers) == len(data_dtypes), ( + len(scols_or_ksers), + len(data_dtypes), + ) + + sdf = self.spark_frame + if not keep_order: + sdf = self.spark_frame.select(self.index_spark_columns + data_spark_columns) + index_spark_columns = [scol_for(sdf, col) for col in self.index_spark_column_names] + data_spark_columns = [ + scol_for(sdf, col) for col in self.spark_frame.select(data_spark_columns).columns + ] + else: + index_spark_columns = self.index_spark_columns + + if column_label_names is _NoValue: + column_label_names = self._column_label_names + + return self.copy( + spark_frame=sdf, + index_spark_columns=index_spark_columns, + column_labels=column_labels, + data_spark_columns=data_spark_columns, + data_dtypes=data_dtypes, + column_label_names=column_label_names, + ) + + def with_filter(self, pred: Union[spark.Column, "Series"]) -> "InternalFrame": + """ Copy the immutable InternalFrame with the updates by the predicate. + + :param pred: the predicate to filter. + :return: the copied InternalFrame. + """ + from pyspark.pandas.series import Series + + if isinstance(pred, Series): + assert isinstance(pred.spark.data_type, BooleanType), pred.spark.data_type + pred = pred.spark.column + else: + spark_type = self.spark_frame.select(pred).schema[0].dataType + assert isinstance(spark_type, BooleanType), spark_type + + return self.with_new_sdf(self.spark_frame.filter(pred).select(self.spark_columns)) + + def with_new_spark_column( + self, + column_label: Tuple, + scol: spark.Column, + *, + dtype: Optional[Dtype] = None, + keep_order: bool = True + ) -> "InternalFrame": + """ + Copy the immutable InternalFrame with the updates by the specified Spark Column. + + :param column_label: the column label to be updated. + :param scol: the new Spark Column + :param dtype: the new dtype. + If not specified, the dtypes will be inferred from the spark Column. + :return: the copied InternalFrame. + """ + assert column_label in self.column_labels, column_label + + idx = self.column_labels.index(column_label) + data_spark_columns = self.data_spark_columns.copy() + data_spark_columns[idx] = scol + data_dtypes = self.data_dtypes.copy() + data_dtypes[idx] = dtype + return self.with_new_columns( + data_spark_columns, data_dtypes=data_dtypes, keep_order=keep_order + ) + + def select_column(self, column_label: Tuple) -> "InternalFrame": + """ + Copy the immutable InternalFrame with the specified column. + + :param column_label: the column label to use. + :return: the copied InternalFrame. + """ + assert column_label in self.column_labels, column_label + + return self.copy( + column_labels=[column_label], + data_spark_columns=[self.spark_column_for(column_label)], + data_dtypes=[self.dtype_for(column_label)], + column_label_names=None, + ) + + def copy( + self, + *, + spark_frame: Union[spark.DataFrame, _NoValueType] = _NoValue, + index_spark_columns: Union[List[spark.Column], _NoValueType] = _NoValue, + index_names: Union[List[Optional[Tuple]], _NoValueType] = _NoValue, + index_dtypes: Optional[Union[List[Dtype], _NoValueType]] = _NoValue, + column_labels: Optional[Union[List[Tuple], _NoValueType]] = _NoValue, + data_spark_columns: Optional[Union[List[spark.Column], _NoValueType]] = _NoValue, + data_dtypes: Optional[Union[List[Dtype], _NoValueType]] = _NoValue, + column_label_names: Optional[Union[List[Optional[Tuple]], _NoValueType]] = _NoValue + ) -> "InternalFrame": + """ Copy the immutable InternalFrame. + + :param spark_frame: the new Spark DataFrame. If not specified, the original one is used. + :param index_spark_columns: the list of Spark Column. + If not specified, the original ones are used. + :param index_names: the index names. If not specified, the original ones are used. + :param index_dtypes: the index dtypes. If not specified, the original dtyeps are used. + :param column_labels: the new column labels. If not specified, the original ones are used. + :param data_spark_columns: the new Spark Columns. + If not specified, the original ones are used. + :param data_dtypes: the data dtypes. If not specified, the original dtyeps are used. + :param column_label_names: the new names of the column index levels. + If not specified, the original ones are used. + :return: the copied immutable InternalFrame. + """ + if spark_frame is _NoValue: + spark_frame = self.spark_frame + if index_spark_columns is _NoValue: + index_spark_columns = self.index_spark_columns + if index_names is _NoValue: + index_names = self.index_names + if index_dtypes is _NoValue: + index_dtypes = self.index_dtypes + if column_labels is _NoValue: + column_labels = self.column_labels + if data_spark_columns is _NoValue: + data_spark_columns = self.data_spark_columns + if data_dtypes is _NoValue: + data_dtypes = self.data_dtypes + if column_label_names is _NoValue: + column_label_names = self.column_label_names + return InternalFrame( + spark_frame=spark_frame, + index_spark_columns=index_spark_columns, + index_names=index_names, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=data_spark_columns, + data_dtypes=data_dtypes, + column_label_names=column_label_names, + ) + + @staticmethod + def from_pandas(pdf: pd.DataFrame) -> "InternalFrame": + """ Create an immutable DataFrame from pandas DataFrame. + + :param pdf: :class:`pd.DataFrame` + :return: the created immutable DataFrame + """ + + index_names = [ + name if name is None or isinstance(name, tuple) else (name,) for name in pdf.index.names + ] + + columns = pdf.columns + if isinstance(columns, pd.MultiIndex): + column_labels = columns.tolist() + else: + column_labels = [(col,) for col in columns] + column_label_names = [ + name if name is None or isinstance(name, tuple) else (name,) for name in columns.names + ] + + ( + pdf, + index_columns, + index_dtypes, + data_columns, + data_dtypes, + ) = InternalFrame.prepare_pandas_frame(pdf) + + schema = StructType( + [ + StructField( + name, infer_pd_series_spark_type(col, dtype), nullable=bool(col.isnull().any()), + ) + for (name, col), dtype in zip(pdf.iteritems(), index_dtypes + data_dtypes) + ] + ) + + sdf = default_session().createDataFrame(pdf, schema=schema) + return InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_columns], + index_names=index_names, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + data_dtypes=data_dtypes, + column_label_names=column_label_names, + ) + + @staticmethod + def prepare_pandas_frame( + pdf: pd.DataFrame, *, retain_index: bool = True + ) -> Tuple[pd.DataFrame, List[str], List[Dtype], List[str], List[Dtype]]: + """ + Prepare pandas DataFrame for creating Spark DataFrame. + + :param pdf: the pandas DataFrame to be prepared. + :param retain_index: whether the indices should be retained. + :return: the tuple of + - the prepared pandas dataFrame + - index column names for Spark DataFrame + - index dtypes of the given pandas DataFrame + - data column names for Spark DataFrame + - data dtypes of the given pandas DataFrame + + >>> pdf = pd.DataFrame( + ... {("x", "a"): ['a', 'b', 'c'], + ... ("y", "b"): pd.Categorical(["i", "k", "j"], categories=["i", "j", "k"])}, + ... index=[10, 20, 30]) + >>> prepared, index_columns, index_dtypes, data_columns, data_dtypes = ( + ... InternalFrame.prepare_pandas_frame(pdf)) + >>> prepared + __index_level_0__ (x, a) (y, b) + 0 10 a 0 + 1 20 b 2 + 2 30 c 1 + >>> index_columns + ['__index_level_0__'] + >>> index_dtypes + [dtype('int64')] + >>> data_columns + ['(x, a)', '(y, b)'] + >>> data_dtypes + [dtype('O'), CategoricalDtype(categories=['i', 'j', 'k'], ordered=False)] + """ + pdf = pdf.copy() + + data_columns = [name_like_string(col) for col in pdf.columns] + pdf.columns = data_columns + + if retain_index: + index_nlevels = pdf.index.nlevels + index_columns = [SPARK_INDEX_NAME_FORMAT(i) for i in range(index_nlevels)] + pdf.index.names = index_columns + reset_index = pdf.reset_index() + else: + index_nlevels = 0 + index_columns = [] + reset_index = pdf + + index_dtypes = list(reset_index.dtypes)[:index_nlevels] + data_dtypes = list(reset_index.dtypes)[index_nlevels:] + + for name, col in reset_index.iteritems(): + dt = col.dtype + if is_datetime64_dtype(dt) or is_datetime64tz_dtype(dt): + continue + elif isinstance(dt, CategoricalDtype): + col = col.cat.codes + reset_index[name] = col.replace({np.nan: None}) + + return reset_index, index_columns, index_dtypes, data_columns, data_dtypes diff --git a/python/pyspark/pandas/missing/__init__.py b/python/pyspark/pandas/missing/__init__.py new file mode 100644 index 0000000000..bfe1597064 --- /dev/null +++ b/python/pyspark/pandas/missing/__init__.py @@ -0,0 +1,48 @@ +# +# 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. +# + +from pyspark.pandas.exceptions import PandasNotImplementedError + + +def unsupported_function(class_name, method_name, deprecated=False, reason=""): + def unsupported_function(*args, **kwargs): + raise PandasNotImplementedError( + class_name=class_name, method_name=method_name, reason=reason + ) + + def deprecated_function(*args, **kwargs): + raise PandasNotImplementedError( + class_name=class_name, method_name=method_name, deprecated=deprecated, reason=reason + ) + + return deprecated_function if deprecated else unsupported_function + + +def unsupported_property(class_name, property_name, deprecated=False, reason=""): + @property + def unsupported_property(self): + raise PandasNotImplementedError( + class_name=class_name, property_name=property_name, reason=reason + ) + + @property + def deprecated_property(self): + raise PandasNotImplementedError( + class_name=class_name, property_name=property_name, deprecated=deprecated, reason=reason + ) + + return deprecated_property if deprecated else unsupported_property diff --git a/python/pyspark/pandas/missing/common.py b/python/pyspark/pandas/missing/common.py new file mode 100644 index 0000000000..8cf39b0ca1 --- /dev/null +++ b/python/pyspark/pandas/missing/common.py @@ -0,0 +1,59 @@ +# +# 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. +# + + +memory_usage = lambda f: f( + "memory_usage", + reason="Unlike pandas, most DataFrames are not materialized in memory in Spark " + "(and Koalas), and as a result memory_usage() does not do what you intend it " + "to do. Use Spark's web UI to monitor disk and memory usage of your application.", +) + +array = lambda f: f( + "array", reason="If you want to collect your data as an NumPy array, use 'to_numpy()' instead." +) + +to_pickle = lambda f: f( + "to_pickle", + reason="For storage, we encourage you to use Delta or Parquet, instead of Python pickle " + "format.", +) + +to_xarray = lambda f: f( + "to_xarray", + reason="If you want to collect your data as an NumPy array, use 'to_numpy()' instead.", +) + +to_list = lambda f: f( + "to_list", + reason="If you want to collect your data as an NumPy array, use 'to_numpy()' instead.", +) + +tolist = lambda f: f( + "tolist", reason="If you want to collect your data as an NumPy array, use 'to_numpy()' instead." +) + +__iter__ = lambda f: f( + "__iter__", + reason="If you want to collect your data as an NumPy array, use 'to_numpy()' instead.", +) + +duplicated = lambda f: f( + "duplicated", + reason="'duplicated' API returns np.ndarray and the data size is too large." + "You can just use DataFrame.deduplicated instead", +) diff --git a/python/pyspark/pandas/missing/frame.py b/python/pyspark/pandas/missing/frame.py new file mode 100644 index 0000000000..3082a2a362 --- /dev/null +++ b/python/pyspark/pandas/missing/frame.py @@ -0,0 +1,98 @@ +# +# 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. +# +from distutils.version import LooseVersion + +import pandas as pd + +from pyspark.pandas.missing import unsupported_function, unsupported_property, common + + +def _unsupported_function(method_name, deprecated=False, reason=""): + return unsupported_function( + class_name="pd.DataFrame", method_name=method_name, deprecated=deprecated, reason=reason + ) + + +def _unsupported_property(property_name, deprecated=False, reason=""): + return unsupported_property( + class_name="pd.DataFrame", property_name=property_name, deprecated=deprecated, reason=reason + ) + + +class _MissingPandasLikeDataFrame(object): + + # Functions + asfreq = _unsupported_function("asfreq") + asof = _unsupported_function("asof") + boxplot = _unsupported_function("boxplot") + combine = _unsupported_function("combine") + combine_first = _unsupported_function("combine_first") + compare = _unsupported_function("compare") + convert_dtypes = _unsupported_function("convert_dtypes") + corrwith = _unsupported_function("corrwith") + cov = _unsupported_function("cov") + ewm = _unsupported_function("ewm") + infer_objects = _unsupported_function("infer_objects") + interpolate = _unsupported_function("interpolate") + lookup = _unsupported_function("lookup") + mode = _unsupported_function("mode") + reorder_levels = _unsupported_function("reorder_levels") + resample = _unsupported_function("resample") + set_axis = _unsupported_function("set_axis") + slice_shift = _unsupported_function("slice_shift") + to_feather = _unsupported_function("to_feather") + to_gbq = _unsupported_function("to_gbq") + to_hdf = _unsupported_function("to_hdf") + to_period = _unsupported_function("to_period") + to_sql = _unsupported_function("to_sql") + to_stata = _unsupported_function("to_stata") + to_timestamp = _unsupported_function("to_timestamp") + tshift = _unsupported_function("tshift") + tz_convert = _unsupported_function("tz_convert") + tz_localize = _unsupported_function("tz_localize") + + # Deprecated functions + convert_objects = _unsupported_function("convert_objects", deprecated=True) + select = _unsupported_function("select", deprecated=True) + to_panel = _unsupported_function("to_panel", deprecated=True) + get_values = _unsupported_function("get_values", deprecated=True) + compound = _unsupported_function("compound", deprecated=True) + reindex_axis = _unsupported_function("reindex_axis", deprecated=True) + + # Functions we won't support. + to_pickle = common.to_pickle(_unsupported_function) + memory_usage = common.memory_usage(_unsupported_function) + to_xarray = common.to_xarray(_unsupported_function) + + if LooseVersion(pd.__version__) < LooseVersion("1.0"): + # Deprecated properties + blocks = _unsupported_property("blocks", deprecated=True) + ftypes = _unsupported_property("ftypes", deprecated=True) + is_copy = _unsupported_property("is_copy", deprecated=True) + ix = _unsupported_property("ix", deprecated=True) + + # Deprecated functions + as_blocks = _unsupported_function("as_blocks", deprecated=True) + as_matrix = _unsupported_function("as_matrix", deprecated=True) + clip_lower = _unsupported_function("clip_lower", deprecated=True) + clip_upper = _unsupported_function("clip_upper", deprecated=True) + get_ftype_counts = _unsupported_function("get_ftype_counts", deprecated=True) + get_value = _unsupported_function("get_value", deprecated=True) + set_value = _unsupported_function("set_value", deprecated=True) + to_dense = _unsupported_function("to_dense", deprecated=True) + to_sparse = _unsupported_function("to_sparse", deprecated=True) + to_msgpack = _unsupported_function("to_msgpack", deprecated=True) diff --git a/python/pyspark/pandas/missing/groupby.py b/python/pyspark/pandas/missing/groupby.py new file mode 100644 index 0000000000..9a1837492a --- /dev/null +++ b/python/pyspark/pandas/missing/groupby.py @@ -0,0 +1,103 @@ +# +# 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. +# + +from pyspark.pandas.missing import unsupported_function, unsupported_property + + +def _unsupported_function(method_name, deprecated=False, reason=""): + return unsupported_function( + class_name="pd.groupby.GroupBy", + method_name=method_name, + deprecated=deprecated, + reason=reason, + ) + + +def _unsupported_property(property_name, deprecated=False, reason=""): + return unsupported_property( + class_name="pd.groupby.GroupBy", + property_name=property_name, + deprecated=deprecated, + reason=reason, + ) + + +class MissingPandasLikeDataFrameGroupBy(object): + + # Properties + corr = _unsupported_property("corr") + corrwith = _unsupported_property("corrwith") + cov = _unsupported_property("cov") + dtypes = _unsupported_property("dtypes") + groups = _unsupported_property("groups") + hist = _unsupported_property("hist") + indices = _unsupported_property("indices") + mad = _unsupported_property("mad") + ngroups = _unsupported_property("ngroups") + plot = _unsupported_property("plot") + quantile = _unsupported_property("quantile") + skew = _unsupported_property("skew") + tshift = _unsupported_property("tshift") + + # Deprecated properties + take = _unsupported_property("take", deprecated=True) + + # Functions + boxplot = _unsupported_function("boxplot") + ngroup = _unsupported_function("ngroup") + nth = _unsupported_function("nth") + ohlc = _unsupported_function("ohlc") + pct_change = _unsupported_function("pct_change") + pipe = _unsupported_function("pipe") + prod = _unsupported_function("prod") + resample = _unsupported_function("resample") + sem = _unsupported_function("sem") + + +class MissingPandasLikeSeriesGroupBy(object): + + # Properties + corr = _unsupported_property("corr") + cov = _unsupported_property("cov") + dtype = _unsupported_property("dtype") + groups = _unsupported_property("groups") + hist = _unsupported_property("hist") + indices = _unsupported_property("indices") + is_monotonic_decreasing = _unsupported_property("is_monotonic_decreasing") + is_monotonic_increasing = _unsupported_property("is_monotonic_increasing") + mad = _unsupported_property("mad") + ngroups = _unsupported_property("ngroups") + plot = _unsupported_property("plot") + quantile = _unsupported_property("quantile") + skew = _unsupported_property("skew") + tshift = _unsupported_property("tshift") + + # Deprecated properties + take = _unsupported_property("take", deprecated=True) + + # Functions + agg = _unsupported_function("agg") + aggregate = _unsupported_function("aggregate") + describe = _unsupported_function("describe") + ngroup = _unsupported_function("ngroup") + nth = _unsupported_function("nth") + ohlc = _unsupported_function("ohlc") + pct_change = _unsupported_function("pct_change") + pipe = _unsupported_function("pipe") + prod = _unsupported_function("prod") + resample = _unsupported_function("resample") + sem = _unsupported_function("sem") diff --git a/python/pyspark/pandas/missing/indexes.py b/python/pyspark/pandas/missing/indexes.py new file mode 100644 index 0000000000..86f387e920 --- /dev/null +++ b/python/pyspark/pandas/missing/indexes.py @@ -0,0 +1,218 @@ +# +# 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. +# +from distutils.version import LooseVersion + +import pandas as pd + +from pyspark.pandas.missing import unsupported_function, unsupported_property, common + + +def _unsupported_function(method_name, deprecated=False, reason="", cls="Index"): + return unsupported_function( + class_name="pd.{}".format(cls), + method_name=method_name, + deprecated=deprecated, + reason=reason, + ) + + +def _unsupported_property(property_name, deprecated=False, reason="", cls="Index"): + return unsupported_property( + class_name="pd.{}".format(cls), + property_name=property_name, + deprecated=deprecated, + reason=reason, + ) + + +class MissingPandasLikeIndex(object): + + # Properties + nbytes = _unsupported_property("nbytes") + + # Functions + argsort = _unsupported_function("argsort") + asof_locs = _unsupported_function("asof_locs") + format = _unsupported_function("format") + get_indexer = _unsupported_function("get_indexer") + get_indexer_for = _unsupported_function("get_indexer_for") + get_indexer_non_unique = _unsupported_function("get_indexer_non_unique") + get_loc = _unsupported_function("get_loc") + get_slice_bound = _unsupported_function("get_slice_bound") + get_value = _unsupported_function("get_value") + groupby = _unsupported_function("groupby") + is_ = _unsupported_function("is_") + is_lexsorted_for_tuple = _unsupported_function("is_lexsorted_for_tuple") + join = _unsupported_function("join") + map = _unsupported_function("map") + putmask = _unsupported_function("putmask") + ravel = _unsupported_function("ravel") + reindex = _unsupported_function("reindex") + searchsorted = _unsupported_function("searchsorted") + slice_indexer = _unsupported_function("slice_indexer") + slice_locs = _unsupported_function("slice_locs") + sortlevel = _unsupported_function("sortlevel") + to_flat_index = _unsupported_function("to_flat_index") + to_native_types = _unsupported_function("to_native_types") + where = _unsupported_function("where") + + # Deprecated functions + is_mixed = _unsupported_function("is_mixed") + get_values = _unsupported_function("get_values", deprecated=True) + set_value = _unsupported_function("set_value") + + # Properties we won't support. + array = common.array(_unsupported_property) + duplicated = common.duplicated(_unsupported_property) + + # Functions we won't support. + memory_usage = common.memory_usage(_unsupported_function) + __iter__ = common.__iter__(_unsupported_function) + + if LooseVersion(pd.__version__) < LooseVersion("1.0"): + # Deprecated properties + strides = _unsupported_property("strides", deprecated=True) + data = _unsupported_property("data", deprecated=True) + itemsize = _unsupported_property("itemsize", deprecated=True) + base = _unsupported_property("base", deprecated=True) + flags = _unsupported_property("flags", deprecated=True) + + # Deprecated functions + get_duplicates = _unsupported_function("get_duplicates", deprecated=True) + summary = _unsupported_function("summary", deprecated=True) + contains = _unsupported_function("contains", deprecated=True) + + +class MissingPandasLikeDatetimeIndex(MissingPandasLikeIndex): + + # Properties + nanosecond = _unsupported_property("nanosecond", cls="DatetimeIndex") + date = _unsupported_property("date", cls="DatetimeIndex") + time = _unsupported_property("time", cls="DatetimeIndex") + timetz = _unsupported_property("timetz", cls="DatetimeIndex") + tz = _unsupported_property("tz", cls="DatetimeIndex") + freq = _unsupported_property("freq", cls="DatetimeIndex") + freqstr = _unsupported_property("freqstr", cls="DatetimeIndex") + inferred_freq = _unsupported_property("inferred_freq", cls="DatetimeIndex") + + # Functions + snap = _unsupported_function("snap", cls="DatetimeIndex") + tz_convert = _unsupported_function("tz_convert", cls="DatetimeIndex") + tz_localize = _unsupported_function("tz_localize", cls="DatetimeIndex") + to_period = _unsupported_function("to_period", cls="DatetimeIndex") + to_perioddelta = _unsupported_function("to_perioddelta", cls="DatetimeIndex") + to_pydatetime = _unsupported_function("to_pydatetime", cls="DatetimeIndex") + mean = _unsupported_function("mean", cls="DatetimeIndex") + std = _unsupported_function("std", cls="DatetimeIndex") + + +class MissingPandasLikeCategoricalIndex(MissingPandasLikeIndex): + + # Functions + rename_categories = _unsupported_function("rename_categories", cls="CategoricalIndex") + reorder_categories = _unsupported_function("reorder_categories", cls="CategoricalIndex") + add_categories = _unsupported_function("add_categories", cls="CategoricalIndex") + remove_categories = _unsupported_function("remove_categories", cls="CategoricalIndex") + remove_unused_categories = _unsupported_function( + "remove_unused_categories", cls="CategoricalIndex" + ) + set_categories = _unsupported_function("set_categories", cls="CategoricalIndex") + as_ordered = _unsupported_function("as_ordered", cls="CategoricalIndex") + as_unordered = _unsupported_function("as_unordered", cls="CategoricalIndex") + map = _unsupported_function("map", cls="CategoricalIndex") + + +class MissingPandasLikeMultiIndex(object): + + # Deprecated properties + strides = _unsupported_property("strides", deprecated=True) + data = _unsupported_property("data", deprecated=True) + itemsize = _unsupported_property("itemsize", deprecated=True) + + # Functions + argsort = _unsupported_function("argsort") + asof_locs = _unsupported_function("asof_locs") + equal_levels = _unsupported_function("equal_levels") + factorize = _unsupported_function("factorize") + format = _unsupported_function("format") + get_indexer = _unsupported_function("get_indexer") + get_indexer_for = _unsupported_function("get_indexer_for") + get_indexer_non_unique = _unsupported_function("get_indexer_non_unique") + get_loc = _unsupported_function("get_loc") + get_loc_level = _unsupported_function("get_loc_level") + get_locs = _unsupported_function("get_locs") + get_slice_bound = _unsupported_function("get_slice_bound") + get_value = _unsupported_function("get_value") + groupby = _unsupported_function("groupby") + is_ = _unsupported_function("is_") + is_lexsorted = _unsupported_function("is_lexsorted") + is_lexsorted_for_tuple = _unsupported_function("is_lexsorted_for_tuple") + join = _unsupported_function("join") + map = _unsupported_function("map") + putmask = _unsupported_function("putmask") + ravel = _unsupported_function("ravel") + reindex = _unsupported_function("reindex") + remove_unused_levels = _unsupported_function("remove_unused_levels") + reorder_levels = _unsupported_function("reorder_levels") + searchsorted = _unsupported_function("searchsorted") + set_codes = _unsupported_function("set_codes") + set_levels = _unsupported_function("set_levels") + slice_indexer = _unsupported_function("slice_indexer") + slice_locs = _unsupported_function("slice_locs") + sortlevel = _unsupported_function("sortlevel") + to_flat_index = _unsupported_function("to_flat_index") + to_native_types = _unsupported_function("to_native_types") + truncate = _unsupported_function("truncate") + where = _unsupported_function("where") + + # Deprecated functions + is_mixed = _unsupported_function("is_mixed") + get_duplicates = _unsupported_function("get_duplicates", deprecated=True) + get_values = _unsupported_function("get_values", deprecated=True) + set_value = _unsupported_function("set_value", deprecated=True) + + # Functions we won't support. + array = common.array(_unsupported_property) + duplicated = common.duplicated(_unsupported_property) + codes = _unsupported_property( + "codes", + reason="'codes' requires to collect all data into the driver which is against the " + "design principle of Koalas. Alternatively, you could call 'to_pandas()' and" + " use 'codes' property in pandas.", + ) + levels = _unsupported_property( + "levels", + reason="'levels' requires to collect all data into the driver which is against the " + "design principle of Koalas. Alternatively, you could call 'to_pandas()' and" + " use 'levels' property in pandas.", + ) + __iter__ = common.__iter__(_unsupported_function) + + # Properties we won't support. + memory_usage = common.memory_usage(_unsupported_function) + + if LooseVersion(pd.__version__) < LooseVersion("1.0"): + # Deprecated properties + base = _unsupported_property("base", deprecated=True) + labels = _unsupported_property("labels", deprecated=True) + flags = _unsupported_property("flags", deprecated=True) + + # Deprecated functions + set_labels = _unsupported_function("set_labels") + summary = _unsupported_function("summary", deprecated=True) + to_hierarchical = _unsupported_function("to_hierarchical", deprecated=True) + contains = _unsupported_function("contains", deprecated=True) diff --git a/python/pyspark/pandas/missing/series.py b/python/pyspark/pandas/missing/series.py new file mode 100644 index 0000000000..ef3f38b227 --- /dev/null +++ b/python/pyspark/pandas/missing/series.py @@ -0,0 +1,125 @@ +# +# 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. +# +from distutils.version import LooseVersion + +import pandas as pd + +from pyspark.pandas.missing import unsupported_function, unsupported_property, common + + +def _unsupported_function(method_name, deprecated=False, reason=""): + return unsupported_function( + class_name="pd.Series", method_name=method_name, deprecated=deprecated, reason=reason + ) + + +def _unsupported_property(property_name, deprecated=False, reason=""): + return unsupported_property( + class_name="pd.Series", property_name=property_name, deprecated=deprecated, reason=reason + ) + + +class MissingPandasLikeSeries(object): + + # Functions + asfreq = _unsupported_function("asfreq") + autocorr = _unsupported_function("autocorr") + combine = _unsupported_function("combine") + convert_dtypes = _unsupported_function("convert_dtypes") + cov = _unsupported_function("cov") + ewm = _unsupported_function("ewm") + infer_objects = _unsupported_function("infer_objects") + interpolate = _unsupported_function("interpolate") + reorder_levels = _unsupported_function("reorder_levels") + resample = _unsupported_function("resample") + searchsorted = _unsupported_function("searchsorted") + set_axis = _unsupported_function("set_axis") + slice_shift = _unsupported_function("slice_shift") + to_hdf = _unsupported_function("to_hdf") + to_period = _unsupported_function("to_period") + to_sql = _unsupported_function("to_sql") + to_timestamp = _unsupported_function("to_timestamp") + tshift = _unsupported_function("tshift") + tz_convert = _unsupported_function("tz_convert") + tz_localize = _unsupported_function("tz_localize") + view = _unsupported_function("view") + + # Deprecated functions + convert_objects = _unsupported_function("convert_objects", deprecated=True) + nonzero = _unsupported_function("nonzero", deprecated=True) + reindex_axis = _unsupported_function("reindex_axis", deprecated=True) + select = _unsupported_function("select", deprecated=True) + get_values = _unsupported_function("get_values", deprecated=True) + + # Properties we won't support. + array = common.array(_unsupported_property) + duplicated = common.duplicated(_unsupported_property) + nbytes = _unsupported_property( + "nbytes", + reason="'nbytes' requires to compute whole dataset. You can calculate manually it, " + "with its 'itemsize', by explicitly executing its count. Use Spark's web UI " + "to monitor disk and memory usage of your application in general.", + ) + + # Functions we won't support. + memory_usage = common.memory_usage(_unsupported_function) + to_pickle = common.to_pickle(_unsupported_function) + to_xarray = common.to_xarray(_unsupported_function) + __iter__ = common.__iter__(_unsupported_function) + ravel = _unsupported_function( + "ravel", + reason="If you want to collect your flattened underlying data as an NumPy array, " + "use 'to_numpy().ravel()' instead.", + ) + + if LooseVersion(pd.__version__) < LooseVersion("1.0"): + # Deprecated properties + blocks = _unsupported_property("blocks", deprecated=True) + ftypes = _unsupported_property("ftypes", deprecated=True) + ftype = _unsupported_property("ftype", deprecated=True) + is_copy = _unsupported_property("is_copy", deprecated=True) + ix = _unsupported_property("ix", deprecated=True) + asobject = _unsupported_property("asobject", deprecated=True) + strides = _unsupported_property("strides", deprecated=True) + imag = _unsupported_property("imag", deprecated=True) + itemsize = _unsupported_property("itemsize", deprecated=True) + data = _unsupported_property("data", deprecated=True) + base = _unsupported_property("base", deprecated=True) + flags = _unsupported_property("flags", deprecated=True) + + # Deprecated functions + as_blocks = _unsupported_function("as_blocks", deprecated=True) + as_matrix = _unsupported_function("as_matrix", deprecated=True) + clip_lower = _unsupported_function("clip_lower", deprecated=True) + clip_upper = _unsupported_function("clip_upper", deprecated=True) + compress = _unsupported_function("compress", deprecated=True) + get_ftype_counts = _unsupported_function("get_ftype_counts", deprecated=True) + get_value = _unsupported_function("get_value", deprecated=True) + set_value = _unsupported_function("set_value", deprecated=True) + valid = _unsupported_function("valid", deprecated=True) + to_dense = _unsupported_function("to_dense", deprecated=True) + to_sparse = _unsupported_function("to_sparse", deprecated=True) + to_msgpack = _unsupported_function("to_msgpack", deprecated=True) + compound = _unsupported_function("compound", deprecated=True) + put = _unsupported_function("put", deprecated=True) + ptp = _unsupported_function("ptp", deprecated=True) + + # Functions we won't support. + real = _unsupported_property( + "real", + reason="If you want to collect your data as an NumPy array, use 'to_numpy()' instead.", + ) diff --git a/python/pyspark/pandas/missing/window.py b/python/pyspark/pandas/missing/window.py new file mode 100644 index 0000000000..a29887e9e1 --- /dev/null +++ b/python/pyspark/pandas/missing/window.py @@ -0,0 +1,126 @@ +# +# 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. +# + +from pyspark.pandas.missing import unsupported_function, unsupported_property + + +def _unsupported_function_expanding(method_name, deprecated=False, reason=""): + return unsupported_function( + class_name="pandas.core.window.Expanding", + method_name=method_name, + deprecated=deprecated, + reason=reason, + ) + + +def _unsupported_property_expanding(property_name, deprecated=False, reason=""): + return unsupported_property( + class_name="pandas.core.window.Expanding", + property_name=property_name, + deprecated=deprecated, + reason=reason, + ) + + +def _unsupported_function_rolling(method_name, deprecated=False, reason=""): + return unsupported_function( + class_name="pandas.core.window.Rolling", + method_name=method_name, + deprecated=deprecated, + reason=reason, + ) + + +def _unsupported_property_rolling(property_name, deprecated=False, reason=""): + return unsupported_property( + class_name="pandas.core.window.Rolling", + property_name=property_name, + deprecated=deprecated, + reason=reason, + ) + + +class MissingPandasLikeExpanding(object): + agg = _unsupported_function_expanding("agg") + aggregate = _unsupported_function_expanding("aggregate") + apply = _unsupported_function_expanding("apply") + corr = _unsupported_function_expanding("corr") + cov = _unsupported_function_expanding("cov") + kurt = _unsupported_function_expanding("kurt") + median = _unsupported_function_expanding("median") + quantile = _unsupported_function_expanding("quantile") + skew = _unsupported_function_expanding("skew") + validate = _unsupported_function_expanding("validate") + + exclusions = _unsupported_property_expanding("exclusions") + is_datetimelike = _unsupported_property_expanding("is_datetimelike") + is_freq_type = _unsupported_property_expanding("is_freq_type") + ndim = _unsupported_property_expanding("ndim") + + +class MissingPandasLikeRolling(object): + agg = _unsupported_function_rolling("agg") + aggregate = _unsupported_function_rolling("aggregate") + apply = _unsupported_function_rolling("apply") + corr = _unsupported_function_rolling("corr") + cov = _unsupported_function_rolling("cov") + kurt = _unsupported_function_rolling("kurt") + median = _unsupported_function_rolling("median") + quantile = _unsupported_function_rolling("quantile") + skew = _unsupported_function_rolling("skew") + validate = _unsupported_function_rolling("validate") + + exclusions = _unsupported_property_rolling("exclusions") + is_datetimelike = _unsupported_property_rolling("is_datetimelike") + is_freq_type = _unsupported_property_rolling("is_freq_type") + ndim = _unsupported_property_rolling("ndim") + + +class MissingPandasLikeExpandingGroupby(object): + agg = _unsupported_function_expanding("agg") + aggregate = _unsupported_function_expanding("aggregate") + apply = _unsupported_function_expanding("apply") + corr = _unsupported_function_expanding("corr") + cov = _unsupported_function_expanding("cov") + kurt = _unsupported_function_expanding("kurt") + median = _unsupported_function_expanding("median") + quantile = _unsupported_function_expanding("quantile") + skew = _unsupported_function_expanding("skew") + validate = _unsupported_function_expanding("validate") + + exclusions = _unsupported_property_expanding("exclusions") + is_datetimelike = _unsupported_property_expanding("is_datetimelike") + is_freq_type = _unsupported_property_expanding("is_freq_type") + ndim = _unsupported_property_expanding("ndim") + + +class MissingPandasLikeRollingGroupby(object): + agg = _unsupported_function_rolling("agg") + aggregate = _unsupported_function_rolling("aggregate") + apply = _unsupported_function_rolling("apply") + corr = _unsupported_function_rolling("corr") + cov = _unsupported_function_rolling("cov") + kurt = _unsupported_function_rolling("kurt") + median = _unsupported_function_rolling("median") + quantile = _unsupported_function_rolling("quantile") + skew = _unsupported_function_rolling("skew") + validate = _unsupported_function_rolling("validate") + + exclusions = _unsupported_property_rolling("exclusions") + is_datetimelike = _unsupported_property_rolling("is_datetimelike") + is_freq_type = _unsupported_property_rolling("is_freq_type") + ndim = _unsupported_property_rolling("ndim") diff --git a/python/pyspark/pandas/ml.py b/python/pyspark/pandas/ml.py new file mode 100644 index 0000000000..6ea2cf339d --- /dev/null +++ b/python/pyspark/pandas/ml.py @@ -0,0 +1,91 @@ +# +# 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. +# + +from typing import List, Tuple, TYPE_CHECKING + +import numpy as np +import pandas as pd +import pyspark + +from pyspark.ml.feature import VectorAssembler +from pyspark.ml.stat import Correlation + +from pyspark.pandas.utils import column_labels_level + +if TYPE_CHECKING: + import pyspark.pandas as pp # noqa: F401 (SPARK-34943) + + +CORRELATION_OUTPUT_COLUMN = "__correlation_output__" + + +def corr(kdf: "pp.DataFrame", method: str = "pearson") -> pd.DataFrame: + """ + The correlation matrix of all the numerical columns of this dataframe. + + Only accepts scalar numerical values for now. + + :param kdf: the Koalas dataframe. + :param method: {'pearson', 'spearman'} + * pearson : standard correlation coefficient + * spearman : Spearman rank correlation + :return: :class:`pandas.DataFrame` + + >>> pp.DataFrame({'A': [0, 1], 'B': [1, 0], 'C': ['x', 'y']}).corr() + A B + A 1.0 -1.0 + B -1.0 1.0 + """ + assert method in ("pearson", "spearman") + ndf, column_labels = to_numeric_df(kdf) + corr = Correlation.corr(ndf, CORRELATION_OUTPUT_COLUMN, method) + pcorr = corr.toPandas() + arr = pcorr.iloc[0, 0].toArray() + if column_labels_level(column_labels) > 1: + idx = pd.MultiIndex.from_tuples(column_labels) + else: + idx = pd.Index([label[0] for label in column_labels]) + return pd.DataFrame(arr, columns=idx, index=idx) + + +def to_numeric_df(kdf: "pp.DataFrame") -> Tuple[pyspark.sql.DataFrame, List[Tuple]]: + """ + Takes a dataframe and turns it into a dataframe containing a single numerical + vector of doubles. This dataframe has a single field called '_1'. + + TODO: index is not preserved currently + :param kdf: the Koalas dataframe. + :return: a pair of dataframe, list of strings (the name of the columns + that were converted to numerical types) + + >>> to_numeric_df(pp.DataFrame({'A': [0, 1], 'B': [1, 0], 'C': ['x', 'y']})) + (DataFrame[__correlation_output__: vector], [('A',), ('B',)]) + """ + # TODO, it should be more robust. + accepted_types = { + np.dtype(dt) + for dt in [np.int8, np.int16, np.int32, np.int64, np.float32, np.float64, np.bool_] + } + numeric_column_labels = [ + label for label in kdf._internal.column_labels if kdf[label].dtype in accepted_types + ] + numeric_df = kdf._internal.spark_frame.select( + *[kdf._internal.spark_column_for(idx) for idx in numeric_column_labels] + ) + va = VectorAssembler(inputCols=numeric_df.columns, outputCol=CORRELATION_OUTPUT_COLUMN) + v = va.transform(numeric_df).select(CORRELATION_OUTPUT_COLUMN) + return v, numeric_column_labels diff --git a/python/pyspark/pandas/mlflow.py b/python/pyspark/pandas/mlflow.py new file mode 100644 index 0000000000..15ff066d20 --- /dev/null +++ b/python/pyspark/pandas/mlflow.py @@ -0,0 +1,192 @@ +# +# 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. +# + +""" +MLflow-related functions to load models and apply them to Koalas dataframes. +""" +from mlflow import pyfunc +from pyspark.sql.types import DataType +import pandas as pd +import numpy as np +from typing import Any + +from pyspark.pandas.utils import lazy_property, default_session +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.series import first_series +from pyspark.pandas.typedef import as_spark_type + +__all__ = ["PythonModelWrapper", "load_model"] + + +class PythonModelWrapper(object): + """ + A wrapper around MLflow's Python object model. + + This wrapper acts as a predictor on koalas + + """ + + def __init__(self, model_uri, return_type_hint): + self._model_uri = model_uri # type: str + self._return_type_hint = return_type_hint + + @lazy_property + def _return_type(self) -> DataType: + hint = self._return_type_hint + # The logic is simple for now, because it corresponds to the default + # case: continuous predictions + # TODO: do something smarter, for example when there is a sklearn.Classifier (it should + # return an integer or a categorical) + # We can do the same for pytorch/tensorflow/keras models by looking at the output types. + # However, this is probably better done in mlflow than here. + if hint == "infer" or not hint: + hint = np.float64 + return as_spark_type(hint) + + @lazy_property + def _model(self) -> Any: + """ + The return object has to follow the API of mlflow.pyfunc.PythonModel. + """ + return pyfunc.load_model(model_uri=self._model_uri) + + @lazy_property + def _model_udf(self): + spark = default_session() + return pyfunc.spark_udf(spark, model_uri=self._model_uri, result_type=self._return_type) + + def __str__(self): + return "PythonModelWrapper({})".format(str(self._model)) + + def __repr__(self): + return "PythonModelWrapper({})".format(repr(self._model)) + + def predict(self, data): + """ + Returns a prediction on the data. + + If the data is a koalas DataFrame, the return is a Koalas Series. + + If the data is a pandas Dataframe, the return is the expected output of the underlying + pyfunc object (typically a pandas Series or a numpy array). + """ + if isinstance(data, pd.DataFrame): + return self._model.predict(data) + if isinstance(data, DataFrame): + return_col = self._model_udf(*data._internal.data_spark_columns) + # TODO: the columns should be named according to the mlflow spec + # However, this is only possible with spark >= 3.0 + # s = F.struct(*data.columns) + # return_col = self._model_udf(s) + column_labels = [ + (col,) for col in data._internal.spark_frame.select(return_col).columns + ] + internal = data._internal.copy( + column_labels=column_labels, data_spark_columns=[return_col], data_dtypes=None + ) + return first_series(DataFrame(internal)) + + +def load_model(model_uri, predict_type="infer") -> PythonModelWrapper: + """ + Loads an MLflow model into an wrapper that can be used both for pandas and Koalas DataFrame. + + Parameters + ---------- + model_uri : str + URI pointing to the model. See MLflow documentation for more details. + predict_type : a python basic type, a numpy basic type, a Spark type or 'infer'. + This is the return type that is expected when calling the predict function of the model. + If 'infer' is specified, the wrapper will attempt to determine automatically the return type + based on the model type. + + Returns + ------- + PythonModelWrapper + A wrapper around MLflow PythonModel objects. This wrapper is expected to adhere to the + interface of mlflow.pyfunc.PythonModel. + + Examples + -------- + Here is a full example that creates a model with scikit-learn and saves the model with + MLflow. The model is then loaded as a predictor that can be applied on a Koalas + Dataframe. + + We first initialize our MLflow environment: + + >>> from mlflow.tracking import MlflowClient, set_tracking_uri + >>> import mlflow.sklearn + >>> from tempfile import mkdtemp + >>> d = mkdtemp("koalas_mlflow") + >>> set_tracking_uri("file:%s"%d) + >>> client = MlflowClient() + >>> exp = mlflow.create_experiment("my_experiment") + >>> mlflow.set_experiment("my_experiment") + + We aim at learning this numerical function using a simple linear regressor. + + >>> from sklearn.linear_model import LinearRegression + >>> train = pd.DataFrame({"x1": np.arange(8), "x2": np.arange(8)**2, + ... "y": np.log(2 + np.arange(8))}) + >>> train_x = train[["x1", "x2"]] + >>> train_y = train[["y"]] + >>> with mlflow.start_run(): + ... lr = LinearRegression() + ... lr.fit(train_x, train_y) + ... mlflow.sklearn.log_model(lr, "model") + LinearRegression(...) + + Now that our model is logged using MLflow, we load it back and apply it on a Koalas dataframe: + + >>> from pyspark.pandas.mlflow import load_model + >>> run_info = client.list_run_infos(exp)[-1] + >>> model = load_model("runs:/{run_id}/model".format(run_id=run_info.run_uuid)) + >>> prediction_df = pp.DataFrame({"x1": [2.0], "x2": [4.0]}) + >>> prediction_df["prediction"] = model.predict(prediction_df) + >>> prediction_df + x1 x2 prediction + 0 2.0 4.0 1.355551 + + The model also works on pandas DataFrames as expected: + + >>> model.predict(prediction_df[["x1", "x2"]].to_pandas()) + array([[1.35555142]]) + + Notes + ----- + Currently, the model prediction can only be merged back with the existing dataframe. + Other columns have to be manually joined. + For example, this code will not work: + + >>> df = pp.DataFrame({"x1": [2.0], "x2": [3.0], "z": [-1]}) + >>> features = df[["x1", "x2"]] + >>> y = model.predict(features) + >>> # Works: + >>> features["y"] = y # doctest: +SKIP + >>> # Will fail with a message about dataframes not aligned. + >>> df["y"] = y # doctest: +SKIP + + A current workaround is to use the .merge() function, using the feature values + as merging keys. + + >>> features['y'] = y + >>> everything = df.merge(features, on=['x1', 'x2']) + >>> everything + x1 x2 z y + 0 2.0 3.0 -1 1.376932 + """ + return PythonModelWrapper(model_uri, predict_type) diff --git a/python/pyspark/pandas/namespace.py b/python/pyspark/pandas/namespace.py new file mode 100644 index 0000000000..3a14d6a998 --- /dev/null +++ b/python/pyspark/pandas/namespace.py @@ -0,0 +1,2877 @@ +# +# 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. +# + +""" +Wrappers around spark that correspond to common pandas functions. +""" +from typing import Any, Optional, Union, List, Tuple, Sized, cast +from collections import OrderedDict +from collections.abc import Iterable +from distutils.version import LooseVersion +from functools import reduce +from io import BytesIO +import json + +import numpy as np +import pandas as pd +from pandas.api.types import is_datetime64_dtype, is_datetime64tz_dtype, is_list_like +import pyarrow as pa +import pyarrow.parquet as pq +import pyspark +from pyspark import sql as spark +from pyspark.sql import functions as F +from pyspark.sql.functions import pandas_udf, PandasUDFType +from pyspark.sql.types import ( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + BooleanType, + TimestampType, + DecimalType, + StringType, + DateType, + StructType, +) + +from pyspark import pandas as pp # noqa: F401 +from pyspark.pandas.base import IndexOpsMixin +from pyspark.pandas.utils import ( + align_diff_frames, + default_session, + is_name_like_tuple, + name_like_string, + same_anchor, + scol_for, + validate_axis, +) +from pyspark.pandas.frame import DataFrame, _reduce_spark_multi +from pyspark.pandas.internal import ( + InternalFrame, + DEFAULT_SERIES_NAME, + HIDDEN_COLUMNS, +) +from pyspark.pandas.series import Series, first_series +from pyspark.pandas.spark.utils import as_nullable_spark_type, force_decimal_precision_scale +from pyspark.pandas.indexes import Index, DatetimeIndex + + +__all__ = [ + "from_pandas", + "range", + "read_csv", + "read_delta", + "read_table", + "read_spark_io", + "read_parquet", + "read_clipboard", + "read_excel", + "read_html", + "to_datetime", + "date_range", + "get_dummies", + "concat", + "melt", + "isna", + "isnull", + "notna", + "notnull", + "read_sql_table", + "read_sql_query", + "read_sql", + "read_json", + "merge", + "to_numeric", + "broadcast", + "read_orc", +] + + +def from_pandas(pobj: Union[pd.DataFrame, pd.Series, pd.Index]) -> Union[Series, DataFrame, Index]: + """Create a Koalas DataFrame, Series or Index from a pandas DataFrame, Series or Index. + + This is similar to Spark's `SparkSession.createDataFrame()` with pandas DataFrame, + but this also works with pandas Series and picks the index. + + Parameters + ---------- + pobj : pandas.DataFrame or pandas.Series + pandas DataFrame or Series to read. + + Returns + ------- + Series or DataFrame + If a pandas Series is passed in, this function returns a Koalas Series. + If a pandas DataFrame is passed in, this function returns a Koalas DataFrame. + """ + if isinstance(pobj, pd.Series): + return Series(pobj) + elif isinstance(pobj, pd.DataFrame): + return DataFrame(pobj) + elif isinstance(pobj, pd.Index): + return DataFrame(pd.DataFrame(index=pobj)).index + else: + raise ValueError("Unknown data type: {}".format(type(pobj).__name__)) + + +_range = range # built-in range + + +def range( + start: int, end: Optional[int] = None, step: int = 1, num_partitions: Optional[int] = None +) -> DataFrame: + """ + Create a DataFrame with some range of numbers. + + The resulting DataFrame has a single int64 column named `id`, containing elements in a range + from ``start`` to ``end`` (exclusive) with step value ``step``. If only the first parameter + (i.e. start) is specified, we treat it as the end value with the start value being 0. + + This is similar to the range function in SparkSession and is used primarily for testing. + + Parameters + ---------- + start : int + the start value (inclusive) + end : int, optional + the end value (exclusive) + step : int, optional, default 1 + the incremental step + num_partitions : int, optional + the number of partitions of the DataFrame + + Returns + ------- + DataFrame + + Examples + -------- + When the first parameter is specified, we generate a range of values up till that number. + + >>> pp.range(5) + id + 0 0 + 1 1 + 2 2 + 3 3 + 4 4 + + When start, end, and step are specified: + + >>> pp.range(start = 100, end = 200, step = 20) + id + 0 100 + 1 120 + 2 140 + 3 160 + 4 180 + """ + sdf = default_session().range(start=start, end=end, step=step, numPartitions=num_partitions) + return DataFrame(sdf) + + +def read_csv( + path, + sep=",", + header="infer", + names=None, + index_col=None, + usecols=None, + squeeze=False, + mangle_dupe_cols=True, + dtype=None, + nrows=None, + parse_dates=False, + quotechar=None, + escapechar=None, + comment=None, + **options +) -> Union[DataFrame, Series]: + """Read CSV (comma-separated) file into DataFrame or Series. + + Parameters + ---------- + path : str + The path string storing the CSV file to be read. + sep : str, default ‘,’ + Delimiter to use. Must be a single character. + header : int, list of int, default ‘infer’ + Whether to to use as the column names, and the start of the data. + Default behavior is to infer the column names: if no names are passed + the behavior is identical to `header=0` and column names are inferred from + the first line of the file, if column names are passed explicitly then + the behavior is identical to `header=None`. Explicitly pass `header=0` to be + able to replace existing names + names : str or array-like, optional + List of column names to use. If file contains no header row, then you should + explicitly pass `header=None`. Duplicates in this list will cause an error to be issued. + If a string is given, it should be a DDL-formatted string in Spark SQL, which is + preferred to avoid schema inference for better performance. + index_col: str or list of str, optional, default: None + Index column of table in Spark. + usecols : list-like or callable, optional + Return a subset of the columns. If list-like, all elements must either be + positional (i.e. integer indices into the document columns) or strings that + correspond to column names provided either by the user in names or inferred + from the document header row(s). + If callable, the callable function will be evaluated against the column names, + returning names where the callable function evaluates to `True`. + squeeze : bool, default False + If the parsed data only contains one column then return a Series. + mangle_dupe_cols : bool, default True + Duplicate columns will be specified as 'X0', 'X1', ... 'XN', rather + than 'X' ... 'X'. Passing in False will cause data to be overwritten if + there are duplicate names in the columns. + Currently only `True` is allowed. + dtype : Type name or dict of column -> type, default None + Data type for data or columns. E.g. {‘a’: np.float64, ‘b’: np.int32} Use str or object + together with suitable na_values settings to preserve and not interpret dtype. + nrows : int, default None + Number of rows to read from the CSV file. + parse_dates : boolean or list of ints or names or list of lists or dict, default `False`. + Currently only `False` is allowed. + quotechar : str (length 1), optional + The character used to denote the start and end of a quoted item. Quoted items can include + the delimiter and it will be ignored. + escapechar : str (length 1), default None + One-character string used to escape delimiter + comment: str, optional + Indicates the line should not be parsed. + options : dict + All other options passed directly into Spark's data source. + + Returns + ------- + DataFrame or Series + + See Also + -------- + DataFrame.to_csv : Write DataFrame to a comma-separated values (csv) file. + + Examples + -------- + >>> pp.read_csv('data.csv') # doctest: +SKIP + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + if mangle_dupe_cols is not True: + raise ValueError("mangle_dupe_cols can only be `True`: %s" % mangle_dupe_cols) + if parse_dates is not False: + raise ValueError("parse_dates can only be `False`: %s" % parse_dates) + + if usecols is not None and not callable(usecols): + usecols = list(usecols) + + if usecols is None or callable(usecols) or len(usecols) > 0: + reader = default_session().read + reader.option("inferSchema", True) + reader.option("sep", sep) + + if header == "infer": + header = 0 if names is None else None + if header == 0: + reader.option("header", True) + elif header is None: + reader.option("header", False) + else: + raise ValueError("Unknown header argument {}".format(header)) + + if quotechar is not None: + reader.option("quote", quotechar) + if escapechar is not None: + reader.option("escape", escapechar) + + if comment is not None: + if not isinstance(comment, str) or len(comment) != 1: + raise ValueError("Only length-1 comment characters supported") + reader.option("comment", comment) + + reader.options(**options) + + if isinstance(names, str): + sdf = reader.schema(names).csv(path) + column_labels = OrderedDict((col, col) for col in sdf.columns) + else: + sdf = reader.csv(path) + if is_list_like(names): + names = list(names) + if len(set(names)) != len(names): + raise ValueError("Found non-unique column index") + if len(names) != len(sdf.columns): + raise ValueError( + "The number of names [%s] does not match the number " + "of columns [%d]. Try names by a Spark SQL DDL-formatted " + "string." % (len(sdf.schema), len(names)) + ) + column_labels = OrderedDict(zip(names, sdf.columns)) + elif header is None: + column_labels = OrderedDict(enumerate(sdf.columns)) + else: + column_labels = OrderedDict((col, col) for col in sdf.columns) + + if usecols is not None: + if callable(usecols): + column_labels = OrderedDict( + (label, col) for label, col in column_labels.items() if usecols(label) + ) + missing = [] + elif all(isinstance(col, int) for col in usecols): + new_column_labels = OrderedDict( + (label, col) + for i, (label, col) in enumerate(column_labels.items()) + if i in usecols + ) + missing = [ + col + for col in usecols + if col >= len(column_labels) + or list(column_labels)[col] not in new_column_labels + ] + column_labels = new_column_labels + elif all(isinstance(col, str) for col in usecols): + new_column_labels = OrderedDict( + (label, col) for label, col in column_labels.items() if label in usecols + ) + missing = [col for col in usecols if col not in new_column_labels] + column_labels = new_column_labels + else: + raise ValueError( + "'usecols' must either be list-like of all strings, " + "all unicode, all integers or a callable." + ) + if len(missing) > 0: + raise ValueError( + "Usecols do not match columns, columns expected but not " "found: %s" % missing + ) + + if len(column_labels) > 0: + sdf = sdf.select([scol_for(sdf, col) for col in column_labels.values()]) + else: + sdf = default_session().createDataFrame([], schema=StructType()) + else: + sdf = default_session().createDataFrame([], schema=StructType()) + column_labels = OrderedDict() + + if nrows is not None: + sdf = sdf.limit(nrows) + + if index_col is not None: + if isinstance(index_col, (str, int)): + index_col = [index_col] + for col in index_col: + if col not in column_labels: + raise KeyError(col) + index_spark_column_names = [column_labels[col] for col in index_col] + index_names = [(col,) for col in index_col] # type: List[Tuple] + column_labels = OrderedDict( + (label, col) for label, col in column_labels.items() if label not in index_col + ) + else: + index_spark_column_names = [] + index_names = [] + + kdf = DataFrame( + InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_spark_column_names], + index_names=index_names, + column_labels=[ + label if is_name_like_tuple(label) else (label,) for label in column_labels + ], + data_spark_columns=[scol_for(sdf, col) for col in column_labels.values()], + ) + ) # type: DataFrame + + if dtype is not None: + if isinstance(dtype, dict): + for col, tpe in dtype.items(): + kdf[col] = kdf[col].astype(tpe) + else: + for col in kdf.columns: + kdf[col] = kdf[col].astype(dtype) + + if squeeze and len(kdf.columns) == 1: + return first_series(kdf) + else: + return kdf + + +def read_json( + path: str, lines: bool = True, index_col: Optional[Union[str, List[str]]] = None, **options +) -> DataFrame: + """ + Convert a JSON string to DataFrame. + + Parameters + ---------- + path : string + File path + lines : bool, default True + Read the file as a json object per line. It should be always True for now. + index_col : str or list of str, optional, default: None + Index column of table in Spark. + options : dict + All other options passed directly into Spark's data source. + + Examples + -------- + >>> df = pp.DataFrame([['a', 'b'], ['c', 'd']], + ... columns=['col 1', 'col 2']) + + >>> df.to_json(path=r'%s/read_json/foo.json' % path, num_files=1) + >>> pp.read_json( + ... path=r'%s/read_json/foo.json' % path + ... ).sort_values(by="col 1") + col 1 col 2 + 0 a b + 1 c d + + >>> df.to_json(path=r'%s/read_json/foo.json' % path, num_files=1, lineSep='___') + >>> pp.read_json( + ... path=r'%s/read_json/foo.json' % path, lineSep='___' + ... ).sort_values(by="col 1") + col 1 col 2 + 0 a b + 1 c d + + You can preserve the index in the roundtrip as below. + + >>> df.to_json(path=r'%s/read_json/bar.json' % path, num_files=1, index_col="index") + >>> pp.read_json( + ... path=r'%s/read_json/bar.json' % path, index_col="index" + ... ).sort_values(by="col 1") # doctest: +NORMALIZE_WHITESPACE + col 1 col 2 + index + 0 a b + 1 c d + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + if not lines: + raise NotImplementedError("lines=False is not implemented yet.") + + return read_spark_io(path, format="json", index_col=index_col, **options) + + +def read_delta( + path: str, + version: Optional[str] = None, + timestamp: Optional[str] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options +) -> DataFrame: + """ + Read a Delta Lake table on some file system and return a DataFrame. + + If the Delta Lake table is already stored in the catalog (aka the metastore), use 'read_table'. + + Parameters + ---------- + path : string + Path to the Delta Lake table. + version : string, optional + Specifies the table version (based on Delta's internal transaction version) to read from, + using Delta's time travel feature. This sets Delta's 'versionAsOf' option. + timestamp : string, optional + Specifies the table version (based on timestamp) to read from, + using Delta's time travel feature. This must be a valid date or timestamp string in Spark, + and sets Delta's 'timestampAsOf' option. + index_col : str or list of str, optional, default: None + Index column of table in Spark. + options + Additional options that can be passed onto Delta. + + Returns + ------- + DataFrame + + See Also + -------- + DataFrame.to_delta + read_table + read_spark_io + read_parquet + + Examples + -------- + >>> pp.range(1).to_delta('%s/read_delta/foo' % path) + >>> pp.read_delta('%s/read_delta/foo' % path) + id + 0 0 + + >>> pp.range(10, 15, num_partitions=1).to_delta('%s/read_delta/foo' % path, mode='overwrite') + >>> pp.read_delta('%s/read_delta/foo' % path) + id + 0 10 + 1 11 + 2 12 + 3 13 + 4 14 + + >>> pp.read_delta('%s/read_delta/foo' % path, version=0) + id + 0 0 + + You can preserve the index in the roundtrip as below. + + >>> pp.range(10, 15, num_partitions=1).to_delta( + ... '%s/read_delta/bar' % path, index_col="index") + >>> pp.read_delta('%s/read_delta/bar' % path, index_col="index") + ... # doctest: +NORMALIZE_WHITESPACE + id + index + 0 10 + 1 11 + 2 12 + 3 13 + 4 14 + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + if version is not None: + options["versionAsOf"] = version + if timestamp is not None: + options["timestampAsOf"] = timestamp + return read_spark_io(path, format="delta", index_col=index_col, **options) + + +def read_table(name: str, index_col: Optional[Union[str, List[str]]] = None) -> DataFrame: + """ + Read a Spark table and return a DataFrame. + + Parameters + ---------- + name : string + Table name in Spark. + + index_col : str or list of str, optional, default: None + Index column of table in Spark. + + Returns + ------- + DataFrame + + See Also + -------- + DataFrame.to_table + read_delta + read_parquet + read_spark_io + + Examples + -------- + >>> pp.range(1).to_table('%s.my_table' % db) + >>> pp.read_table('%s.my_table' % db) + id + 0 0 + + >>> pp.range(1).to_table('%s.my_table' % db, index_col="index") + >>> pp.read_table('%s.my_table' % db, index_col="index") # doctest: +NORMALIZE_WHITESPACE + id + index + 0 0 + """ + sdf = default_session().read.table(name) + index_spark_columns, index_names = _get_index_map(sdf, index_col) + + return DataFrame( + InternalFrame( + spark_frame=sdf, index_spark_columns=index_spark_columns, index_names=index_names + ) + ) + + +def read_spark_io( + path: Optional[str] = None, + format: Optional[str] = None, + schema: Union[str, "StructType"] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options +) -> DataFrame: + """Load a DataFrame from a Spark data source. + + Parameters + ---------- + path : string, optional + Path to the data source. + format : string, optional + Specifies the output data source format. Some common ones are: + + - 'delta' + - 'parquet' + - 'orc' + - 'json' + - 'csv' + schema : string or StructType, optional + Input schema. If none, Spark tries to infer the schema automatically. + The schema can either be a Spark StructType, or a DDL-formatted string like + `col0 INT, col1 DOUBLE`. + index_col : str or list of str, optional, default: None + Index column of table in Spark. + options : dict + All other options passed directly into Spark's data source. + + See Also + -------- + DataFrame.to_spark_io + DataFrame.read_table + DataFrame.read_delta + DataFrame.read_parquet + + Examples + -------- + >>> pp.range(1).to_spark_io('%s/read_spark_io/data.parquet' % path) + >>> pp.read_spark_io( + ... '%s/read_spark_io/data.parquet' % path, format='parquet', schema='id long') + id + 0 0 + + >>> pp.range(10, 15, num_partitions=1).to_spark_io('%s/read_spark_io/data.json' % path, + ... format='json', lineSep='__') + >>> pp.read_spark_io( + ... '%s/read_spark_io/data.json' % path, format='json', schema='id long', lineSep='__') + id + 0 10 + 1 11 + 2 12 + 3 13 + 4 14 + + You can preserve the index in the roundtrip as below. + + >>> pp.range(10, 15, num_partitions=1).to_spark_io('%s/read_spark_io/data.orc' % path, + ... format='orc', index_col="index") + >>> pp.read_spark_io( + ... path=r'%s/read_spark_io/data.orc' % path, format="orc", index_col="index") + ... # doctest: +NORMALIZE_WHITESPACE + id + index + 0 10 + 1 11 + 2 12 + 3 13 + 4 14 + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + sdf = default_session().read.load(path=path, format=format, schema=schema, **options) + index_spark_columns, index_names = _get_index_map(sdf, index_col) + + return DataFrame( + InternalFrame( + spark_frame=sdf, index_spark_columns=index_spark_columns, index_names=index_names + ) + ) + + +def read_parquet(path, columns=None, index_col=None, pandas_metadata=False, **options) -> DataFrame: + """Load a parquet object from the file path, returning a DataFrame. + + Parameters + ---------- + path : string + File path + columns : list, default=None + If not None, only these columns will be read from the file. + index_col : str or list of str, optional, default: None + Index column of table in Spark. + pandas_metadata : bool, default: False + If True, try to respect the metadata if the Parquet file is written from pandas. + options : dict + All other options passed directly into Spark's data source. + + Returns + ------- + DataFrame + + See Also + -------- + DataFrame.to_parquet + DataFrame.read_table + DataFrame.read_delta + DataFrame.read_spark_io + + Examples + -------- + >>> pp.range(1).to_parquet('%s/read_spark_io/data.parquet' % path) + >>> pp.read_parquet('%s/read_spark_io/data.parquet' % path, columns=['id']) + id + 0 0 + + You can preserve the index in the roundtrip as below. + + >>> pp.range(1).to_parquet('%s/read_spark_io/data.parquet' % path, index_col="index") + >>> pp.read_parquet('%s/read_spark_io/data.parquet' % path, columns=['id'], index_col="index") + ... # doctest: +NORMALIZE_WHITESPACE + id + index + 0 0 + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + if columns is not None: + columns = list(columns) + + index_names = None + + if index_col is None and pandas_metadata: + if LooseVersion(pyspark.__version__) < LooseVersion("3.0.0"): + raise ValueError("pandas_metadata is not supported with Spark < 3.0.") + + # Try to read pandas metadata + + @pandas_udf("index_col array, index_names array", PandasUDFType.SCALAR) + def read_index_metadata(pser): + binary = pser.iloc[0] + metadata = pq.ParquetFile(pa.BufferReader(binary)).metadata.metadata + if b"pandas" in metadata: + pandas_metadata = json.loads(metadata[b"pandas"].decode("utf8")) + if all(isinstance(col, str) for col in pandas_metadata["index_columns"]): + index_col = [] + index_names = [] + for col in pandas_metadata["index_columns"]: + index_col.append(col) + for column in pandas_metadata["columns"]: + if column["field_name"] == col: + index_names.append(column["name"]) + break + else: + index_names.append(None) + return pd.DataFrame({"index_col": [index_col], "index_names": [index_names]}) + return pd.DataFrame({"index_col": [None], "index_names": [None]}) + + index_col, index_names = ( + default_session() + .read.format("binaryFile") + .load(path) + .limit(1) + .select(read_index_metadata("content").alias("index_metadata")) + .select("index_metadata.*") + .head() + ) + + kdf = read_spark_io(path=path, format="parquet", options=options, index_col=index_col) + + if columns is not None: + new_columns = [c for c in columns if c in kdf.columns] + if len(new_columns) > 0: + kdf = kdf[new_columns] + else: + sdf = default_session().createDataFrame([], schema=StructType()) + index_spark_columns, index_names = _get_index_map(sdf, index_col) + kdf = DataFrame( + InternalFrame( + spark_frame=sdf, + index_spark_columns=index_spark_columns, + index_names=index_names, + ) + ) + + if index_names is not None: + kdf.index.names = index_names + + return kdf + + +def read_clipboard(sep=r"\s+", **kwargs) -> DataFrame: + r""" + Read text from clipboard and pass to read_csv. See read_csv for the + full argument list + + Parameters + ---------- + sep : str, default '\s+' + A string or regex delimiter. The default of '\s+' denotes + one or more whitespace characters. + + See Also + -------- + DataFrame.to_clipboard : Write text out to clipboard. + + Returns + ------- + parsed : DataFrame + """ + return cast(DataFrame, from_pandas(pd.read_clipboard(sep, **kwargs))) + + +def read_excel( + io, + sheet_name=0, + header=0, + names=None, + index_col=None, + usecols=None, + squeeze=False, + dtype=None, + engine=None, + converters=None, + true_values=None, + false_values=None, + skiprows=None, + nrows=None, + na_values=None, + keep_default_na=True, + verbose=False, + parse_dates=False, + date_parser=None, + thousands=None, + comment=None, + skipfooter=0, + convert_float=True, + mangle_dupe_cols=True, + **kwds +) -> Union[DataFrame, Series, OrderedDict]: + """ + Read an Excel file into a Koalas DataFrame or Series. + + Support both `xls` and `xlsx` file extensions from a local filesystem or URL. + Support an option to read a single sheet or a list of sheets. + + Parameters + ---------- + io : str, file descriptor, pathlib.Path, ExcelFile or xlrd.Book + The string could be a URL. The value URL must be available in Spark's DataFrameReader. + + .. note:: + If the underlying Spark is below 3.0, the parameter as a string is not supported. + You can use `pp.from_pandas(pd.read_excel(...))` as a workaround. + + sheet_name : str, int, list, or None, default 0 + Strings are used for sheet names. Integers are used in zero-indexed + sheet positions. Lists of strings/integers are used to request + multiple sheets. Specify None to get all sheets. + + Available cases: + + * Defaults to ``0``: 1st sheet as a `DataFrame` + * ``1``: 2nd sheet as a `DataFrame` + * ``"Sheet1"``: Load sheet with name "Sheet1" + * ``[0, 1, "Sheet5"]``: Load first, second and sheet named "Sheet5" + as a dict of `DataFrame` + * None: All sheets. + + header : int, list of int, default 0 + Row (0-indexed) to use for the column labels of the parsed + DataFrame. If a list of integers is passed those row positions will + be combined into a ``MultiIndex``. Use None if there is no header. + names : array-like, default None + List of column names to use. If file contains no header row, + then you should explicitly pass header=None. + index_col : int, list of int, default None + Column (0-indexed) to use as the row labels of the DataFrame. + Pass None if there is no such column. If a list is passed, + those columns will be combined into a ``MultiIndex``. If a + subset of data is selected with ``usecols``, index_col + is based on the subset. + usecols : int, str, list-like, or callable default None + Return a subset of the columns. + + * If None, then parse all columns. + * If str, then indicates comma separated list of Excel column letters + and column ranges (e.g. "A:E" or "A,C,E:F"). Ranges are inclusive of + both sides. + * If list of int, then indicates list of column numbers to be parsed. + * If list of string, then indicates list of column names to be parsed. + * If callable, then evaluate each column name against it and parse the + column if the callable returns ``True``. + squeeze : bool, default False + If the parsed data only contains one column then return a Series. + dtype : Type name or dict of column -> type, default None + Data type for data or columns. E.g. {'a': np.float64, 'b': np.int32} + Use `object` to preserve data as stored in Excel and not interpret dtype. + If converters are specified, they will be applied INSTEAD + of dtype conversion. + engine : str, default None + If io is not a buffer or path, this must be set to identify io. + Acceptable values are None or xlrd. + converters : dict, default None + Dict of functions for converting values in certain columns. Keys can + either be integers or column labels, values are functions that take one + input argument, the Excel cell content, and return the transformed + content. + true_values : list, default None + Values to consider as True. + false_values : list, default None + Values to consider as False. + skiprows : list-like + Rows to skip at the beginning (0-indexed). + nrows : int, default None + Number of rows to parse. + na_values : scalar, str, list-like, or dict, default None + Additional strings to recognize as NA/NaN. If dict passed, specific + per-column NA values. By default the following values are interpreted + as NaN. + keep_default_na : bool, default True + If na_values are specified and keep_default_na is False the default NaN + values are overridden, otherwise they're appended to. + verbose : bool, default False + Indicate number of NA values placed in non-numeric columns. + parse_dates : bool, list-like, or dict, default False + The behavior is as follows: + + * bool. If True -> try parsing the index. + * list of int or names. e.g. If [1, 2, 3] -> try parsing columns 1, 2, 3 + each as a separate date column. + * list of lists. e.g. If [[1, 3]] -> combine columns 1 and 3 and parse as + a single date column. + * dict, e.g. {{'foo' : [1, 3]}} -> parse columns 1, 3 as date and call + result 'foo' + + If a column or index contains an unparseable date, the entire column or + index will be returned unaltered as an object data type. For non-standard + datetime parsing, use ``pd.to_datetime`` after ``pd.read_csv`` + + Note: A fast-path exists for iso8601-formatted dates. + date_parser : function, optional + Function to use for converting a sequence of string columns to an array of + datetime instances. The default uses ``dateutil.parser.parser`` to do the + conversion. Koalas will try to call `date_parser` in three different ways, + advancing to the next if an exception occurs: 1) Pass one or more arrays + (as defined by `parse_dates`) as arguments; 2) concatenate (row-wise) the + string values from the columns defined by `parse_dates` into a single array + and pass that; and 3) call `date_parser` once for each row using one or + more strings (corresponding to the columns defined by `parse_dates`) as + arguments. + thousands : str, default None + Thousands separator for parsing string columns to numeric. Note that + this parameter is only necessary for columns stored as TEXT in Excel, + any numeric columns will automatically be parsed, regardless of display + format. + comment : str, default None + Comments out remainder of line. Pass a character or characters to this + argument to indicate comments in the input file. Any data between the + comment string and the end of the current line is ignored. + skipfooter : int, default 0 + Rows at the end to skip (0-indexed). + convert_float : bool, default True + Convert integral floats to int (i.e., 1.0 --> 1). If False, all numeric + data will be read in as floats: Excel stores all numbers as floats + internally. + mangle_dupe_cols : bool, default True + Duplicate columns will be specified as 'X', 'X.1', ...'X.N', rather than + 'X'...'X'. Passing in False will cause data to be overwritten if there + are duplicate names in the columns. + **kwds : optional + Optional keyword arguments can be passed to ``TextFileReader``. + + Returns + ------- + DataFrame or dict of DataFrames + DataFrame from the passed in Excel file. See notes in sheet_name + argument for more information on when a dict of DataFrames is returned. + + See Also + -------- + DataFrame.to_excel : Write DataFrame to an Excel file. + DataFrame.to_csv : Write DataFrame to a comma-separated values (csv) file. + read_csv : Read a comma-separated values (csv) file into DataFrame. + + Examples + -------- + The file can be read using the file name as string or an open file object: + + >>> pp.read_excel('tmp.xlsx', index_col=0) # doctest: +SKIP + Name Value + 0 string1 1 + 1 string2 2 + 2 #Comment 3 + + >>> pp.read_excel(open('tmp.xlsx', 'rb'), + ... sheet_name='Sheet3') # doctest: +SKIP + Unnamed: 0 Name Value + 0 0 string1 1 + 1 1 string2 2 + 2 2 #Comment 3 + + Index and header can be specified via the `index_col` and `header` arguments + + >>> pp.read_excel('tmp.xlsx', index_col=None, header=None) # doctest: +SKIP + 0 1 2 + 0 NaN Name Value + 1 0.0 string1 1 + 2 1.0 string2 2 + 3 2.0 #Comment 3 + + Column types are inferred but can be explicitly specified + + >>> pp.read_excel('tmp.xlsx', index_col=0, + ... dtype={'Name': str, 'Value': float}) # doctest: +SKIP + Name Value + 0 string1 1.0 + 1 string2 2.0 + 2 #Comment 3.0 + + True, False, and NA values, and thousands separators have defaults, + but can be explicitly specified, too. Supply the values you would like + as strings or lists of strings! + + >>> pp.read_excel('tmp.xlsx', index_col=0, + ... na_values=['string1', 'string2']) # doctest: +SKIP + Name Value + 0 None 1 + 1 None 2 + 2 #Comment 3 + + Comment lines in the excel input file can be skipped using the `comment` kwarg + + >>> pp.read_excel('tmp.xlsx', index_col=0, comment='#') # doctest: +SKIP + Name Value + 0 string1 1.0 + 1 string2 2.0 + 2 None NaN + """ + + def pd_read_excel(io_or_bin, sn, sq): + return pd.read_excel( + io=BytesIO(io_or_bin) if isinstance(io_or_bin, (bytes, bytearray)) else io_or_bin, + sheet_name=sn, + header=header, + names=names, + index_col=index_col, + usecols=usecols, + squeeze=sq, + dtype=dtype, + engine=engine, + converters=converters, + true_values=true_values, + false_values=false_values, + skiprows=skiprows, + nrows=nrows, + na_values=na_values, + keep_default_na=keep_default_na, + verbose=verbose, + parse_dates=parse_dates, + date_parser=date_parser, + thousands=thousands, + comment=comment, + skipfooter=skipfooter, + convert_float=convert_float, + mangle_dupe_cols=mangle_dupe_cols, + **kwds + ) + + if isinstance(io, str): + if LooseVersion(pyspark.__version__) < LooseVersion("3.0.0"): + raise ValueError( + "The `io` parameter as a string is not supported if the underlying Spark is " + "below 3.0. You can use `pp.from_pandas(pd.read_excel(...))` as a workaround" + ) + # 'binaryFile' format is available since Spark 3.0.0. + binaries = default_session().read.format("binaryFile").load(io).select("content").head(2) + io_or_bin = binaries[0][0] + single_file = len(binaries) == 1 + else: + io_or_bin = io + single_file = True + + pdf_or_psers = pd_read_excel(io_or_bin, sn=sheet_name, sq=squeeze) + + if single_file: + if isinstance(pdf_or_psers, dict): + return OrderedDict( + [(sn, from_pandas(pdf_or_pser)) for sn, pdf_or_pser in pdf_or_psers.items()] + ) + else: + return cast(Union[DataFrame, Series], from_pandas(pdf_or_psers)) + else: + + def read_excel_on_spark(pdf_or_pser, sn): + + if isinstance(pdf_or_pser, pd.Series): + pdf = pdf_or_pser.to_frame() + else: + pdf = pdf_or_pser + + kdf = from_pandas(pdf) + return_schema = force_decimal_precision_scale( + as_nullable_spark_type(kdf._internal.spark_frame.drop(*HIDDEN_COLUMNS).schema) + ) + + def output_func(pdf): + pdf = pd.concat( + [pd_read_excel(bin, sn=sn, sq=False) for bin in pdf[pdf.columns[0]]] + ) + + reset_index = pdf.reset_index() + for name, col in reset_index.iteritems(): + dt = col.dtype + if is_datetime64_dtype(dt) or is_datetime64tz_dtype(dt): + continue + reset_index[name] = col.replace({np.nan: None}) + pdf = reset_index + + # Just positionally map the column names to given schema's. + return pdf.rename(columns=dict(zip(pdf.columns, return_schema.names))) + + sdf = ( + default_session() + .read.format("binaryFile") + .load(io) + .select("content") + .mapInPandas(lambda iterator: map(output_func, iterator), schema=return_schema) + ) + + kdf = DataFrame(kdf._internal.with_new_sdf(sdf)) + if squeeze and len(kdf.columns) == 1: + return first_series(kdf) + else: + return kdf + + if isinstance(pdf_or_psers, dict): + return OrderedDict( + [ + (sn, read_excel_on_spark(pdf_or_pser, sn)) + for sn, pdf_or_pser in pdf_or_psers.items() + ] + ) + else: + return read_excel_on_spark(pdf_or_psers, sheet_name) + + +def read_html( + io, + match=".+", + flavor=None, + header=None, + index_col=None, + skiprows=None, + attrs=None, + parse_dates=False, + thousands=",", + encoding=None, + decimal=".", + converters=None, + na_values=None, + keep_default_na=True, + displayed_only=True, +) -> List[DataFrame]: + r"""Read HTML tables into a ``list`` of ``DataFrame`` objects. + + Parameters + ---------- + io : str or file-like + A URL, a file-like object, or a raw string containing HTML. Note that + lxml only accepts the http, ftp and file url protocols. If you have a + URL that starts with ``'https'`` you might try removing the ``'s'``. + + match : str or compiled regular expression, optional + The set of tables containing text matching this regex or string will be + returned. Unless the HTML is extremely simple you will probably need to + pass a non-empty string here. Defaults to '.+' (match any non-empty + string). The default value will return all tables contained on a page. + This value is converted to a regular expression so that there is + consistent behavior between Beautiful Soup and lxml. + + flavor : str or None, container of strings + The parsing engine to use. 'bs4' and 'html5lib' are synonymous with + each other, they are both there for backwards compatibility. The + default of ``None`` tries to use ``lxml`` to parse and if that fails it + falls back on ``bs4`` + ``html5lib``. + + header : int or list-like or None, optional + The row (or list of rows for a :class:`~pp.MultiIndex`) to use to + make the columns headers. + + index_col : int or list-like or None, optional + The column (or list of columns) to use to create the index. + + skiprows : int or list-like or slice or None, optional + 0-based. Number of rows to skip after parsing the column integer. If a + sequence of integers or a slice is given, will skip the rows indexed by + that sequence. Note that a single element sequence means 'skip the nth + row' whereas an integer means 'skip n rows'. + + attrs : dict or None, optional + This is a dictionary of attributes that you can pass to use to identify + the table in the HTML. These are not checked for validity before being + passed to lxml or Beautiful Soup. However, these attributes must be + valid HTML table attributes to work correctly. For example, :: + + attrs = {'id': 'table'} + + is a valid attribute dictionary because the 'id' HTML tag attribute is + a valid HTML attribute for *any* HTML tag as per `this document + `__. :: + + attrs = {'asdf': 'table'} + + is *not* a valid attribute dictionary because 'asdf' is not a valid + HTML attribute even if it is a valid XML attribute. Valid HTML 4.01 + table attributes can be found `here + `__. A + working draft of the HTML 5 spec can be found `here + `__. It contains the + latest information on table attributes for the modern web. + + parse_dates : bool, optional + See :func:`~pp.read_csv` for more details. + + thousands : str, optional + Separator to use to parse thousands. Defaults to ``','``. + + encoding : str or None, optional + The encoding used to decode the web page. Defaults to ``None``.``None`` + preserves the previous encoding behavior, which depends on the + underlying parser library (e.g., the parser library will try to use + the encoding provided by the document). + + decimal : str, default '.' + Character to recognize as decimal point (e.g. use ',' for European + data). + + converters : dict, default None + Dict of functions for converting values in certain columns. Keys can + either be integers or column labels, values are functions that take one + input argument, the cell (not column) content, and return the + transformed content. + + na_values : iterable, default None + Custom NA values + + keep_default_na : bool, default True + If na_values are specified and keep_default_na is False the default NaN + values are overridden, otherwise they're appended to + + displayed_only : bool, default True + Whether elements with "display: none" should be parsed + + Returns + ------- + dfs : list of DataFrames + + See Also + -------- + read_csv + DataFrame.to_html + """ + pdfs = pd.read_html( + io=io, + match=match, + flavor=flavor, + header=header, + index_col=index_col, + skiprows=skiprows, + attrs=attrs, + parse_dates=parse_dates, + thousands=thousands, + encoding=encoding, + decimal=decimal, + converters=converters, + na_values=na_values, + keep_default_na=keep_default_na, + displayed_only=displayed_only, + ) + return cast(List[DataFrame], [from_pandas(pdf) for pdf in pdfs]) + + +# TODO: add `coerce_float` and 'parse_dates' parameters +def read_sql_table( + table_name, con, schema=None, index_col=None, columns=None, **options +) -> DataFrame: + """ + Read SQL database table into a DataFrame. + + Given a table name and a JDBC URI, returns a DataFrame. + + Parameters + ---------- + table_name : str + Name of SQL table in database. + con : str + A JDBC URI could be provided as as str. + + .. note:: The URI must be JDBC URI instead of Python's database URI. + + schema : str, default None + Name of SQL schema in database to query (if database flavor + supports this). Uses default schema if None (default). + index_col : str or list of str, optional, default: None + Column(s) to set as index(MultiIndex). + columns : list, default None + List of column names to select from SQL table. + options : dict + All other options passed directly into Spark's JDBC data source. + + Returns + ------- + DataFrame + A SQL table is returned as two-dimensional data structure with labeled + axes. + + See Also + -------- + read_sql_query : Read SQL query into a DataFrame. + read_sql : Read SQL query or database table into a DataFrame. + + Examples + -------- + >>> pp.read_sql_table('table_name', 'jdbc:postgresql:db_name') # doctest: +SKIP + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + reader = default_session().read + reader.option("dbtable", table_name) + reader.option("url", con) + if schema is not None: + reader.schema(schema) + reader.options(**options) + sdf = reader.format("jdbc").load() + index_spark_columns, index_names = _get_index_map(sdf, index_col) + kdf = DataFrame( + InternalFrame( + spark_frame=sdf, index_spark_columns=index_spark_columns, index_names=index_names + ) + ) # type: DataFrame + if columns is not None: + if isinstance(columns, str): + columns = [columns] + kdf = kdf[columns] + return kdf + + +# TODO: add `coerce_float`, `params`, and 'parse_dates' parameters +def read_sql_query(sql, con, index_col=None, **options) -> DataFrame: + """Read SQL query into a DataFrame. + + Returns a DataFrame corresponding to the result set of the query + string. Optionally provide an `index_col` parameter to use one of the + columns as the index, otherwise default index will be used. + + .. note:: Some database might hit the issue of Spark: SPARK-27596 + + Parameters + ---------- + sql : string SQL query + SQL query to be executed. + con : str + A JDBC URI could be provided as as str. + + .. note:: The URI must be JDBC URI instead of Python's database URI. + + index_col : string or list of strings, optional, default: None + Column(s) to set as index(MultiIndex). + options : dict + All other options passed directly into Spark's JDBC data source. + + Returns + ------- + DataFrame + + See Also + -------- + read_sql_table : Read SQL database table into a DataFrame. + read_sql + + Examples + -------- + >>> pp.read_sql_query('SELECT * FROM table_name', 'jdbc:postgresql:db_name') # doctest: +SKIP + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + reader = default_session().read + reader.option("query", sql) + reader.option("url", con) + reader.options(**options) + sdf = reader.format("jdbc").load() + index_spark_columns, index_names = _get_index_map(sdf, index_col) + return DataFrame( + InternalFrame( + spark_frame=sdf, index_spark_columns=index_spark_columns, index_names=index_names + ) + ) + + +# TODO: add `coerce_float`, `params`, and 'parse_dates' parameters +def read_sql(sql, con, index_col=None, columns=None, **options) -> DataFrame: + """ + Read SQL query or database table into a DataFrame. + + This function is a convenience wrapper around ``read_sql_table`` and + ``read_sql_query`` (for backward compatibility). It will delegate + to the specific function depending on the provided input. A SQL query + will be routed to ``read_sql_query``, while a database table name will + be routed to ``read_sql_table``. Note that the delegated function might + have more specific notes about their functionality not listed here. + + .. note:: Some database might hit the issue of Spark: SPARK-27596 + + Parameters + ---------- + sql : string + SQL query to be executed or a table name. + con : str + A JDBC URI could be provided as as str. + + .. note:: The URI must be JDBC URI instead of Python's database URI. + + index_col : string or list of strings, optional, default: None + Column(s) to set as index(MultiIndex). + columns : list, default: None + List of column names to select from SQL table (only used when reading + a table). + options : dict + All other options passed directly into Spark's JDBC data source. + + Returns + ------- + DataFrame + + See Also + -------- + read_sql_table : Read SQL database table into a DataFrame. + read_sql_query : Read SQL query into a DataFrame. + + Examples + -------- + >>> pp.read_sql('table_name', 'jdbc:postgresql:db_name') # doctest: +SKIP + >>> pp.read_sql('SELECT * FROM table_name', 'jdbc:postgresql:db_name') # doctest: +SKIP + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + striped = sql.strip() + if " " not in striped: # TODO: identify the table name or not more precisely. + return read_sql_table(sql, con, index_col=index_col, columns=columns, **options) + else: + return read_sql_query(sql, con, index_col=index_col, **options) + + +def to_datetime( + arg, errors="raise", format=None, unit=None, infer_datetime_format=False, origin="unix" +): + """ + Convert argument to datetime. + + Parameters + ---------- + arg : integer, float, string, datetime, list, tuple, 1-d array, Series + or DataFrame/dict-like + + errors : {'ignore', 'raise', 'coerce'}, default 'raise' + + - If 'raise', then invalid parsing will raise an exception + - If 'coerce', then invalid parsing will be set as NaT + - If 'ignore', then invalid parsing will return the input + format : string, default None + strftime to parse time, eg "%d/%m/%Y", note that "%f" will parse + all the way up to nanoseconds. + unit : string, default None + unit of the arg (D,s,ms,us,ns) denote the unit, which is an + integer or float number. This will be based off the origin. + Example, with unit='ms' and origin='unix' (the default), this + would calculate the number of milliseconds to the unix epoch start. + infer_datetime_format : boolean, default False + If True and no `format` is given, attempt to infer the format of the + datetime strings, and if it can be inferred, switch to a faster + method of parsing them. In some cases this can increase the parsing + speed by ~5-10x. + origin : scalar, default 'unix' + Define the reference date. The numeric values would be parsed as number + of units (defined by `unit`) since this reference date. + + - If 'unix' (or POSIX) time; origin is set to 1970-01-01. + - If 'julian', unit must be 'D', and origin is set to beginning of + Julian Calendar. Julian day number 0 is assigned to the day starting + at noon on January 1, 4713 BC. + - If Timestamp convertible, origin is set to Timestamp identified by + origin. + + Returns + ------- + ret : datetime if parsing succeeded. + Return type depends on input: + + - list-like: DatetimeIndex + - Series: Series of datetime64 dtype + - scalar: Timestamp + + In case when it is not possible to return designated types (e.g. when + any element of input is before Timestamp.min or after Timestamp.max) + return will have datetime.datetime type (or corresponding + array/Series). + + Examples + -------- + Assembling a datetime from multiple columns of a DataFrame. The keys can be + common abbreviations like ['year', 'month', 'day', 'minute', 'second', + 'ms', 'us', 'ns']) or plurals of the same + + >>> df = pp.DataFrame({'year': [2015, 2016], + ... 'month': [2, 3], + ... 'day': [4, 5]}) + >>> pp.to_datetime(df) + 0 2015-02-04 + 1 2016-03-05 + dtype: datetime64[ns] + + If a date does not meet the `timestamp limitations + `_, passing errors='ignore' + will return the original input instead of raising any exception. + + Passing errors='coerce' will force an out-of-bounds date to NaT, + in addition to forcing non-dates (or non-parseable dates) to NaT. + + >>> pp.to_datetime('13000101', format='%Y%m%d', errors='ignore') + datetime.datetime(1300, 1, 1, 0, 0) + >>> pp.to_datetime('13000101', format='%Y%m%d', errors='coerce') + NaT + + Passing infer_datetime_format=True can often-times speedup a parsing + if its not an ISO8601 format exactly, but in a regular format. + + >>> s = pp.Series(['3/11/2000', '3/12/2000', '3/13/2000'] * 1000) + >>> s.head() + 0 3/11/2000 + 1 3/12/2000 + 2 3/13/2000 + 3 3/11/2000 + 4 3/12/2000 + dtype: object + + >>> import timeit + >>> timeit.timeit( + ... lambda: repr(pp.to_datetime(s, infer_datetime_format=True)), + ... number = 1) # doctest: +SKIP + 0.35832712500000063 + + >>> timeit.timeit( + ... lambda: repr(pp.to_datetime(s, infer_datetime_format=False)), + ... number = 1) # doctest: +SKIP + 0.8895321660000004 + + Using a unix epoch time + + >>> pp.to_datetime(1490195805, unit='s') + Timestamp('2017-03-22 15:16:45') + >>> pp.to_datetime(1490195805433502912, unit='ns') + Timestamp('2017-03-22 15:16:45.433502912') + + Using a non-unix epoch origin + + >>> pp.to_datetime([1, 2, 3], unit='D', origin=pd.Timestamp('1960-01-01')) + DatetimeIndex(['1960-01-02', '1960-01-03', '1960-01-04'], dtype='datetime64[ns]', freq=None) + """ + + def pandas_to_datetime(pser_or_pdf) -> Series[np.datetime64]: + if isinstance(pser_or_pdf, pd.DataFrame): + pser_or_pdf = pser_or_pdf[["year", "month", "day"]] + return pd.to_datetime( + pser_or_pdf, + errors=errors, + format=format, + unit=unit, + infer_datetime_format=infer_datetime_format, + origin=origin, + ) + + if isinstance(arg, Series): + return arg.koalas.transform_batch(pandas_to_datetime) + if isinstance(arg, DataFrame): + kdf = arg[["year", "month", "day"]] + return kdf.koalas.transform_batch(pandas_to_datetime) + return pd.to_datetime( + arg, + errors=errors, + format=format, + unit=unit, + infer_datetime_format=infer_datetime_format, + origin=origin, + ) + + +def date_range( + start=None, + end=None, + periods=None, + freq=None, + tz=None, + normalize=False, + name=None, + closed=None, + **kwargs +) -> DatetimeIndex: + """ + Return a fixed frequency DatetimeIndex. + + Parameters + ---------- + start : str or datetime-like, optional + Left bound for generating dates. + end : str or datetime-like, optional + Right bound for generating dates. + periods : int, optional + Number of periods to generate. + freq : str or DateOffset, default 'D' + Frequency strings can have multiples, e.g. '5H'. + tz : str or tzinfo, optional + Time zone name for returning localized DatetimeIndex, for example + 'Asia/Hong_Kong'. By default, the resulting DatetimeIndex is + timezone-naive. + normalize : bool, default False + Normalize start/end dates to midnight before generating date range. + name : str, default None + Name of the resulting DatetimeIndex. + closed : {None, 'left', 'right'}, optional + Make the interval closed with respect to the given frequency to + the 'left', 'right', or both sides (None, the default). + **kwargs + For compatibility. Has no effect on the result. + + Returns + ------- + rng : DatetimeIndex + + See Also + -------- + DatetimeIndex : An immutable container for datetimes. + + Notes + ----- + Of the four parameters ``start``, ``end``, ``periods``, and ``freq``, + exactly three must be specified. If ``freq`` is omitted, the resulting + ``DatetimeIndex`` will have ``periods`` linearly spaced elements between + ``start`` and ``end`` (closed on both sides). + + To learn more about the frequency strings, please see `this link + `__. + + Examples + -------- + **Specifying the values** + + The next four examples generate the same `DatetimeIndex`, but vary + the combination of `start`, `end` and `periods`. + + Specify `start` and `end`, with the default daily frequency. + + >>> pp.date_range(start='1/1/2018', end='1/08/2018') # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-01', '2018-01-02', '2018-01-03', '2018-01-04', + '2018-01-05', '2018-01-06', '2018-01-07', '2018-01-08'], + dtype='datetime64[ns]', freq=None) + + Specify `start` and `periods`, the number of periods (days). + + >>> pp.date_range(start='1/1/2018', periods=8) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-01', '2018-01-02', '2018-01-03', '2018-01-04', + '2018-01-05', '2018-01-06', '2018-01-07', '2018-01-08'], + dtype='datetime64[ns]', freq=None) + + Specify `end` and `periods`, the number of periods (days). + + >>> pp.date_range(end='1/1/2018', periods=8) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2017-12-25', '2017-12-26', '2017-12-27', '2017-12-28', + '2017-12-29', '2017-12-30', '2017-12-31', '2018-01-01'], + dtype='datetime64[ns]', freq=None) + + Specify `start`, `end`, and `periods`; the frequency is generated + automatically (linearly spaced). + + >>> pp.date_range( + ... start='2018-04-24', end='2018-04-27', periods=3 + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-04-24 00:00:00', '2018-04-25 12:00:00', + '2018-04-27 00:00:00'], + dtype='datetime64[ns]', freq=None) + + **Other Parameters** + + Changed the `freq` (frequency) to ``'M'`` (month end frequency). + + >>> pp.date_range(start='1/1/2018', periods=5, freq='M') # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-31', '2018-02-28', '2018-03-31', '2018-04-30', + '2018-05-31'], + dtype='datetime64[ns]', freq=None) + + Multiples are allowed + + >>> pp.date_range(start='1/1/2018', periods=5, freq='3M') # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-31', '2018-04-30', '2018-07-31', '2018-10-31', + '2019-01-31'], + dtype='datetime64[ns]', freq=None) + + `freq` can also be specified as an Offset object. + + >>> pp.date_range( + ... start='1/1/2018', periods=5, freq=pd.offsets.MonthEnd(3) + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2018-01-31', '2018-04-30', '2018-07-31', '2018-10-31', + '2019-01-31'], + dtype='datetime64[ns]', freq=None) + + `closed` controls whether to include `start` and `end` that are on the + boundary. The default includes boundary points on either end. + + >>> pp.date_range( + ... start='2017-01-01', end='2017-01-04', closed=None + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2017-01-01', '2017-01-02', '2017-01-03', '2017-01-04'], + dtype='datetime64[ns]', freq=None) + + Use ``closed='left'`` to exclude `end` if it falls on the boundary. + + >>> pp.date_range( + ... start='2017-01-01', end='2017-01-04', closed='left' + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2017-01-01', '2017-01-02', '2017-01-03'], dtype='datetime64[ns]', freq=None) + + Use ``closed='right'`` to exclude `start` if it falls on the boundary. + + >>> pp.date_range( + ... start='2017-01-01', end='2017-01-04', closed='right' + ... ) # doctest: +NORMALIZE_WHITESPACE + DatetimeIndex(['2017-01-02', '2017-01-03', '2017-01-04'], dtype='datetime64[ns]', freq=None) + """ + assert freq not in ["N", "ns"], "nanoseconds is not supported" + assert tz is None, "Localized DatetimeIndex is not supported" + + return cast( + DatetimeIndex, + pp.from_pandas( + pd.date_range( + start=start, + end=end, + periods=periods, + freq=freq, + tz=tz, + normalize=normalize, + name=name, + closed=closed, + **kwargs + ) + ), + ) + + +def get_dummies( + data, + prefix=None, + prefix_sep="_", + dummy_na=False, + columns=None, + sparse=False, + drop_first=False, + dtype=None, +) -> DataFrame: + """ + Convert categorical variable into dummy/indicator variables, also + known as one hot encoding. + + Parameters + ---------- + data : array-like, Series, or DataFrame + prefix : string, list of strings, or dict of strings, default None + String to append DataFrame column names. + Pass a list with length equal to the number of columns + when calling get_dummies on a DataFrame. Alternatively, `prefix` + can be a dictionary mapping column names to prefixes. + prefix_sep : string, default '_' + If appending prefix, separator/delimiter to use. Or pass a + list or dictionary as with `prefix.` + dummy_na : bool, default False + Add a column to indicate NaNs, if False NaNs are ignored. + columns : list-like, default None + Column names in the DataFrame to be encoded. + If `columns` is None then all the columns with + `object` or `category` dtype will be converted. + sparse : bool, default False + Whether the dummy-encoded columns should be be backed by + a :class:`SparseArray` (True) or a regular NumPy array (False). + In Koalas, this value must be "False". + drop_first : bool, default False + Whether to get k-1 dummies out of k categorical levels by removing the + first level. + dtype : dtype, default np.uint8 + Data type for new columns. Only a single dtype is allowed. + + Returns + ------- + dummies : DataFrame + + See Also + -------- + Series.str.get_dummies + + Examples + -------- + >>> s = pp.Series(list('abca')) + + >>> pp.get_dummies(s) + a b c + 0 1 0 0 + 1 0 1 0 + 2 0 0 1 + 3 1 0 0 + + >>> df = pp.DataFrame({'A': ['a', 'b', 'a'], 'B': ['b', 'a', 'c'], + ... 'C': [1, 2, 3]}, + ... columns=['A', 'B', 'C']) + + >>> pp.get_dummies(df, prefix=['col1', 'col2']) + C col1_a col1_b col2_a col2_b col2_c + 0 1 1 0 0 1 0 + 1 2 0 1 1 0 0 + 2 3 1 0 0 0 1 + + >>> pp.get_dummies(pp.Series(list('abcaa'))) + a b c + 0 1 0 0 + 1 0 1 0 + 2 0 0 1 + 3 1 0 0 + 4 1 0 0 + + >>> pp.get_dummies(pp.Series(list('abcaa')), drop_first=True) + b c + 0 0 0 + 1 1 0 + 2 0 1 + 3 0 0 + 4 0 0 + + >>> pp.get_dummies(pp.Series(list('abc')), dtype=float) + a b c + 0 1.0 0.0 0.0 + 1 0.0 1.0 0.0 + 2 0.0 0.0 1.0 + """ + if sparse is not False: + raise NotImplementedError("get_dummies currently does not support sparse") + + if columns is not None: + if not is_list_like(columns): + raise TypeError("Input must be a list-like for parameter `columns`") + + if dtype is None: + dtype = "byte" + + if isinstance(data, Series): + if prefix is not None: + prefix = [str(prefix)] + kdf = data.to_frame() + column_labels = kdf._internal.column_labels + remaining_columns = [] + else: + if isinstance(prefix, str): + raise NotImplementedError( + "get_dummies currently does not support prefix as string types" + ) + kdf = data.copy() + + if columns is None: + column_labels = [ + label + for label in kdf._internal.column_labels + if isinstance( + kdf._internal.spark_type_for(label), _get_dummies_default_accept_types + ) + ] + else: + if is_name_like_tuple(columns): + column_labels = [ + label + for label in kdf._internal.column_labels + if label[: len(columns)] == columns + ] + if len(column_labels) == 0: + raise KeyError(name_like_string(columns)) + if prefix is None: + prefix = [ + str(label[len(columns):]) + if len(label) > len(columns) + 1 + else label[len(columns)] + if len(label) == len(columns) + 1 + else "" + for label in column_labels + ] + elif any(isinstance(col, tuple) for col in columns) and any( + not is_name_like_tuple(col) for col in columns + ): + raise ValueError( + "Expected tuple, got {}".format( + type(set(col for col in columns if not is_name_like_tuple(col)).pop()) + ) + ) + else: + column_labels = [ + label + for key in columns + for label in kdf._internal.column_labels + if label == key or label[0] == key + ] + if len(column_labels) == 0: + if columns is None: + return kdf + raise KeyError("{} not in index".format(columns)) + + if prefix is None: + prefix = [str(label) if len(label) > 1 else label[0] for label in column_labels] + + column_labels_set = set(column_labels) + remaining_columns = [ + ( + kdf[label] + if kdf._internal.column_labels_level == 1 + else kdf[label].rename(name_like_string(label)) + ) + for label in kdf._internal.column_labels + if label not in column_labels_set + ] + + if any( + not isinstance(kdf._internal.spark_type_for(label), _get_dummies_acceptable_types) + for label in column_labels + ): + raise NotImplementedError( + "get_dummies currently only accept {} values".format( + ", ".join([t.typeName() for t in _get_dummies_acceptable_types]) + ) + ) + + if prefix is not None and len(column_labels) != len(prefix): + raise ValueError( + "Length of 'prefix' ({}) did not match the length of " + "the columns being encoded ({}).".format(len(prefix), len(column_labels)) + ) + elif isinstance(prefix, dict): + prefix = [prefix[column_label[0]] for column_label in column_labels] + + all_values = _reduce_spark_multi( + kdf._internal.spark_frame, + [F.collect_set(kdf._internal.spark_column_for(label)) for label in column_labels], + ) + for i, label in enumerate(column_labels): + values = all_values[i] + if isinstance(values, np.ndarray): + values = values.tolist() + values = sorted(values) + if drop_first: + values = values[1:] + + def column_name(value): + if prefix is None or prefix[i] == "": + return value + else: + return "{}{}{}".format(prefix[i], prefix_sep, value) + + for value in values: + remaining_columns.append( + (kdf[label].notnull() & (kdf[label] == value)) + .astype(dtype) + .rename(column_name(value)) + ) + if dummy_na: + remaining_columns.append(kdf[label].isnull().astype(dtype).rename(column_name(np.nan))) + + return kdf[remaining_columns] + + +# TODO: there are many parameters to implement and support. See pandas's pd.concat. +def concat(objs, axis=0, join="outer", ignore_index=False, sort=False) -> Union[Series, DataFrame]: + """ + Concatenate Koalas objects along a particular axis with optional set logic + along the other axes. + + Parameters + ---------- + objs : a sequence of Series or DataFrame + Any None objects will be dropped silently unless + they are all None in which case a ValueError will be raised + axis : {0/'index', 1/'columns'}, default 0 + The axis to concatenate along. + join : {'inner', 'outer'}, default 'outer' + How to handle indexes on other axis (or axes). + ignore_index : bool, default False + If True, do not use the index values along the concatenation axis. The + resulting axis will be labeled 0, ..., n - 1. This is useful if you are + concatenating objects where the concatenation axis does not have + meaningful indexing information. Note the index values on the other + axes are still respected in the join. + sort : bool, default False + Sort non-concatenation axis if it is not already aligned. + + Returns + ------- + object, type of objs + When concatenating all ``Series`` along the index (axis=0), a + ``Series`` is returned. When ``objs`` contains at least one + ``DataFrame``, a ``DataFrame`` is returned. When concatenating along + the columns (axis=1), a ``DataFrame`` is returned. + + See Also + -------- + Series.append : Concatenate Series. + DataFrame.join : Join DataFrames using indexes. + DataFrame.merge : Merge DataFrames by indexes or columns. + + Examples + -------- + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + + Combine two ``Series``. + + >>> s1 = pp.Series(['a', 'b']) + >>> s2 = pp.Series(['c', 'd']) + >>> pp.concat([s1, s2]) + 0 a + 1 b + 0 c + 1 d + dtype: object + + Clear the existing index and reset it in the result + by setting the ``ignore_index`` option to ``True``. + + >>> pp.concat([s1, s2], ignore_index=True) + 0 a + 1 b + 2 c + 3 d + dtype: object + + Combine two ``DataFrame`` objects with identical columns. + + >>> df1 = pp.DataFrame([['a', 1], ['b', 2]], + ... columns=['letter', 'number']) + >>> df1 + letter number + 0 a 1 + 1 b 2 + >>> df2 = pp.DataFrame([['c', 3], ['d', 4]], + ... columns=['letter', 'number']) + >>> df2 + letter number + 0 c 3 + 1 d 4 + + >>> pp.concat([df1, df2]) + letter number + 0 a 1 + 1 b 2 + 0 c 3 + 1 d 4 + + Combine ``DataFrame`` and ``Series`` objects with different columns. + + >>> pp.concat([df2, s1]) + letter number 0 + 0 c 3.0 None + 1 d 4.0 None + 0 None NaN a + 1 None NaN b + + Combine ``DataFrame`` objects with overlapping columns + and return everything. Columns outside the intersection will + be filled with ``None`` values. + + >>> df3 = pp.DataFrame([['c', 3, 'cat'], ['d', 4, 'dog']], + ... columns=['letter', 'number', 'animal']) + >>> df3 + letter number animal + 0 c 3 cat + 1 d 4 dog + + >>> pp.concat([df1, df3]) + letter number animal + 0 a 1 None + 1 b 2 None + 0 c 3 cat + 1 d 4 dog + + Sort the columns. + + >>> pp.concat([df1, df3], sort=True) + animal letter number + 0 None a 1 + 1 None b 2 + 0 cat c 3 + 1 dog d 4 + + Combine ``DataFrame`` objects with overlapping columns + and return only those that are shared by passing ``inner`` to + the ``join`` keyword argument. + + >>> pp.concat([df1, df3], join="inner") + letter number + 0 a 1 + 1 b 2 + 0 c 3 + 1 d 4 + + >>> df4 = pp.DataFrame([['bird', 'polly'], ['monkey', 'george']], + ... columns=['animal', 'name']) + + Combine with column axis. + + >>> pp.concat([df1, df4], axis=1) + letter number animal name + 0 a 1 bird polly + 1 b 2 monkey george + + >>> reset_option("compute.ops_on_diff_frames") + """ + if isinstance(objs, (DataFrame, IndexOpsMixin)) or not isinstance( + objs, Iterable + ): # TODO: support dict + raise TypeError( + "first argument must be an iterable of Koalas " + "objects, you passed an object of type " + '"{name}"'.format(name=type(objs).__name__) + ) + + if len(cast(Sized, objs)) == 0: + raise ValueError("No objects to concatenate") + objs = list(filter(lambda obj: obj is not None, objs)) + if len(objs) == 0: + raise ValueError("All objects passed were None") + + for obj in objs: + if not isinstance(obj, (Series, DataFrame)): + raise TypeError( + "cannot concatenate object of type " + "'{name}" + "; only pp.Series " + "and pp.DataFrame are valid".format(name=type(objs).__name__) + ) + + if join not in ["inner", "outer"]: + raise ValueError("Only can inner (intersect) or outer (union) join the other axis.") + + axis = validate_axis(axis) + if axis == 1: + kdfs = [obj.to_frame() if isinstance(obj, Series) else obj for obj in objs] + + level = min(kdf._internal.column_labels_level for kdf in kdfs) + kdfs = [ + DataFrame._index_normalized_frame(level, kdf) + if kdf._internal.column_labels_level > level + else kdf + for kdf in kdfs + ] + + concat_kdf = kdfs[0] + column_labels = concat_kdf._internal.column_labels.copy() + + kdfs_not_same_anchor = [] + for kdf in kdfs[1:]: + duplicated = [label for label in kdf._internal.column_labels if label in column_labels] + if len(duplicated) > 0: + pretty_names = [name_like_string(label) for label in duplicated] + raise ValueError( + "Labels have to be unique; however, got duplicated labels %s." % pretty_names + ) + column_labels.extend(kdf._internal.column_labels) + + if same_anchor(concat_kdf, kdf): + concat_kdf = DataFrame( + concat_kdf._internal.with_new_columns( + [ + concat_kdf._kser_for(label) + for label in concat_kdf._internal.column_labels + ] + + [kdf._kser_for(label) for label in kdf._internal.column_labels] + ) + ) + else: + kdfs_not_same_anchor.append(kdf) + + if len(kdfs_not_same_anchor) > 0: + + def resolve_func(kdf, this_column_labels, that_column_labels): + raise AssertionError("This should not happen.") + + for kdf in kdfs_not_same_anchor: + if join == "inner": + concat_kdf = align_diff_frames( + resolve_func, concat_kdf, kdf, fillna=False, how="inner", + ) + elif join == "outer": + concat_kdf = align_diff_frames( + resolve_func, concat_kdf, kdf, fillna=False, how="full", + ) + + concat_kdf = concat_kdf[column_labels] + + if ignore_index: + concat_kdf.columns = list(map(str, _range(len(concat_kdf.columns)))) + + if sort: + concat_kdf = concat_kdf.sort_index() + + return concat_kdf + + # Series, Series ... + # We should return Series if objects are all Series. + should_return_series = all(map(lambda obj: isinstance(obj, Series), objs)) + + # DataFrame, Series ... & Series, Series ... + # In this case, we should return DataFrame. + new_objs = [] + num_series = 0 + series_names = set() + for obj in objs: + if isinstance(obj, Series): + num_series += 1 + series_names.add(obj.name) + obj = obj.to_frame(DEFAULT_SERIES_NAME) + new_objs.append(obj) + objs = new_objs + + column_labels_levels = set(obj._internal.column_labels_level for obj in objs) + if len(column_labels_levels) != 1: + raise ValueError("MultiIndex columns should have the same levels") + + # DataFrame, DataFrame, ... + # All Series are converted into DataFrame and then compute concat. + if not ignore_index: + indices_of_kdfs = [kdf.index for kdf in objs] + index_of_first_kdf = indices_of_kdfs[0] + for index_of_kdf in indices_of_kdfs: + if index_of_first_kdf.names != index_of_kdf.names: + raise ValueError( + "Index type and names should be same in the objects to concatenate. " + "You passed different indices " + "{index_of_first_kdf} and {index_of_kdf}".format( + index_of_first_kdf=index_of_first_kdf.names, index_of_kdf=index_of_kdf.names + ) + ) + + column_labels_of_kdfs = [kdf._internal.column_labels for kdf in objs] + if ignore_index: + index_names_of_kdfs = [[] for _ in objs] # type: List + else: + index_names_of_kdfs = [kdf._internal.index_names for kdf in objs] + + if all(name == index_names_of_kdfs[0] for name in index_names_of_kdfs) and all( + idx == column_labels_of_kdfs[0] for idx in column_labels_of_kdfs + ): + # If all columns are in the same order and values, use it. + kdfs = objs + else: + if join == "inner": + interested_columns = set.intersection(*map(set, column_labels_of_kdfs)) + # Keep the column order with its firsts DataFrame. + merged_columns = [ + label for label in column_labels_of_kdfs[0] if label in interested_columns + ] + + # When multi-index column, although pandas is flaky if `join="inner" and sort=False`, + # always sort to follow the `join="outer"` case behavior. + if (len(merged_columns) > 0 and len(merged_columns[0]) > 1) or sort: + # FIXME: better ordering + merged_columns = sorted(merged_columns, key=name_like_string) + + kdfs = [kdf[merged_columns] for kdf in objs] + elif join == "outer": + merged_columns = [] + for labels in column_labels_of_kdfs: + merged_columns.extend(label for label in labels if label not in merged_columns) + + assert len(merged_columns) > 0 + + if LooseVersion(pd.__version__) < LooseVersion("0.24"): + # Always sort when multi-index columns, and if there are Series, never sort. + sort = len(merged_columns[0]) > 1 or (num_series == 0 and sort) + else: + # Always sort when multi-index columns or there are more than two Series, + # and if there is only one Series, never sort. + sort = len(merged_columns[0]) > 1 or num_series > 1 or (num_series != 1 and sort) + + if sort: + # FIXME: better ordering + merged_columns = sorted(merged_columns, key=name_like_string) + + kdfs = [] + for kdf in objs: + columns_to_add = list(set(merged_columns) - set(kdf._internal.column_labels)) + + # TODO: NaN and None difference for missing values. pandas seems filling NaN. + sdf = kdf._internal.resolved_copy.spark_frame + for label in columns_to_add: + sdf = sdf.withColumn(name_like_string(label), F.lit(None)) + + data_columns = kdf._internal.data_spark_column_names + [ + name_like_string(label) for label in columns_to_add + ] + kdf = DataFrame( + kdf._internal.copy( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in kdf._internal.index_spark_column_names + ], + column_labels=(kdf._internal.column_labels + columns_to_add), + data_spark_columns=[scol_for(sdf, col) for col in data_columns], + data_dtypes=(kdf._internal.data_dtypes + ([None] * len(columns_to_add))), + ) + ) + + kdfs.append(kdf[merged_columns]) + + if ignore_index: + sdfs = [kdf._internal.spark_frame.select(kdf._internal.data_spark_columns) for kdf in kdfs] + else: + sdfs = [ + kdf._internal.spark_frame.select( + kdf._internal.index_spark_columns + kdf._internal.data_spark_columns + ) + for kdf in kdfs + ] + concatenated = reduce(lambda x, y: x.union(y), sdfs) + + if ignore_index: + index_spark_column_names = [] + index_names = [] + index_dtypes = [] + else: + index_spark_column_names = kdfs[0]._internal.index_spark_column_names + index_names = kdfs[0]._internal.index_names + index_dtypes = kdfs[0]._internal.index_dtypes + + result_kdf = DataFrame( + kdfs[0]._internal.copy( + spark_frame=concatenated, + index_spark_columns=[scol_for(concatenated, col) for col in index_spark_column_names], + index_names=index_names, + index_dtypes=index_dtypes, + data_spark_columns=[ + scol_for(concatenated, col) for col in kdfs[0]._internal.data_spark_column_names + ], + data_dtypes=None, # TODO: dtypes? + ) + ) # type: DataFrame + + if should_return_series: + # If all input were Series, we should return Series. + if len(series_names) == 1: + name = series_names.pop() + else: + name = None + return first_series(result_kdf).rename(name) + else: + return result_kdf + + +def melt(frame, id_vars=None, value_vars=None, var_name=None, value_name="value") -> DataFrame: + return DataFrame.melt(frame, id_vars, value_vars, var_name, value_name) + + +melt.__doc__ = DataFrame.melt.__doc__ + + +def isna(obj): + """ + Detect missing values for an array-like object. + + This function takes a scalar or array-like object and indicates + whether values are missing (``NaN`` in numeric arrays, ``None`` or ``NaN`` + in object arrays). + + Parameters + ---------- + obj : scalar or array-like + Object to check for null or missing values. + + Returns + ------- + bool or array-like of bool + For scalar input, returns a scalar boolean. + For array input, returns an array of boolean indicating whether each + corresponding element is missing. + + See Also + -------- + Series.isna : Detect missing values in a Series. + Series.isnull : Detect missing values in a Series. + DataFrame.isna : Detect missing values in a DataFrame. + DataFrame.isnull : Detect missing values in a DataFrame. + Index.isna : Detect missing values in an Index. + Index.isnull : Detect missing values in an Index. + + Examples + -------- + Scalar arguments (including strings) result in a scalar boolean. + + >>> pp.isna('dog') + False + + >>> pp.isna(np.nan) + True + + ndarrays result in an ndarray of booleans. + + >>> array = np.array([[1, np.nan, 3], [4, 5, np.nan]]) + >>> array + array([[ 1., nan, 3.], + [ 4., 5., nan]]) + >>> pp.isna(array) + array([[False, True, False], + [False, False, True]]) + + For Series and DataFrame, the same type is returned, containing booleans. + + >>> df = pp.DataFrame({'a': ['ant', 'bee', 'cat'], 'b': ['dog', None, 'fly']}) + >>> df + a b + 0 ant dog + 1 bee None + 2 cat fly + + >>> pp.isna(df) + a b + 0 False False + 1 False True + 2 False False + + >>> pp.isnull(df.b) + 0 False + 1 True + 2 False + Name: b, dtype: bool + """ + # TODO: Add back: + # notnull : Boolean inverse of pandas.isnull. + # into the See Also in the docstring. It does not find the method in the latest numpydoc. + if isinstance(obj, (DataFrame, Series)): + return obj.isnull() + else: + return pd.isnull(obj) + + +isnull = isna + + +def notna(obj): + """ + Detect existing (non-missing) values. + + Return a boolean same-sized object indicating if the values are not NA. + Non-missing values get mapped to True. NA values, such as None or + :attr:`numpy.NaN`, get mapped to False values. + + Returns + ------- + bool or array-like of bool + Mask of bool values for each element that + indicates whether an element is not an NA value. + + See Also + -------- + isna : Detect missing values for an array-like object. + Series.notna : Boolean inverse of Series.isna. + DataFrame.notnull : Boolean inverse of DataFrame.isnull. + Index.notna : Boolean inverse of Index.isna. + Index.notnull : Boolean inverse of Index.isnull. + + Examples + -------- + Show which entries in a DataFrame are not NA. + + >>> df = pp.DataFrame({'age': [5, 6, np.NaN], + ... 'born': [pd.NaT, pd.Timestamp('1939-05-27'), + ... pd.Timestamp('1940-04-25')], + ... 'name': ['Alfred', 'Batman', ''], + ... 'toy': [None, 'Batmobile', 'Joker']}) + >>> df + age born name toy + 0 5.0 NaT Alfred None + 1 6.0 1939-05-27 Batman Batmobile + 2 NaN 1940-04-25 Joker + + >>> df.notnull() + age born name toy + 0 True False True False + 1 True True True True + 2 False True True True + + Show which entries in a Series are not NA. + + >>> ser = pp.Series([5, 6, np.NaN]) + >>> ser + 0 5.0 + 1 6.0 + 2 NaN + dtype: float64 + + >>> pp.notna(ser) + 0 True + 1 True + 2 False + dtype: bool + + >>> pp.notna(ser.index) + True + """ + # TODO: Add back: + # Series.notnull :Boolean inverse of Series.isnull. + # DataFrame.notna :Boolean inverse of DataFrame.isna. + # into the See Also in the docstring. It does not find the method in the latest numpydoc. + if isinstance(obj, (DataFrame, Series)): + return obj.notna() + else: + return pd.notna(obj) + + +notnull = notna + + +def merge( + obj, + right: "DataFrame", + how: str = "inner", + on: Union[Any, List[Any], Tuple, List[Tuple]] = None, + left_on: Union[Any, List[Any], Tuple, List[Tuple]] = None, + right_on: Union[Any, List[Any], Tuple, List[Tuple]] = None, + left_index: bool = False, + right_index: bool = False, + suffixes: Tuple[str, str] = ("_x", "_y"), +) -> "DataFrame": + """ + Merge DataFrame objects with a database-style join. + + The index of the resulting DataFrame will be one of the following: + - 0...n if no index is used for merging + - Index of the left DataFrame if merged only on the index of the right DataFrame + - Index of the right DataFrame if merged only on the index of the left DataFrame + - All involved indices if merged using the indices of both DataFrames + e.g. if `left` with indices (a, x) and `right` with indices (b, x), the result will + be an index (x, a, b) + + Parameters + ---------- + right: Object to merge with. + how: Type of merge to be performed. + {'left', 'right', 'outer', 'inner'}, default 'inner' + + left: use only keys from left frame, similar to a SQL left outer join; preserve key + order. + right: use only keys from right frame, similar to a SQL right outer join; preserve key + order. + outer: use union of keys from both frames, similar to a SQL full outer join; sort keys + lexicographically. + inner: use intersection of keys from both frames, similar to a SQL inner join; + preserve the order of the left keys. + on: Column or index level names to join on. These must be found in both DataFrames. If on + is None and not merging on indexes then this defaults to the intersection of the + columns in both DataFrames. + left_on: Column or index level names to join on in the left DataFrame. Can also + be an array or list of arrays of the length of the left DataFrame. + These arrays are treated as if they are columns. + right_on: Column or index level names to join on in the right DataFrame. Can also + be an array or list of arrays of the length of the right DataFrame. + These arrays are treated as if they are columns. + left_index: Use the index from the left DataFrame as the join key(s). If it is a + MultiIndex, the number of keys in the other DataFrame (either the index or a number of + columns) must match the number of levels. + right_index: Use the index from the right DataFrame as the join key. Same caveats as + left_index. + suffixes: Suffix to apply to overlapping column names in the left and right side, + respectively. + + Returns + ------- + DataFrame + A DataFrame of the two merged objects. + + Examples + -------- + + >>> df1 = pp.DataFrame({'lkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [1, 2, 3, 5]}, + ... columns=['lkey', 'value']) + >>> df2 = pp.DataFrame({'rkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [5, 6, 7, 8]}, + ... columns=['rkey', 'value']) + >>> df1 + lkey value + 0 foo 1 + 1 bar 2 + 2 baz 3 + 3 foo 5 + >>> df2 + rkey value + 0 foo 5 + 1 bar 6 + 2 baz 7 + 3 foo 8 + + Merge df1 and df2 on the lkey and rkey columns. The value columns have + the default suffixes, _x and _y, appended. + + >>> merged = pp.merge(df1, df2, left_on='lkey', right_on='rkey') + >>> merged.sort_values(by=['lkey', 'value_x', 'rkey', 'value_y']) # doctest: +ELLIPSIS + lkey value_x rkey value_y + ...bar 2 bar 6 + ...baz 3 baz 7 + ...foo 1 foo 5 + ...foo 1 foo 8 + ...foo 5 foo 5 + ...foo 5 foo 8 + + >>> left_kdf = pp.DataFrame({'A': [1, 2]}) + >>> right_kdf = pp.DataFrame({'B': ['x', 'y']}, index=[1, 2]) + + >>> pp.merge(left_kdf, right_kdf, left_index=True, right_index=True).sort_index() + A B + 1 2 x + + >>> pp.merge(left_kdf, right_kdf, left_index=True, right_index=True, how='left').sort_index() + A B + 0 1 None + 1 2 x + + >>> pp.merge(left_kdf, right_kdf, left_index=True, right_index=True, how='right').sort_index() + A B + 1 2.0 x + 2 NaN y + + >>> pp.merge(left_kdf, right_kdf, left_index=True, right_index=True, how='outer').sort_index() + A B + 0 1.0 None + 1 2.0 x + 2 NaN y + + Notes + ----- + As described in #263, joining string columns currently returns None for missing values + instead of NaN. + """ + return obj.merge( + right, + how=how, + on=on, + left_on=left_on, + right_on=right_on, + left_index=left_index, + right_index=right_index, + suffixes=suffixes, + ) + + +def to_numeric(arg): + """ + Convert argument to a numeric type. + + Parameters + ---------- + arg : scalar, list, tuple, 1-d array, or Series + + Returns + ------- + ret : numeric if parsing succeeded. + + See Also + -------- + DataFrame.astype : Cast argument to a specified dtype. + to_datetime : Convert argument to datetime. + to_timedelta : Convert argument to timedelta. + numpy.ndarray.astype : Cast a numpy array to a specified type. + + Examples + -------- + + >>> kser = pp.Series(['1.0', '2', '-3']) + >>> kser + 0 1.0 + 1 2 + 2 -3 + dtype: object + + >>> pp.to_numeric(kser) + 0 1.0 + 1 2.0 + 2 -3.0 + dtype: float32 + + If given Series contains invalid value to cast float, just cast it to `np.nan` + + >>> kser = pp.Series(['apple', '1.0', '2', '-3']) + >>> kser + 0 apple + 1 1.0 + 2 2 + 3 -3 + dtype: object + + >>> pp.to_numeric(kser) + 0 NaN + 1 1.0 + 2 2.0 + 3 -3.0 + dtype: float32 + + Also support for list, tuple, np.array, or a scalar + + >>> pp.to_numeric(['1.0', '2', '-3']) + array([ 1., 2., -3.]) + + >>> pp.to_numeric(('1.0', '2', '-3')) + array([ 1., 2., -3.]) + + >>> pp.to_numeric(np.array(['1.0', '2', '-3'])) + array([ 1., 2., -3.]) + + >>> pp.to_numeric('1.0') + 1.0 + """ + if isinstance(arg, Series): + return arg._with_new_scol(arg.spark.column.cast("float")) + else: + return pd.to_numeric(arg) + + +def broadcast(obj) -> DataFrame: + """ + Marks a DataFrame as small enough for use in broadcast joins. + + Parameters + ---------- + obj : DataFrame + + Returns + ------- + ret : DataFrame with broadcast hint. + + See Also + -------- + DataFrame.merge : Merge DataFrame objects with a database-style join. + DataFrame.join : Join columns of another DataFrame. + DataFrame.update : Modify in place using non-NA values from another DataFrame. + DataFrame.hint : Specifies some hint on the current DataFrame. + + Examples + -------- + >>> df1 = pp.DataFrame({'lkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [1, 2, 3, 5]}, + ... columns=['lkey', 'value']).set_index('lkey') + >>> df2 = pp.DataFrame({'rkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [5, 6, 7, 8]}, + ... columns=['rkey', 'value']).set_index('rkey') + >>> merged = df1.merge(pp.broadcast(df2), left_index=True, right_index=True) + >>> merged.spark.explain() # doctest: +ELLIPSIS + == Physical Plan == + ... + ...BroadcastHashJoin... + ... + """ + if not isinstance(obj, DataFrame): + raise ValueError("Invalid type : expected DataFrame got {}".format(type(obj).__name__)) + return DataFrame( + obj._internal.with_new_sdf(F.broadcast(obj._internal.resolved_copy.spark_frame)) + ) + + +def read_orc( + path, + columns: Optional[List[str]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options +) -> "DataFrame": + """ + Load an ORC object from the file path, returning a DataFrame. + + Parameters + ---------- + path : str + The path string storing the ORC file to be read. + columns : list, default None + If not None, only these columns will be read from the file. + index_col : str or list of str, optional, default: None + Index column of table in Spark. + options : dict + All other options passed directly into Spark's data source. + + Returns + ------- + DataFrame + + Examples + -------- + >>> pp.range(1).to_orc('%s/read_spark_io/data.orc' % path) + >>> pp.read_orc('%s/read_spark_io/data.orc' % path, columns=['id']) + id + 0 0 + + You can preserve the index in the roundtrip as below. + + >>> pp.range(1).to_orc('%s/read_spark_io/data.orc' % path, index_col="index") + >>> pp.read_orc('%s/read_spark_io/data.orc' % path, columns=['id'], index_col="index") + ... # doctest: +NORMALIZE_WHITESPACE + id + index + 0 0 + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + kdf = read_spark_io(path, format="orc", index_col=index_col, **options) + + if columns is not None: + kdf_columns = kdf.columns + new_columns = list() + for column in list(columns): + if column in kdf_columns: + new_columns.append(column) + else: + raise ValueError("Unknown column name '{}'".format(column)) + kdf = kdf[new_columns] + + return kdf + + +def _get_index_map( + sdf: spark.DataFrame, index_col: Optional[Union[str, List[str]]] = None +) -> Tuple[Optional[List[spark.Column]], Optional[List[Tuple]]]: + if index_col is not None: + if isinstance(index_col, str): + index_col = [index_col] + sdf_columns = set(sdf.columns) + for col in index_col: + if col not in sdf_columns: + raise KeyError(col) + index_spark_columns = [ + scol_for(sdf, col) for col in index_col + ] # type: Optional[List[spark.Column]] + index_names = [(col,) for col in index_col] # type: Optional[List[Tuple]] + else: + index_spark_columns = None + index_names = None + + return index_spark_columns, index_names + + +_get_dummies_default_accept_types = (DecimalType, StringType, DateType) +_get_dummies_acceptable_types = _get_dummies_default_accept_types + ( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + BooleanType, + TimestampType, +) diff --git a/python/pyspark/pandas/numpy_compat.py b/python/pyspark/pandas/numpy_compat.py new file mode 100644 index 0000000000..3ab4073a77 --- /dev/null +++ b/python/pyspark/pandas/numpy_compat.py @@ -0,0 +1,210 @@ +# +# 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. +# +from collections import OrderedDict +from typing import Callable, Any + +import numpy as np +from pyspark.sql import functions as F, Column +from pyspark.sql.types import DoubleType, LongType, BooleanType + + +unary_np_spark_mappings = OrderedDict( + { + "abs": F.abs, + "absolute": F.abs, + "arccos": F.acos, + "arccosh": F.pandas_udf(lambda s: np.arccosh(s), DoubleType()), + "arcsin": F.asin, + "arcsinh": F.pandas_udf(lambda s: np.arcsinh(s), DoubleType()), + "arctan": F.atan, + "arctanh": F.pandas_udf(lambda s: np.arctanh(s), DoubleType()), + "bitwise_not": F.bitwiseNOT, + "cbrt": F.cbrt, + "ceil": F.ceil, + # It requires complex type which Koalas does not support yet + "conj": lambda _: NotImplemented, + "conjugate": lambda _: NotImplemented, # It requires complex type + "cos": F.cos, + "cosh": F.pandas_udf(lambda s: np.cosh(s), DoubleType()), + "deg2rad": F.pandas_udf(lambda s: np.deg2rad(s), DoubleType()), + "degrees": F.degrees, + "exp": F.exp, + "exp2": F.pandas_udf(lambda s: np.exp2(s), DoubleType()), + "expm1": F.expm1, + "fabs": F.pandas_udf(lambda s: np.fabs(s), DoubleType()), + "floor": F.floor, + "frexp": lambda _: NotImplemented, # 'frexp' output lengths become different + # and it cannot be supported via pandas UDF. + "invert": F.pandas_udf(lambda s: np.invert(s), DoubleType()), + "isfinite": lambda c: c != float("inf"), + "isinf": lambda c: c == float("inf"), + "isnan": F.isnan, + "isnat": lambda c: NotImplemented, # Koalas and PySpark does not have Nat concept. + "log": F.log, + "log10": F.log10, + "log1p": F.log1p, + "log2": F.pandas_udf(lambda s: np.log2(s), DoubleType()), + "logical_not": lambda c: ~(c.cast(BooleanType())), + "matmul": lambda _: NotImplemented, # Can return a NumPy array in pandas. + "negative": lambda c: c * -1, + "positive": lambda c: c, + "rad2deg": F.pandas_udf(lambda s: np.rad2deg(s), DoubleType()), + "radians": F.radians, + "reciprocal": F.pandas_udf(lambda s: np.reciprocal(s), DoubleType()), + "rint": F.pandas_udf(lambda s: np.rint(s), DoubleType()), + "sign": lambda c: F.when(c == 0, 0).when(c < 0, -1).otherwise(1), + "signbit": lambda c: F.when(c < 0, True).otherwise(False), + "sin": F.sin, + "sinh": F.pandas_udf(lambda s: np.sinh(s), DoubleType()), + "spacing": F.pandas_udf(lambda s: np.spacing(s), DoubleType()), + "sqrt": F.sqrt, + "square": F.pandas_udf(lambda s: np.square(s), DoubleType()), + "tan": F.tan, + "tanh": F.pandas_udf(lambda s: np.tanh(s), DoubleType()), + "trunc": F.pandas_udf(lambda s: np.trunc(s), DoubleType()), + } +) + +binary_np_spark_mappings = OrderedDict( + { + "arctan2": F.atan2, + "bitwise_and": lambda c1, c2: c1.bitwiseAND(c2), + "bitwise_or": lambda c1, c2: c1.bitwiseOR(c2), + "bitwise_xor": lambda c1, c2: c1.bitwiseXOR(c2), + "copysign": F.pandas_udf(lambda s1, s2: np.copysign(s1, s2), DoubleType()), + "float_power": F.pandas_udf(lambda s1, s2: np.float_power(s1, s2), DoubleType()), + "floor_divide": F.pandas_udf(lambda s1, s2: np.floor_divide(s1, s2), DoubleType()), + "fmax": F.pandas_udf(lambda s1, s2: np.fmax(s1, s2), DoubleType()), + "fmin": F.pandas_udf(lambda s1, s2: np.fmin(s1, s2), DoubleType()), + "fmod": F.pandas_udf(lambda s1, s2: np.fmod(s1, s2), DoubleType()), + "gcd": F.pandas_udf(lambda s1, s2: np.gcd(s1, s2), DoubleType()), + "heaviside": F.pandas_udf(lambda s1, s2: np.heaviside(s1, s2), DoubleType()), + "hypot": F.hypot, + "lcm": F.pandas_udf(lambda s1, s2: np.lcm(s1, s2), DoubleType()), + "ldexp": F.pandas_udf(lambda s1, s2: np.ldexp(s1, s2), DoubleType()), + "left_shift": F.pandas_udf(lambda s1, s2: np.left_shift(s1, s2), LongType()), + "logaddexp": F.pandas_udf(lambda s1, s2: np.logaddexp(s1, s2), DoubleType()), + "logaddexp2": F.pandas_udf(lambda s1, s2: np.logaddexp2(s1, s2), DoubleType()), + "logical_and": lambda c1, c2: c1.cast(BooleanType()) & c2.cast(BooleanType()), + "logical_or": lambda c1, c2: c1.cast(BooleanType()) | c2.cast(BooleanType()), + "logical_xor": lambda c1, c2: ( + # mimics xor by logical operators. + (c1.cast(BooleanType()) | c2.cast(BooleanType())) + & (~(c1.cast(BooleanType())) | ~(c2.cast(BooleanType()))) + ), + "maximum": F.greatest, + "minimum": F.least, + "modf": F.pandas_udf(lambda s1, s2: np.modf(s1, s2), DoubleType()), + "nextafter": F.pandas_udf(lambda s1, s2: np.nextafter(s1, s2), DoubleType()), + "right_shift": F.pandas_udf(lambda s1, s2: np.right_shift(s1, s2), LongType()), + } +) + + +# Copied from pandas. +# See also https://docs.scipy.org/doc/numpy/reference/arrays.classes.html#standard-array-subclasses +def maybe_dispatch_ufunc_to_dunder_op( + ser_or_index, ufunc: Callable, method: str, *inputs, **kwargs: Any +): + special = { + "add", + "sub", + "mul", + "pow", + "mod", + "floordiv", + "truediv", + "divmod", + "eq", + "ne", + "lt", + "gt", + "le", + "ge", + "remainder", + "matmul", + } + aliases = { + "absolute": "abs", + "multiply": "mul", + "floor_divide": "floordiv", + "true_divide": "truediv", + "power": "pow", + "remainder": "mod", + "divide": "div", + "equal": "eq", + "not_equal": "ne", + "less": "lt", + "less_equal": "le", + "greater": "gt", + "greater_equal": "ge", + } + + # For op(., Array) -> Array.__r{op}__ + flipped = { + "lt": "__gt__", + "le": "__ge__", + "gt": "__lt__", + "ge": "__le__", + "eq": "__eq__", + "ne": "__ne__", + } + + op_name = ufunc.__name__ + op_name = aliases.get(op_name, op_name) + + def not_implemented(*args, **kwargs): + return NotImplemented + + if method == "__call__" and op_name in special and kwargs.get("out") is None: + if isinstance(inputs[0], type(ser_or_index)): + name = "__{}__".format(op_name) + return getattr(ser_or_index, name, not_implemented)(inputs[1]) + else: + name = flipped.get(op_name, "__r{}__".format(op_name)) + return getattr(ser_or_index, name, not_implemented)(inputs[0]) + else: + return NotImplemented + + +# See also https://docs.scipy.org/doc/numpy/reference/arrays.classes.html#standard-array-subclasses +def maybe_dispatch_ufunc_to_spark_func( + ser_or_index, ufunc: Callable, method: str, *inputs, **kwargs: Any +): + from pyspark.pandas.base import column_op + + op_name = ufunc.__name__ + + if ( + method == "__call__" + and (op_name in unary_np_spark_mappings or op_name in binary_np_spark_mappings) + and kwargs.get("out") is None + ): + + np_spark_map_func = unary_np_spark_mappings.get(op_name) or binary_np_spark_mappings.get( + op_name + ) + + def convert_arguments(*args): + args = [ # type: ignore + F.lit(inp) if not isinstance(inp, Column) else inp for inp in args + ] # type: ignore + return np_spark_map_func(*args) + + return column_op(convert_arguments)(*inputs) # type: ignore + else: + return NotImplemented diff --git a/python/pyspark/pandas/plot/__init__.py b/python/pyspark/pandas/plot/__init__.py new file mode 100644 index 0000000000..8b3376e7b2 --- /dev/null +++ b/python/pyspark/pandas/plot/__init__.py @@ -0,0 +1,17 @@ +# +# 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. +# +from pyspark.pandas.plot.core import * # noqa: F401 diff --git a/python/pyspark/pandas/plot/core.py b/python/pyspark/pandas/plot/core.py new file mode 100644 index 0000000000..a16e2d2d16 --- /dev/null +++ b/python/pyspark/pandas/plot/core.py @@ -0,0 +1,1134 @@ +# +# 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 importlib + +import pandas as pd +import numpy as np +from pyspark.ml.feature import Bucketizer +from pyspark.mllib.stat import KernelDensity +from pyspark.sql import functions as F +from pandas.core.base import PandasObject +from pandas.core.dtypes.inference import is_integer + +from pyspark.pandas.missing import unsupported_function +from pyspark.pandas.config import get_option +from pyspark.pandas.utils import name_like_string + + +class TopNPlotBase: + def get_top_n(self, data): + from pyspark.pandas import DataFrame, Series + + max_rows = get_option("plotting.max_rows") + # Simply use the first 1k elements and make it into a pandas dataframe + # For categorical variables, it is likely called from df.x.value_counts().plot.xxx(). + if isinstance(data, (Series, DataFrame)): + data = data.head(max_rows + 1).to_pandas() + else: + raise ValueError("Only DataFrame and Series are supported for plotting.") + + self.partial = False + if len(data) > max_rows: + self.partial = True + data = data.iloc[:max_rows] + return data + + def set_result_text(self, ax): + max_rows = get_option("plotting.max_rows") + assert hasattr(self, "partial") + + if self.partial: + ax.text( + 1, + 1, + "showing top {} elements only".format(max_rows), + size=6, + ha="right", + va="bottom", + transform=ax.transAxes, + ) + + +class SampledPlotBase: + def get_sampled(self, data): + from pyspark.pandas import DataFrame, Series + + fraction = get_option("plotting.sample_ratio") + if fraction is None: + fraction = 1 / (len(data) / get_option("plotting.max_rows")) + fraction = min(1.0, fraction) + self.fraction = fraction + + if isinstance(data, (DataFrame, Series)): + if isinstance(data, Series): + data = data.to_frame() + sampled = data._internal.resolved_copy.spark_frame.sample(fraction=self.fraction) + return DataFrame(data._internal.with_new_sdf(sampled)).to_pandas() + else: + raise ValueError("Only DataFrame and Series are supported for plotting.") + + def set_result_text(self, ax): + assert hasattr(self, "fraction") + + if self.fraction < 1: + ax.text( + 1, + 1, + "showing the sampled result by fraction %s" % self.fraction, + size=6, + ha="right", + va="bottom", + transform=ax.transAxes, + ) + + +class HistogramPlotBase: + @staticmethod + def prepare_hist_data(data, bins): + # TODO: this logic is similar with KdePlotBase. Might have to deduplicate it. + from pyspark.pandas.series import Series + + if isinstance(data, Series): + data = data.to_frame() + + numeric_data = data.select_dtypes( + include=["byte", "decimal", "integer", "float", "long", "double", np.datetime64] + ) + + # no empty frames or series allowed + if len(numeric_data.columns) == 0: + raise TypeError( + "Empty {0!r}: no numeric data to " "plot".format(numeric_data.__class__.__name__) + ) + + if is_integer(bins): + # computes boundaries for the column + bins = HistogramPlotBase.get_bins(data.to_spark(), bins) + + return numeric_data, bins + + @staticmethod + def get_bins(sdf, bins): + # 'data' is a Spark DataFrame that selects all columns. + if len(sdf.columns) > 1: + min_col = F.least(*map(F.min, sdf)) + max_col = F.greatest(*map(F.max, sdf)) + else: + min_col = F.min(sdf.columns[-1]) + max_col = F.max(sdf.columns[-1]) + boundaries = sdf.select(min_col, max_col).first() + + # divides the boundaries into bins + if boundaries[0] == boundaries[1]: + boundaries = (boundaries[0] - 0.5, boundaries[1] + 0.5) + + return np.linspace(boundaries[0], boundaries[1], bins + 1) + + @staticmethod + def compute_hist(kdf, bins): + # 'data' is a Spark DataFrame that selects one column. + assert isinstance(bins, (np.ndarray, np.generic)) + + sdf = kdf._internal.spark_frame + scols = [] + input_column_names = [] + for label in kdf._internal.column_labels: + input_column_name = name_like_string(label) + input_column_names.append(input_column_name) + scols.append(kdf._internal.spark_column_for(label).alias(input_column_name)) + sdf = sdf.select(*scols) + + # 1. Make the bucket output flat to: + # +----------+-------+ + # |__group_id|buckets| + # +----------+-------+ + # |0 |0.0 | + # |0 |0.0 | + # |0 |1.0 | + # |0 |2.0 | + # |0 |3.0 | + # |0 |3.0 | + # |1 |0.0 | + # |1 |1.0 | + # |1 |1.0 | + # |1 |2.0 | + # |1 |1.0 | + # |1 |0.0 | + # +----------+-------+ + colnames = sdf.columns + bucket_names = ["__{}_bucket".format(colname) for colname in colnames] + + output_df = None + for group_id, (colname, bucket_name) in enumerate(zip(colnames, bucket_names)): + # creates a Bucketizer to get corresponding bin of each value + bucketizer = Bucketizer( + splits=bins, inputCol=colname, outputCol=bucket_name, handleInvalid="skip" + ) + + bucket_df = bucketizer.transform(sdf) + + if output_df is None: + output_df = bucket_df.select( + F.lit(group_id).alias("__group_id"), F.col(bucket_name).alias("__bucket") + ) + else: + output_df = output_df.union( + bucket_df.select( + F.lit(group_id).alias("__group_id"), F.col(bucket_name).alias("__bucket") + ) + ) + + # 2. Calculate the count based on each group and bucket. + # +----------+-------+------+ + # |__group_id|buckets| count| + # +----------+-------+------+ + # |0 |0.0 |2 | + # |0 |1.0 |1 | + # |0 |2.0 |1 | + # |0 |3.0 |2 | + # |1 |0.0 |2 | + # |1 |1.0 |3 | + # |1 |2.0 |1 | + # +----------+-------+------+ + result = ( + output_df.groupby("__group_id", "__bucket") + .agg(F.count("*").alias("count")) + .toPandas() + .sort_values(by=["__group_id", "__bucket"]) + ) + + # 3. Fill empty bins and calculate based on each group id. From: + # +----------+--------+------+ + # |__group_id|__bucket| count| + # +----------+--------+------+ + # |0 |0.0 |2 | + # |0 |1.0 |1 | + # |0 |2.0 |1 | + # |0 |3.0 |2 | + # +----------+--------+------+ + # +----------+--------+------+ + # |__group_id|__bucket| count| + # +----------+--------+------+ + # |1 |0.0 |2 | + # |1 |1.0 |3 | + # |1 |2.0 |1 | + # +----------+--------+------+ + # + # to: + # +-----------------+ + # |__values1__bucket| + # +-----------------+ + # |2 | + # |1 | + # |1 | + # |2 | + # |0 | + # +-----------------+ + # +-----------------+ + # |__values2__bucket| + # +-----------------+ + # |2 | + # |3 | + # |1 | + # |0 | + # |0 | + # +-----------------+ + output_series = [] + for i, (input_column_name, bucket_name) in enumerate(zip(input_column_names, bucket_names)): + current_bucket_result = result[result["__group_id"] == i] + # generates a pandas DF with one row for each bin + # we need this as some of the bins may be empty + indexes = pd.DataFrame({"__bucket": np.arange(0, len(bins) - 1)}) + # merges the bins with counts on it and fills remaining ones with zeros + pdf = indexes.merge(current_bucket_result, how="left", on=["__bucket"]).fillna(0)[ + ["count"] + ] + pdf.columns = [input_column_name] + output_series.append(pdf[input_column_name]) + + return output_series + + +class BoxPlotBase: + @staticmethod + def compute_stats(data, colname, whis, precision): + # Computes mean, median, Q1 and Q3 with approx_percentile and precision + pdf = data._kdf._internal.resolved_copy.spark_frame.agg( + *[ + F.expr( + "approx_percentile(`{}`, {}, {})".format(colname, q, int(1.0 / precision)) + ).alias("{}_{}%".format(colname, int(q * 100))) + for q in [0.25, 0.50, 0.75] + ], + F.mean("`%s`" % colname).alias("{}_mean".format(colname)), + ).toPandas() + + # Computes IQR and Tukey's fences + iqr = "{}_iqr".format(colname) + p75 = "{}_75%".format(colname) + p25 = "{}_25%".format(colname) + pdf.loc[:, iqr] = pdf.loc[:, p75] - pdf.loc[:, p25] + pdf.loc[:, "{}_lfence".format(colname)] = pdf.loc[:, p25] - whis * pdf.loc[:, iqr] + pdf.loc[:, "{}_ufence".format(colname)] = pdf.loc[:, p75] + whis * pdf.loc[:, iqr] + + qnames = ["25%", "50%", "75%", "mean", "lfence", "ufence"] + col_summ = pdf[["{}_{}".format(colname, q) for q in qnames]] + col_summ.columns = qnames + lfence, ufence = col_summ["lfence"], col_summ["ufence"] + + stats = { + "mean": col_summ["mean"].values[0], + "med": col_summ["50%"].values[0], + "q1": col_summ["25%"].values[0], + "q3": col_summ["75%"].values[0], + } + + return stats, (lfence.values[0], ufence.values[0]) + + @staticmethod + def outliers(data, colname, lfence, ufence): + # Builds expression to identify outliers + expression = F.col("`%s`" % colname).between(lfence, ufence) + # Creates a column to flag rows as outliers or not + return data._kdf._internal.resolved_copy.spark_frame.withColumn( + "__{}_outlier".format(colname), ~expression + ) + + @staticmethod + def calc_whiskers(colname, outliers): + # Computes min and max values of non-outliers - the whiskers + minmax = ( + outliers.filter("not `__{}_outlier`".format(colname)) + .agg(F.min("`%s`" % colname).alias("min"), F.max(colname).alias("max")) + .toPandas() + ) + return minmax.iloc[0][["min", "max"]].values + + @staticmethod + def get_fliers(colname, outliers, min_val): + # Filters only the outliers, should "showfliers" be True + fliers_df = outliers.filter("`__{}_outlier`".format(colname)) + + # If shows fliers, takes the top 1k with highest absolute values + # Here we normalize the values by subtracting the minimum value from + # each, and use absolute values. + order_col = F.abs(F.col("`{}`".format(colname)) - min_val.item()) + fliers = ( + fliers_df.select(F.col("`{}`".format(colname))) + .orderBy(order_col) + .limit(1001) + .toPandas()[colname] + .values + ) + + return fliers + + +class KdePlotBase: + @staticmethod + def prepare_kde_data(data): + # TODO: this logic is similar with HistogramPlotBase. Might have to deduplicate it. + from pyspark.pandas.series import Series + + if isinstance(data, Series): + data = data.to_frame() + + numeric_data = data.select_dtypes( + include=["byte", "decimal", "integer", "float", "long", "double", np.datetime64] + ) + + # no empty frames or series allowed + if len(numeric_data.columns) == 0: + raise TypeError( + "Empty {0!r}: no numeric data to " "plot".format(numeric_data.__class__.__name__) + ) + + return numeric_data + + @staticmethod + def get_ind(sdf, ind): + def calc_min_max(): + if len(sdf.columns) > 1: + min_col = F.least(*map(F.min, sdf)) + max_col = F.greatest(*map(F.max, sdf)) + else: + min_col = F.min(sdf.columns[-1]) + max_col = F.max(sdf.columns[-1]) + return sdf.select(min_col, max_col).first() + + if ind is None: + min_val, max_val = calc_min_max() + sample_range = max_val - min_val + ind = np.linspace(min_val - 0.5 * sample_range, max_val + 0.5 * sample_range, 1000,) + elif is_integer(ind): + min_val, max_val = calc_min_max() + sample_range = max_val - min_val + ind = np.linspace(min_val - 0.5 * sample_range, max_val + 0.5 * sample_range, ind,) + return ind + + @staticmethod + def compute_kde(sdf, bw_method=None, ind=None): + # 'sdf' is a Spark DataFrame that selects one column. + + # Using RDD is slow so we might have to change it to Dataset based implementation + # once Spark has that implementation. + sample = sdf.rdd.map(lambda x: float(x[0])) + kd = KernelDensity() + kd.setSample(sample) + + assert isinstance(bw_method, (int, float)), "'bw_method' must be set as a scalar number." + + if bw_method is not None: + # Match the bandwidth with Spark. + kd.setBandwidth(float(bw_method)) + return kd.estimate(list(map(float, ind))) + + +class KoalasPlotAccessor(PandasObject): + """ + Series/Frames plotting accessor and method. + + Uses the backend specified by the + option ``plotting.backend``. By default, plotly is used. + + Plotting methods can also be accessed by calling the accessor as a method + with the ``kind`` argument: + ``s.plot(kind='hist')`` is equivalent to ``s.plot.hist()`` + """ + + pandas_plot_data_map = { + "pie": TopNPlotBase().get_top_n, + "bar": TopNPlotBase().get_top_n, + "barh": TopNPlotBase().get_top_n, + "scatter": TopNPlotBase().get_top_n, + "area": SampledPlotBase().get_sampled, + "line": SampledPlotBase().get_sampled, + } + _backends = {} # type: ignore + + def __init__(self, data): + self.data = data + + @staticmethod + def _find_backend(backend): + """ + Find a Koalas plotting backend + """ + try: + return KoalasPlotAccessor._backends[backend] + except KeyError: + try: + module = importlib.import_module(backend) + except ImportError: + # We re-raise later on. + pass + else: + if hasattr(module, "plot") or hasattr(module, "plot_koalas"): + # Validate that the interface is implemented when the option + # is set, rather than at plot time. + KoalasPlotAccessor._backends[backend] = module + return module + + raise ValueError( + "Could not find plotting backend '{backend}'. Ensure that you've installed " + "the package providing the '{backend}' entrypoint, or that the package has a " + "top-level `.plot` method.".format(backend=backend) + ) + + @staticmethod + def _get_plot_backend(backend=None): + backend = backend or get_option("plotting.backend") + # Shortcut + if backend in KoalasPlotAccessor._backends: + return KoalasPlotAccessor._backends[backend] + + if backend == "matplotlib": + # Because matplotlib is an optional dependency, + # we need to attempt an import here to raise an ImportError if needed. + try: + # test if matplotlib can be imported + import matplotlib # noqa: F401 + from pyspark.pandas.plot import matplotlib as module + except ImportError: + raise ImportError( + "matplotlib is required for plotting when the " + "default backend 'matplotlib' is selected." + ) from None + + KoalasPlotAccessor._backends["matplotlib"] = module + elif backend == "plotly": + try: + # test if plotly can be imported + import plotly # noqa: F401 + from pyspark.pandas.plot import plotly as module + except ImportError: + raise ImportError( + "plotly is required for plotting when the " + "default backend 'plotly' is selected." + ) from None + + KoalasPlotAccessor._backends["plotly"] = module + else: + module = KoalasPlotAccessor._find_backend(backend) + KoalasPlotAccessor._backends[backend] = module + return module + + def __call__(self, kind="line", backend=None, **kwargs): + plot_backend = KoalasPlotAccessor._get_plot_backend(backend) + plot_data = self.data + + kind = {"density": "kde"}.get(kind, kind) + if hasattr(plot_backend, "plot_koalas"): + # use if there's koalas specific method. + return plot_backend.plot_koalas(plot_data, kind=kind, **kwargs) + else: + # fallback to use pandas' + if not KoalasPlotAccessor.pandas_plot_data_map[kind]: + raise NotImplementedError( + "'%s' plot is not supported with '%s' plot " + "backend yet." % (kind, plot_backend.__name__) + ) + plot_data = KoalasPlotAccessor.pandas_plot_data_map[kind](plot_data) + return plot_backend.plot(plot_data, kind=kind, **kwargs) + + def line(self, x=None, y=None, **kwargs): + """ + Plot DataFrame/Series as lines. + + This function is useful to plot lines using Series's values + as coordinates. + + Parameters + ---------- + x : int or str, optional + Columns to use for the horizontal axis. + Either the location or the label of the columns to be used. + By default, it will use the DataFrame indices. + y : int, str, or list of them, optional + The values to be plotted. + Either the location or the label of the columns to be used. + By default, it will use the remaining DataFrame numeric columns. + **kwds + Keyword arguments to pass on to :meth:`Series.plot` or :meth:`DataFrame.plot`. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + See Also + -------- + plotly.express.line : Plot y versus x as lines and/or markers (plotly). + matplotlib.pyplot.plot : Plot y versus x as lines and/or markers (matplotlib). + + Examples + -------- + Basic plot. + + For Series: + + .. plotly:: + + >>> s = pp.Series([1, 3, 2]) + >>> s.plot.line() # doctest: +SKIP + + For DataFrame: + + .. plotly:: + + The following example shows the populations for some animals + over the years. + + >>> df = pp.DataFrame({'pig': [20, 18, 489, 675, 1776], + ... 'horse': [4, 25, 281, 600, 1900]}, + ... index=[1990, 1997, 2003, 2009, 2014]) + >>> df.plot.line() # doctest: +SKIP + + .. plotly:: + + The following example shows the relationship between both + populations. + + >>> df = pp.DataFrame({'pig': [20, 18, 489, 675, 1776], + ... 'horse': [4, 25, 281, 600, 1900]}, + ... index=[1990, 1997, 2003, 2009, 2014]) + >>> df.plot.line(x='pig', y='horse') # doctest: +SKIP + """ + return self(kind="line", x=x, y=y, **kwargs) + + def bar(self, x=None, y=None, **kwds): + """ + Vertical bar plot. + + Parameters + ---------- + x : label or position, optional + Allows plotting of one column versus another. + If not specified, the index of the DataFrame is used. + y : label or position, optional + Allows plotting of one column versus another. + If not specified, all numerical columns are used. + **kwds : optional + Additional keyword arguments are documented in + :meth:`Koalas.Series.plot` or :meth:`Koalas.DataFrame.plot`. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + Examples + -------- + Basic plot. + + For Series: + + .. plotly:: + + >>> s = pp.Series([1, 3, 2]) + >>> s.plot.bar() # doctest: +SKIP + + For DataFrame: + + .. plotly:: + + >>> df = pp.DataFrame({'lab': ['A', 'B', 'C'], 'val': [10, 30, 20]}) + >>> df.plot.bar(x='lab', y='val') # doctest: +SKIP + + Plot a whole dataframe to a bar plot. Each column is stacked with a + distinct color along the horizontal axis. + + .. plotly:: + + >>> speed = [0.1, 17.5, 40, 48, 52, 69, 88] + >>> lifespan = [2, 8, 70, 1.5, 25, 12, 28] + >>> index = ['snail', 'pig', 'elephant', + ... 'rabbit', 'giraffe', 'coyote', 'horse'] + >>> df = pp.DataFrame({'speed': speed, + ... 'lifespan': lifespan}, index=index) + >>> df.plot.bar() # doctest: +SKIP + + Instead of stacking, the figure can be split by column with plotly + APIs. + + .. plotly:: + + >>> from plotly.subplots import make_subplots + >>> speed = [0.1, 17.5, 40, 48, 52, 69, 88] + >>> lifespan = [2, 8, 70, 1.5, 25, 12, 28] + >>> index = ['snail', 'pig', 'elephant', + ... 'rabbit', 'giraffe', 'coyote', 'horse'] + >>> df = pp.DataFrame({'speed': speed, + ... 'lifespan': lifespan}, index=index) + >>> fig = (make_subplots(rows=2, cols=1) + ... .add_trace(df.plot.bar(y='speed').data[0], row=1, col=1) + ... .add_trace(df.plot.bar(y='speed').data[0], row=1, col=1) + ... .add_trace(df.plot.bar(y='lifespan').data[0], row=2, col=1)) + >>> fig # doctest: +SKIP + + Plot a single column. + + .. plotly:: + + >>> speed = [0.1, 17.5, 40, 48, 52, 69, 88] + >>> lifespan = [2, 8, 70, 1.5, 25, 12, 28] + >>> index = ['snail', 'pig', 'elephant', + ... 'rabbit', 'giraffe', 'coyote', 'horse'] + >>> df = pp.DataFrame({'speed': speed, + ... 'lifespan': lifespan}, index=index) + >>> df.plot.bar(y='speed') # doctest: +SKIP + + Plot only selected categories for the DataFrame. + + .. plotly:: + + >>> speed = [0.1, 17.5, 40, 48, 52, 69, 88] + >>> lifespan = [2, 8, 70, 1.5, 25, 12, 28] + >>> index = ['snail', 'pig', 'elephant', + ... 'rabbit', 'giraffe', 'coyote', 'horse'] + >>> df = pp.DataFrame({'speed': speed, + ... 'lifespan': lifespan}, index=index) + >>> df.plot.bar(x='lifespan') # doctest: +SKIP + """ + from pyspark.pandas import DataFrame, Series + + if isinstance(self.data, Series): + return self(kind="bar", **kwds) + elif isinstance(self.data, DataFrame): + return self(kind="bar", x=x, y=y, **kwds) + + def barh(self, x=None, y=None, **kwargs): + """ + Make a horizontal bar plot. + + A horizontal bar plot is a plot that presents quantitative data with + rectangular bars with lengths proportional to the values that they + represent. A bar plot shows comparisons among discrete categories. One + axis of the plot shows the specific categories being compared, and the + other axis represents a measured value. + + Parameters + ---------- + x : label or position, default DataFrame.index + Column to be used for categories. + y : label or position, default All numeric columns in dataframe + Columns to be plotted from the DataFrame. + **kwds + Keyword arguments to pass on to + :meth:`pyspark.pandas.DataFrame.plot` or :meth:`pyspark.pandas.Series.plot`. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + See Also + -------- + plotly.express.bar : Plot a vertical bar plot using plotly. + matplotlib.axes.Axes.bar : Plot a vertical bar plot using matplotlib. + + Examples + -------- + For Series: + + .. plotly:: + + >>> df = pp.DataFrame({'lab': ['A', 'B', 'C'], 'val': [10, 30, 20]}) + >>> df.val.plot.barh() # doctest: +SKIP + + For DataFrame: + + .. plotly:: + + >>> df = pp.DataFrame({'lab': ['A', 'B', 'C'], 'val': [10, 30, 20]}) + >>> df.plot.barh(x='lab', y='val') # doctest: +SKIP + + Plot a whole DataFrame to a horizontal bar plot + + .. plotly:: + + >>> speed = [0.1, 17.5, 40, 48, 52, 69, 88] + >>> lifespan = [2, 8, 70, 1.5, 25, 12, 28] + >>> index = ['snail', 'pig', 'elephant', + ... 'rabbit', 'giraffe', 'coyote', 'horse'] + >>> df = pp.DataFrame({'speed': speed, + ... 'lifespan': lifespan}, index=index) + >>> df.plot.barh() # doctest: +SKIP + + Plot a column of the DataFrame to a horizontal bar plot + + .. plotly:: + + >>> speed = [0.1, 17.5, 40, 48, 52, 69, 88] + >>> lifespan = [2, 8, 70, 1.5, 25, 12, 28] + >>> index = ['snail', 'pig', 'elephant', + ... 'rabbit', 'giraffe', 'coyote', 'horse'] + >>> df = pp.DataFrame({'speed': speed, + ... 'lifespan': lifespan}, index=index) + >>> df.plot.barh(y='speed') # doctest: +SKIP + + Plot DataFrame versus the desired column + + .. plotly:: + + >>> speed = [0.1, 17.5, 40, 48, 52, 69, 88] + >>> lifespan = [2, 8, 70, 1.5, 25, 12, 28] + >>> index = ['snail', 'pig', 'elephant', + ... 'rabbit', 'giraffe', 'coyote', 'horse'] + >>> df = pp.DataFrame({'speed': speed, + ... 'lifespan': lifespan}, index=index) + >>> df.plot.barh(x='lifespan') # doctest: +SKIP + """ + from pyspark.pandas import DataFrame, Series + + if isinstance(self.data, Series): + return self(kind="barh", **kwargs) + elif isinstance(self.data, DataFrame): + return self(kind="barh", x=x, y=y, **kwargs) + + def box(self, **kwds): + """ + Make a box plot of the Series columns. + + Parameters + ---------- + **kwds : optional + Additional keyword arguments are documented in + :meth:`Koalas.Series.plot`. + + precision: scalar, default = 0.01 + This argument is used by Koalas to compute approximate statistics + for building a boxplot. Use *smaller* values to get more precise + statistics (matplotlib-only). + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + Notes + ----- + There are behavior differences between Koalas and pandas. + + * Koalas computes approximate statistics - expect differences between + pandas and Koalas boxplots, especially regarding 1st and 3rd quartiles. + * The `whis` argument is only supported as a single number. + * Koalas doesn't support the following argument(s) (matplotlib-only). + + * `bootstrap` argument is not supported + * `autorange` argument is not supported + + Examples + -------- + Draw a box plot from a DataFrame with four columns of randomly + generated data. + + For Series: + + .. plotly:: + + >>> data = np.random.randn(25, 4) + >>> df = pp.DataFrame(data, columns=list('ABCD')) + >>> df['A'].plot.box() # doctest: +SKIP + + This is an unsupported function for DataFrame type + """ + from pyspark.pandas import DataFrame, Series + + if isinstance(self.data, Series): + return self(kind="box", **kwds) + elif isinstance(self.data, DataFrame): + return unsupported_function(class_name="pd.DataFrame", method_name="box")() + + def hist(self, bins=10, **kwds): + """ + Draw one histogram of the DataFrame’s columns. + A `histogram`_ is a representation of the distribution of data. + This function calls :meth:`plotting.backend.plot`, + on each series in the DataFrame, resulting in one histogram per column. + + .. _histogram: https://en.wikipedia.org/wiki/Histogram + + Parameters + ---------- + bins : integer or sequence, default 10 + Number of histogram bins to be used. If an integer is given, bins + 1 + bin edges are calculated and returned. If bins is a sequence, gives + bin edges, including left edge of first bin and right edge of last + bin. In this case, bins is returned unmodified. + **kwds + All other plotting keyword arguments to be passed to + plotting backend. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + Examples + -------- + Basic plot. + + For Series: + + .. plotly:: + + >>> s = pp.Series([1, 3, 2]) + >>> s.plot.hist() # doctest: +SKIP + + For DataFrame: + + .. plotly:: + + >>> df = pd.DataFrame( + ... np.random.randint(1, 7, 6000), + ... columns=['one']) + >>> df['two'] = df['one'] + np.random.randint(1, 7, 6000) + >>> df = pp.from_pandas(df) + >>> df.plot.hist(bins=12, alpha=0.5) # doctest: +SKIP + """ + return self(kind="hist", bins=bins, **kwds) + + def kde(self, bw_method=None, ind=None, **kwargs): + """ + Generate Kernel Density Estimate plot using Gaussian kernels. + + Parameters + ---------- + bw_method : scalar + The method used to calculate the estimator bandwidth. + See KernelDensity in PySpark for more information. + ind : NumPy array or integer, optional + Evaluation points for the estimated PDF. If None (default), + 1000 equally spaced points are used. If `ind` is a NumPy array, the + KDE is evaluated at the points passed. If `ind` is an integer, + `ind` number of equally spaced points are used. + **kwargs : optional + Keyword arguments to pass on to :meth:`Koalas.Series.plot`. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + Examples + -------- + A scalar bandwidth should be specified. Using a small bandwidth value can + lead to over-fitting, while using a large bandwidth value may result + in under-fitting: + + .. plotly:: + + >>> s = pp.Series([1, 2, 2.5, 3, 3.5, 4, 5]) + >>> s.plot.kde(bw_method=0.3) # doctest: +SKIP + + .. plotly:: + + >>> s = pp.Series([1, 2, 2.5, 3, 3.5, 4, 5]) + >>> s.plot.kde(bw_method=3) # doctest: +SKIP + + The `ind` parameter determines the evaluation points for the + plot of the estimated KDF: + + .. plotly:: + + >>> s = pp.Series([1, 2, 2.5, 3, 3.5, 4, 5]) + >>> s.plot.kde(ind=[1, 2, 3, 4, 5], bw_method=0.3) # doctest: +SKIP + + For DataFrame, it works in the same way as Series: + + .. plotly:: + + >>> df = pp.DataFrame({ + ... 'x': [1, 2, 2.5, 3, 3.5, 4, 5], + ... 'y': [4, 4, 4.5, 5, 5.5, 6, 6], + ... }) + >>> df.plot.kde(bw_method=0.3) # doctest: +SKIP + + .. plotly:: + + >>> df = pp.DataFrame({ + ... 'x': [1, 2, 2.5, 3, 3.5, 4, 5], + ... 'y': [4, 4, 4.5, 5, 5.5, 6, 6], + ... }) + >>> df.plot.kde(bw_method=3) # doctest: +SKIP + + .. plotly:: + + >>> df = pp.DataFrame({ + ... 'x': [1, 2, 2.5, 3, 3.5, 4, 5], + ... 'y': [4, 4, 4.5, 5, 5.5, 6, 6], + ... }) + >>> df.plot.kde(ind=[1, 2, 3, 4, 5, 6], bw_method=0.3) # doctest: +SKIP + """ + return self(kind="kde", bw_method=bw_method, ind=ind, **kwargs) + + density = kde + + def area(self, x=None, y=None, **kwds): + """ + Draw a stacked area plot. + + An area plot displays quantitative data visually. + This function wraps the plotly area function. + + Parameters + ---------- + x : label or position, optional + Coordinates for the X axis. By default uses the index. + y : label or position, optional + Column to plot. By default uses all columns. + stacked : bool, default True + Area plots are stacked by default. Set to False to create a + unstacked plot (matplotlib-only). + **kwds : optional + Additional keyword arguments are documented in + :meth:`DataFrame.plot`. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + Examples + -------- + + For Series + + .. plotly:: + + >>> df = pp.DataFrame({ + ... 'sales': [3, 2, 3, 9, 10, 6], + ... 'signups': [5, 5, 6, 12, 14, 13], + ... 'visits': [20, 42, 28, 62, 81, 50], + ... }, index=pd.date_range(start='2018/01/01', end='2018/07/01', + ... freq='M')) + >>> df.sales.plot.area() # doctest: +SKIP + + For DataFrame + + .. plotly:: + + >>> df = pp.DataFrame({ + ... 'sales': [3, 2, 3, 9, 10, 6], + ... 'signups': [5, 5, 6, 12, 14, 13], + ... 'visits': [20, 42, 28, 62, 81, 50], + ... }, index=pd.date_range(start='2018/01/01', end='2018/07/01', + ... freq='M')) + >>> df.plot.area() # doctest: +SKIP + """ + from pyspark.pandas import DataFrame, Series + + if isinstance(self.data, Series): + return self(kind="area", **kwds) + elif isinstance(self.data, DataFrame): + return self(kind="area", x=x, y=y, **kwds) + + def pie(self, **kwds): + """ + Generate a pie plot. + + A pie plot is a proportional representation of the numerical data in a + column. This function wraps :meth:`plotly.express.pie` for the + specified column. + + Parameters + ---------- + y : int or label, optional + Label or position of the column to plot. + If not provided, ``subplots=True`` argument must be passed (matplotlib-only). + **kwds + Keyword arguments to pass on to :meth:`Koalas.Series.plot`. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + Examples + -------- + + For Series: + + .. plotly:: + + >>> df = pp.DataFrame({'mass': [0.330, 4.87, 5.97], + ... 'radius': [2439.7, 6051.8, 6378.1]}, + ... index=['Mercury', 'Venus', 'Earth']) + >>> df.mass.plot.pie() # doctest: +SKIP + + + For DataFrame: + + .. plotly:: + + >>> df = pp.DataFrame({'mass': [0.330, 4.87, 5.97], + ... 'radius': [2439.7, 6051.8, 6378.1]}, + ... index=['Mercury', 'Venus', 'Earth']) + >>> df.plot.pie(y='mass') # doctest: +SKIP + """ + from pyspark.pandas import DataFrame, Series + + if isinstance(self.data, Series): + return self(kind="pie", **kwds) + else: + # pandas will raise an error if y is None and subplots if not True + if ( + isinstance(self.data, DataFrame) + and kwds.get("y", None) is None + and not kwds.get("subplots", False) + ): + raise ValueError( + "pie requires either y column or 'subplots=True' (matplotlib-only)" + ) + return self(kind="pie", **kwds) + + def scatter(self, x, y, **kwds): + """ + Create a scatter plot with varying marker point size and color. + + The coordinates of each point are defined by two dataframe columns and + filled circles are used to represent each point. This kind of plot is + useful to see complex correlations between two variables. Points could + be for instance natural 2D coordinates like longitude and latitude in + a map or, in general, any pair of metrics that can be plotted against + each other. + + Parameters + ---------- + x : int or str + The column name or column position to be used as horizontal + coordinates for each point. + y : int or str + The column name or column position to be used as vertical + coordinates for each point. + s : scalar or array_like, optional + (matplotlib-only). + c : str, int or array_like, optional + (matplotlib-only). + + **kwds: Optional + Keyword arguments to pass on to :meth:`pyspark.pandas.DataFrame.plot`. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + Return an custom object when ``backend!=plotly``. + Return an ndarray when ``subplots=True`` (matplotlib-only). + + See Also + -------- + plotly.express.scatter : Scatter plot using multiple input data + formats (plotly). + matplotlib.pyplot.scatter : Scatter plot using multiple input data + formats (matplotlib). + + Examples + -------- + Let's see how to draw a scatter plot using coordinates from the values + in a DataFrame's columns. + + .. plotly:: + + >>> df = pp.DataFrame([[5.1, 3.5, 0], [4.9, 3.0, 0], [7.0, 3.2, 1], + ... [6.4, 3.2, 1], [5.9, 3.0, 2]], + ... columns=['length', 'width', 'species']) + >>> df.plot.scatter(x='length', y='width') # doctest: +SKIP + + And now with dark scheme: + + .. plotly:: + + >>> df = pp.DataFrame([[5.1, 3.5, 0], [4.9, 3.0, 0], [7.0, 3.2, 1], + ... [6.4, 3.2, 1], [5.9, 3.0, 2]], + ... columns=['length', 'width', 'species']) + >>> fig = df.plot.scatter(x='length', y='width') + >>> fig.update_layout(template="plotly_dark") # doctest: +SKIP + """ + return self(kind="scatter", x=x, y=y, **kwds) + + def hexbin(self, **kwds): + return unsupported_function(class_name="pd.DataFrame", method_name="hexbin")() diff --git a/python/pyspark/pandas/plot/matplotlib.py b/python/pyspark/pandas/plot/matplotlib.py new file mode 100644 index 0000000000..3aadbe4f24 --- /dev/null +++ b/python/pyspark/pandas/plot/matplotlib.py @@ -0,0 +1,897 @@ +# +# 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. +# + +from distutils.version import LooseVersion + +import matplotlib as mat +import numpy as np +import pandas as pd +from matplotlib.axes._base import _process_plot_format +from pandas.core.dtypes.inference import is_list_like +from pandas.io.formats.printing import pprint_thing + +from pyspark.pandas.plot import ( + TopNPlotBase, + SampledPlotBase, + HistogramPlotBase, + BoxPlotBase, + unsupported_function, + KdePlotBase, +) + + +if LooseVersion(pd.__version__) < LooseVersion("0.25"): + from pandas.plotting._core import ( + _all_kinds, + BarPlot as PandasBarPlot, + BoxPlot as PandasBoxPlot, + HistPlot as PandasHistPlot, + MPLPlot as PandasMPLPlot, + PiePlot as PandasPiePlot, + AreaPlot as PandasAreaPlot, + LinePlot as PandasLinePlot, + BarhPlot as PandasBarhPlot, + ScatterPlot as PandasScatterPlot, + KdePlot as PandasKdePlot, + ) +else: + from pandas.plotting._matplotlib import ( + BarPlot as PandasBarPlot, + BoxPlot as PandasBoxPlot, + HistPlot as PandasHistPlot, + PiePlot as PandasPiePlot, + AreaPlot as PandasAreaPlot, + LinePlot as PandasLinePlot, + BarhPlot as PandasBarhPlot, + ScatterPlot as PandasScatterPlot, + KdePlot as PandasKdePlot, + ) + from pandas.plotting._core import PlotAccessor + from pandas.plotting._matplotlib.core import MPLPlot as PandasMPLPlot + + _all_kinds = PlotAccessor._all_kinds + + +class KoalasBarPlot(PandasBarPlot, TopNPlotBase): + def __init__(self, data, **kwargs): + super().__init__(self.get_top_n(data), **kwargs) + + def _plot(self, ax, x, y, w, start=0, log=False, **kwds): + self.set_result_text(ax) + return ax.bar(x, y, w, bottom=start, log=log, **kwds) + + +class KoalasBoxPlot(PandasBoxPlot, BoxPlotBase): + def boxplot( + self, + ax, + bxpstats, + notch=None, + sym=None, + vert=None, + whis=None, + positions=None, + widths=None, + patch_artist=None, + bootstrap=None, + usermedians=None, + conf_intervals=None, + meanline=None, + showmeans=None, + showcaps=None, + showbox=None, + showfliers=None, + boxprops=None, + labels=None, + flierprops=None, + medianprops=None, + meanprops=None, + capprops=None, + whiskerprops=None, + manage_ticks=None, + # manage_xticks is for compatibility of matplotlib < 3.1.0. + # Remove this when minimum version is 3.0.0 + manage_xticks=None, + autorange=False, + zorder=None, + precision=None, + ): + def update_dict(dictionary, rc_name, properties): + """ Loads properties in the dictionary from rc file if not already + in the dictionary""" + rc_str = "boxplot.{0}.{1}" + if dictionary is None: + dictionary = dict() + for prop_dict in properties: + dictionary.setdefault(prop_dict, mat.rcParams[rc_str.format(rc_name, prop_dict)]) + return dictionary + + # Common property dictionaries loading from rc + flier_props = [ + "color", + "marker", + "markerfacecolor", + "markeredgecolor", + "markersize", + "linestyle", + "linewidth", + ] + default_props = ["color", "linewidth", "linestyle"] + + boxprops = update_dict(boxprops, "boxprops", default_props) + whiskerprops = update_dict(whiskerprops, "whiskerprops", default_props) + capprops = update_dict(capprops, "capprops", default_props) + medianprops = update_dict(medianprops, "medianprops", default_props) + meanprops = update_dict(meanprops, "meanprops", default_props) + flierprops = update_dict(flierprops, "flierprops", flier_props) + + if patch_artist: + boxprops["linestyle"] = "solid" + boxprops["edgecolor"] = boxprops.pop("color") + + # if non-default sym value, put it into the flier dictionary + # the logic for providing the default symbol ('b+') now lives + # in bxp in the initial value of final_flierprops + # handle all of the `sym` related logic here so we only have to pass + # on the flierprops dict. + if sym is not None: + # no-flier case, which should really be done with + # 'showfliers=False' but none-the-less deal with it to keep back + # compatibility + if sym == "": + # blow away existing dict and make one for invisible markers + flierprops = dict(linestyle="none", marker="", color="none") + # turn the fliers off just to be safe + showfliers = False + # now process the symbol string + else: + # process the symbol string + # discarded linestyle + _, marker, color = _process_plot_format(sym) + # if we have a marker, use it + if marker is not None: + flierprops["marker"] = marker + # if we have a color, use it + if color is not None: + # assume that if color is passed in the user want + # filled symbol, if the users want more control use + # flierprops + flierprops["color"] = color + flierprops["markerfacecolor"] = color + flierprops["markeredgecolor"] = color + + # replace medians if necessary: + if usermedians is not None: + if len(np.ravel(usermedians)) != len(bxpstats) or np.shape(usermedians)[0] != len( + bxpstats + ): + raise ValueError("usermedians length not compatible with x") + else: + # reassign medians as necessary + for stats, med in zip(bxpstats, usermedians): + if med is not None: + stats["med"] = med + + if conf_intervals is not None: + if np.shape(conf_intervals)[0] != len(bxpstats): + err_mess = "conf_intervals length not compatible with x" + raise ValueError(err_mess) + else: + for stats, ci in zip(bxpstats, conf_intervals): + if ci is not None: + if len(ci) != 2: + raise ValueError("each confidence interval must " "have two values") + else: + if ci[0] is not None: + stats["cilo"] = ci[0] + if ci[1] is not None: + stats["cihi"] = ci[1] + + should_manage_ticks = True + if manage_xticks is not None: + should_manage_ticks = manage_xticks + if manage_ticks is not None: + should_manage_ticks = manage_ticks + + if LooseVersion(mat.__version__) < LooseVersion("3.1.0"): + extra_args = {"manage_xticks": should_manage_ticks} + else: + extra_args = {"manage_ticks": should_manage_ticks} + + artists = ax.bxp( + bxpstats, + positions=positions, + widths=widths, + vert=vert, + patch_artist=patch_artist, + shownotches=notch, + showmeans=showmeans, + showcaps=showcaps, + showbox=showbox, + boxprops=boxprops, + flierprops=flierprops, + medianprops=medianprops, + meanprops=meanprops, + meanline=meanline, + showfliers=showfliers, + capprops=capprops, + whiskerprops=whiskerprops, + zorder=zorder, + **extra_args, + ) + return artists + + def _plot(self, ax, bxpstats, column_num=None, return_type="axes", **kwds): + bp = self.boxplot(ax, bxpstats, **kwds) + + if return_type == "dict": + return bp, bp + elif return_type == "both": + return self.BP(ax=ax, lines=bp), bp + else: + return ax, bp + + def _compute_plot_data(self): + colname = self.data.name + spark_column_name = self.data._internal.spark_column_name_for(self.data._column_label) + data = self.data + + # Updates all props with the rc defaults from matplotlib + self.kwds.update(KoalasBoxPlot.rc_defaults(**self.kwds)) + + # Gets some important kwds + showfliers = self.kwds.get("showfliers", False) + whis = self.kwds.get("whis", 1.5) + labels = self.kwds.get("labels", [colname]) + + # This one is Koalas specific to control precision for approx_percentile + precision = self.kwds.get("precision", 0.01) + + # # Computes mean, median, Q1 and Q3 with approx_percentile and precision + col_stats, col_fences = BoxPlotBase.compute_stats(data, spark_column_name, whis, precision) + + # # Creates a column to flag rows as outliers or not + outliers = BoxPlotBase.outliers(data, spark_column_name, *col_fences) + + # # Computes min and max values of non-outliers - the whiskers + whiskers = BoxPlotBase.calc_whiskers(spark_column_name, outliers) + + if showfliers: + fliers = BoxPlotBase.get_fliers(spark_column_name, outliers, whiskers[0]) + else: + fliers = [] + + # Builds bxpstats dict + stats = [] + item = { + "mean": col_stats["mean"], + "med": col_stats["med"], + "q1": col_stats["q1"], + "q3": col_stats["q3"], + "whislo": whiskers[0], + "whishi": whiskers[1], + "fliers": fliers, + "label": labels[0], + } + stats.append(item) + + self.data = {labels[0]: stats} + + def _make_plot(self): + bxpstats = list(self.data.values())[0] + ax = self._get_ax(0) + kwds = self.kwds.copy() + + for stats in bxpstats: + if len(stats["fliers"]) > 1000: + stats["fliers"] = stats["fliers"][:1000] + ax.text( + 1, + 1, + "showing top 1,000 fliers only", + size=6, + ha="right", + va="bottom", + transform=ax.transAxes, + ) + + ret, bp = self._plot(ax, bxpstats, column_num=0, return_type=self.return_type, **kwds) + self.maybe_color_bp(bp) + self._return_obj = ret + + labels = [l for l, _ in self.data.items()] + labels = [pprint_thing(l) for l in labels] + if not self.use_index: + labels = [pprint_thing(key) for key in range(len(labels))] + self._set_ticklabels(ax, labels) + + @staticmethod + def rc_defaults( + notch=None, + vert=None, + whis=None, + patch_artist=None, + bootstrap=None, + meanline=None, + showmeans=None, + showcaps=None, + showbox=None, + showfliers=None, + **kwargs + ): + # Missing arguments default to rcParams. + if whis is None: + whis = mat.rcParams["boxplot.whiskers"] + if bootstrap is None: + bootstrap = mat.rcParams["boxplot.bootstrap"] + + if notch is None: + notch = mat.rcParams["boxplot.notch"] + if vert is None: + vert = mat.rcParams["boxplot.vertical"] + if patch_artist is None: + patch_artist = mat.rcParams["boxplot.patchartist"] + if meanline is None: + meanline = mat.rcParams["boxplot.meanline"] + if showmeans is None: + showmeans = mat.rcParams["boxplot.showmeans"] + if showcaps is None: + showcaps = mat.rcParams["boxplot.showcaps"] + if showbox is None: + showbox = mat.rcParams["boxplot.showbox"] + if showfliers is None: + showfliers = mat.rcParams["boxplot.showfliers"] + + return dict( + whis=whis, + bootstrap=bootstrap, + notch=notch, + vert=vert, + patch_artist=patch_artist, + meanline=meanline, + showmeans=showmeans, + showcaps=showcaps, + showbox=showbox, + showfliers=showfliers, + ) + + +class KoalasHistPlot(PandasHistPlot, HistogramPlotBase): + def _args_adjust(self): + if is_list_like(self.bottom): + self.bottom = np.array(self.bottom) + + def _compute_plot_data(self): + self.data, self.bins = HistogramPlotBase.prepare_hist_data(self.data, self.bins) + + def _make_plot(self): + # TODO: this logic is similar with KdePlot. Might have to deduplicate it. + # 'num_colors' requires to calculate `shape` which has to count all. + # Use 1 for now to save the computation. + colors = self._get_colors(num_colors=1) + stacking_id = self._get_stacking_id() + output_series = HistogramPlotBase.compute_hist(self.data, self.bins) + + for (i, label), y in zip(enumerate(self.data._internal.column_labels), output_series): + ax = self._get_ax(i) + + kwds = self.kwds.copy() + + label = pprint_thing(label if len(label) > 1 else label[0]) + kwds["label"] = label + + style, kwds = self._apply_style_colors(colors, kwds, i, label) + if style is not None: + kwds["style"] = style + + kwds = self._make_plot_keywords(kwds, y) + artists = self._plot(ax, y, column_num=i, stacking_id=stacking_id, **kwds) + self._add_legend_handle(artists[0], label, index=i) + + @classmethod + def _plot(cls, ax, y, style=None, bins=None, bottom=0, column_num=0, stacking_id=None, **kwds): + if column_num == 0: + cls._initialize_stacker(ax, stacking_id, len(bins) - 1) + + base = np.zeros(len(bins) - 1) + bottom = bottom + cls._get_stacked_values(ax, stacking_id, base, kwds["label"]) + + # Since the counts were computed already, we use them as weights and just generate + # one entry for each bin + n, bins, patches = ax.hist(bins[:-1], bins=bins, bottom=bottom, weights=y, **kwds) + + cls._update_stacker(ax, stacking_id, n) + return patches + + +class KoalasPiePlot(PandasPiePlot, TopNPlotBase): + def __init__(self, data, **kwargs): + super().__init__(self.get_top_n(data), **kwargs) + + def _make_plot(self): + self.set_result_text(self._get_ax(0)) + super()._make_plot() + + +class KoalasAreaPlot(PandasAreaPlot, SampledPlotBase): + def __init__(self, data, **kwargs): + super().__init__(self.get_sampled(data), **kwargs) + + def _make_plot(self): + self.set_result_text(self._get_ax(0)) + super()._make_plot() + + +class KoalasLinePlot(PandasLinePlot, SampledPlotBase): + def __init__(self, data, **kwargs): + super().__init__(self.get_sampled(data), **kwargs) + + def _make_plot(self): + self.set_result_text(self._get_ax(0)) + super()._make_plot() + + +class KoalasBarhPlot(PandasBarhPlot, TopNPlotBase): + def __init__(self, data, **kwargs): + super().__init__(self.get_top_n(data), **kwargs) + + def _make_plot(self): + self.set_result_text(self._get_ax(0)) + super()._make_plot() + + +class KoalasScatterPlot(PandasScatterPlot, TopNPlotBase): + def __init__(self, data, x, y, **kwargs): + super().__init__(self.get_top_n(data), x, y, **kwargs) + + def _make_plot(self): + self.set_result_text(self._get_ax(0)) + super()._make_plot() + + +class KoalasKdePlot(PandasKdePlot, KdePlotBase): + def _compute_plot_data(self): + self.data = KdePlotBase.prepare_kde_data(self.data) + + def _make_plot(self): + # 'num_colors' requires to calculate `shape` which has to count all. + # Use 1 for now to save the computation. + colors = self._get_colors(num_colors=1) + stacking_id = self._get_stacking_id() + + sdf = self.data._internal.spark_frame + + for i, label in enumerate(self.data._internal.column_labels): + # 'y' is a Spark DataFrame that selects one column. + y = sdf.select(self.data._internal.spark_column_for(label)) + ax = self._get_ax(i) + + kwds = self.kwds.copy() + + label = pprint_thing(label if len(label) > 1 else label[0]) + kwds["label"] = label + + style, kwds = self._apply_style_colors(colors, kwds, i, label) + if style is not None: + kwds["style"] = style + + kwds = self._make_plot_keywords(kwds, y) + artists = self._plot(ax, y, column_num=i, stacking_id=stacking_id, **kwds) + self._add_legend_handle(artists[0], label, index=i) + + def _get_ind(self, y): + return KdePlotBase.get_ind(y, self.ind) + + @classmethod + def _plot( + cls, ax, y, style=None, bw_method=None, ind=None, column_num=None, stacking_id=None, **kwds + ): + y = KdePlotBase.compute_kde(y, bw_method=bw_method, ind=ind) + lines = PandasMPLPlot._plot(ax, ind, y, style=style, **kwds) + return lines + + +_klasses = [ + KoalasHistPlot, + KoalasBarPlot, + KoalasBoxPlot, + KoalasPiePlot, + KoalasAreaPlot, + KoalasLinePlot, + KoalasBarhPlot, + KoalasScatterPlot, + KoalasKdePlot, +] +_plot_klass = {getattr(klass, "_kind"): klass for klass in _klasses} +_common_kinds = {"area", "bar", "barh", "box", "hist", "kde", "line", "pie"} +_series_kinds = _common_kinds.union(set()) +_dataframe_kinds = _common_kinds.union({"scatter", "hexbin"}) +_koalas_all_kinds = _common_kinds.union(_series_kinds).union(_dataframe_kinds) + + +def plot_koalas(data, kind, **kwargs): + if kind not in _koalas_all_kinds: + raise ValueError("{} is not a valid plot kind".format(kind)) + + from pyspark.pandas import DataFrame, Series + + if isinstance(data, Series): + if kind not in _series_kinds: + return unsupported_function(class_name="pd.Series", method_name=kind)() + return plot_series(data=data, kind=kind, **kwargs) + elif isinstance(data, DataFrame): + if kind not in _dataframe_kinds: + return unsupported_function(class_name="pd.DataFrame", method_name=kind)() + return plot_frame(data=data, kind=kind, **kwargs) + + +def plot_series( + data, + kind="line", + ax=None, # Series unique + figsize=None, + use_index=True, + title=None, + grid=None, + legend=False, + style=None, + logx=False, + logy=False, + loglog=False, + xticks=None, + yticks=None, + xlim=None, + ylim=None, + rot=None, + fontsize=None, + colormap=None, + table=False, + yerr=None, + xerr=None, + label=None, + secondary_y=False, # Series unique + **kwds +): + """ + Make plots of Series using matplotlib / pylab. + + Each plot kind has a corresponding method on the + ``Series.plot`` accessor: + ``s.plot(kind='line')`` is equivalent to + ``s.plot.line()``. + + Parameters + ---------- + data : Series + + kind : str + - 'line' : line plot (default) + - 'bar' : vertical bar plot + - 'barh' : horizontal bar plot + - 'hist' : histogram + - 'box' : boxplot + - 'kde' : Kernel Density Estimation plot + - 'density' : same as 'kde' + - 'area' : area plot + - 'pie' : pie plot + + ax : matplotlib axes object + If not passed, uses gca() + figsize : a tuple (width, height) in inches + use_index : boolean, default True + Use index as ticks for x axis + title : string or list + Title to use for the plot. If a string is passed, print the string at + the top of the figure. If a list is passed and `subplots` is True, + print each item in the list above the corresponding subplot. + grid : boolean, default None (matlab style default) + Axis grid lines + legend : False/True/'reverse' + Place legend on axis subplots + style : list or dict + matplotlib line style per column + logx : boolean, default False + Use log scaling on x axis + logy : boolean, default False + Use log scaling on y axis + loglog : boolean, default False + Use log scaling on both x and y axes + xticks : sequence + Values to use for the xticks + yticks : sequence + Values to use for the yticks + xlim : 2-tuple/list + ylim : 2-tuple/list + rot : int, default None + Rotation for ticks (xticks for vertical, yticks for horizontal plots) + fontsize : int, default None + Font size for xticks and yticks + colormap : str or matplotlib colormap object, default None + Colormap to select colors from. If string, load colormap with that name + from matplotlib. + colorbar : boolean, optional + If True, plot colorbar (only relevant for 'scatter' and 'hexbin' plots) + position : float + Specify relative alignments for bar plot layout. + From 0 (left/bottom-end) to 1 (right/top-end). Default is 0.5 (center) + table : boolean, Series or DataFrame, default False + If True, draw a table using the data in the DataFrame and the data will + be transposed to meet matplotlib's default layout. + If a Series or DataFrame is passed, use passed data to draw a table. + yerr : DataFrame, Series, array-like, dict and str + See :ref:`Plotting with Error Bars ` for + detail. + xerr : same types as yerr. + label : label argument to provide to plot + secondary_y : boolean or sequence of ints, default False + If True then y-axis will be on the right + mark_right : boolean, default True + When using a secondary_y axis, automatically mark the column + labels with "(right)" in the legend + **kwds : keywords + Options to pass to matplotlib plotting method + + Returns + ------- + axes : :class:`matplotlib.axes.Axes` or numpy.ndarray of them + + Notes + ----- + + - See matplotlib documentation online for more on this subject + - If `kind` = 'bar' or 'barh', you can specify relative alignments + for bar plot layout by `position` keyword. + From 0 (left/bottom-end) to 1 (right/top-end). Default is 0.5 (center) + """ + + # function copied from pandas.plotting._core + # so it calls modified _plot below + + import matplotlib.pyplot as plt + + if ax is None and len(plt.get_fignums()) > 0: + with plt.rc_context(): + ax = plt.gca() + ax = PandasMPLPlot._get_ax_layer(ax) + return _plot( + data, + kind=kind, + ax=ax, + figsize=figsize, + use_index=use_index, + title=title, + grid=grid, + legend=legend, + style=style, + logx=logx, + logy=logy, + loglog=loglog, + xticks=xticks, + yticks=yticks, + xlim=xlim, + ylim=ylim, + rot=rot, + fontsize=fontsize, + colormap=colormap, + table=table, + yerr=yerr, + xerr=xerr, + label=label, + secondary_y=secondary_y, + **kwds, + ) + + +def plot_frame( + data, + x=None, + y=None, + kind="line", + ax=None, + subplots=None, + sharex=None, + sharey=False, + layout=None, + figsize=None, + use_index=True, + title=None, + grid=None, + legend=True, + style=None, + logx=False, + logy=False, + loglog=False, + xticks=None, + yticks=None, + xlim=None, + ylim=None, + rot=None, + fontsize=None, + colormap=None, + table=False, + yerr=None, + xerr=None, + secondary_y=False, + sort_columns=False, + **kwds +): + """ + Make plots of DataFrames using matplotlib / pylab. + + Each plot kind has a corresponding method on the + ``DataFrame.plot`` accessor: + ``kdf.plot(kind='line')`` is equivalent to + ``kdf.plot.line()``. + + Parameters + ---------- + data : DataFrame + + kind : str + - 'line' : line plot (default) + - 'bar' : vertical bar plot + - 'barh' : horizontal bar plot + - 'hist' : histogram + - 'box' : boxplot + - 'kde' : Kernel Density Estimation plot + - 'density' : same as 'kde' + - 'area' : area plot + - 'pie' : pie plot + - 'scatter' : scatter plot + ax : matplotlib axes object + If not passed, uses gca() + x : label or position, default None + y : label, position or list of label, positions, default None + Allows plotting of one column versus another. + figsize : a tuple (width, height) in inches + use_index : boolean, default True + Use index as ticks for x axis + title : string or list + Title to use for the plot. If a string is passed, print the string at + the top of the figure. If a list is passed and `subplots` is True, + print each item in the list above the corresponding subplot. + grid : boolean, default None (matlab style default) + Axis grid lines + legend : False/True/'reverse' + Place legend on axis subplots + style : list or dict + matplotlib line style per column + logx : boolean, default False + Use log scaling on x axis + logy : boolean, default False + Use log scaling on y axis + loglog : boolean, default False + Use log scaling on both x and y axes + xticks : sequence + Values to use for the xticks + yticks : sequence + Values to use for the yticks + xlim : 2-tuple/list + ylim : 2-tuple/list + sharex: bool or None, default is None + Whether to share x axis or not. + sharey: bool, default is False + Whether to share y axis or not. + rot : int, default None + Rotation for ticks (xticks for vertical, yticks for horizontal plots) + fontsize : int, default None + Font size for xticks and yticks + colormap : str or matplotlib colormap object, default None + Colormap to select colors from. If string, load colormap with that name + from matplotlib. + colorbar : boolean, optional + If True, plot colorbar (only relevant for 'scatter' and 'hexbin' plots) + position : float + Specify relative alignments for bar plot layout. + From 0 (left/bottom-end) to 1 (right/top-end). Default is 0.5 (center) + table : boolean, Series or DataFrame, default False + If True, draw a table using the data in the DataFrame and the data will + be transposed to meet matplotlib's default layout. + If a Series or DataFrame is passed, use passed data to draw a table. + yerr : DataFrame, Series, array-like, dict and str + See :ref:`Plotting with Error Bars ` for + detail. + xerr : same types as yerr. + label : label argument to provide to plot + secondary_y : boolean or sequence of ints, default False + If True then y-axis will be on the right + mark_right : boolean, default True + When using a secondary_y axis, automatically mark the column + labels with "(right)" in the legend + sort_columns: bool, default is False + When True, will sort values on plots. + **kwds : keywords + Options to pass to matplotlib plotting method + + Returns + ------- + axes : :class:`matplotlib.axes.Axes` or numpy.ndarray of them + + Notes + ----- + + - See matplotlib documentation online for more on this subject + - If `kind` = 'bar' or 'barh', you can specify relative alignments + for bar plot layout by `position` keyword. + From 0 (left/bottom-end) to 1 (right/top-end). Default is 0.5 (center) + """ + + return _plot( + data, + kind=kind, + x=x, + y=y, + ax=ax, + figsize=figsize, + use_index=use_index, + title=title, + grid=grid, + legend=legend, + subplots=subplots, + style=style, + logx=logx, + logy=logy, + loglog=loglog, + xticks=xticks, + yticks=yticks, + xlim=xlim, + ylim=ylim, + rot=rot, + fontsize=fontsize, + colormap=colormap, + table=table, + yerr=yerr, + xerr=xerr, + sharex=sharex, + sharey=sharey, + secondary_y=secondary_y, + layout=layout, + sort_columns=sort_columns, + **kwds, + ) + + +def _plot(data, x=None, y=None, subplots=False, ax=None, kind="line", **kwds): + from pyspark.pandas import DataFrame + + # function copied from pandas.plotting._core + # and adapted to handle Koalas DataFrame and Series + + kind = kind.lower().strip() + kind = {"density": "kde"}.get(kind, kind) + if kind in _all_kinds: + klass = _plot_klass[kind] + else: + raise ValueError("%r is not a valid plot kind" % kind) + + # scatter and hexbin are inherited from PlanePlot which require x and y + if kind in ("scatter", "hexbin"): + plot_obj = klass(data, x, y, subplots=subplots, ax=ax, kind=kind, **kwds) + else: + + # check data type and do preprocess before applying plot + if isinstance(data, DataFrame): + if x is not None: + data = data.set_index(x) + # TODO: check if value of y is plottable + if y is not None: + data = data[y] + + plot_obj = klass(data, subplots=subplots, ax=ax, kind=kind, **kwds) + plot_obj.generate() + plot_obj.draw() + return plot_obj.result diff --git a/python/pyspark/pandas/plot/plotly.py b/python/pyspark/pandas/plot/plotly.py new file mode 100644 index 0000000000..bb2d9283bf --- /dev/null +++ b/python/pyspark/pandas/plot/plotly.py @@ -0,0 +1,212 @@ +# +# 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. +# +from typing import TYPE_CHECKING, Union + +import pandas as pd + +from pyspark.pandas.plot import ( + HistogramPlotBase, + name_like_string, + KoalasPlotAccessor, + BoxPlotBase, + KdePlotBase, +) + +if TYPE_CHECKING: + import pyspark.pandas as pp # noqa: F401 (SPARK-34943) + + +def plot_koalas(data: Union["pp.DataFrame", "pp.Series"], kind: str, **kwargs): + import plotly + + # Koalas specific plots + if kind == "pie": + return plot_pie(data, **kwargs) + if kind == "hist": + return plot_histogram(data, **kwargs) + if kind == "box": + return plot_box(data, **kwargs) + if kind == "kde" or kind == "density": + return plot_kde(data, **kwargs) + + # Other plots. + return plotly.plot(KoalasPlotAccessor.pandas_plot_data_map[kind](data), kind, **kwargs) + + +def plot_pie(data: Union["pp.DataFrame", "pp.Series"], **kwargs): + from plotly import express + + data = KoalasPlotAccessor.pandas_plot_data_map["pie"](data) + + if isinstance(data, pd.Series): + pdf = data.to_frame() + return express.pie(pdf, values=pdf.columns[0], names=pdf.index, **kwargs) + elif isinstance(data, pd.DataFrame): + values = kwargs.pop("y", None) + default_names = None + if values is not None: + default_names = data.index + + return express.pie( + data, + values=kwargs.pop("values", values), + names=kwargs.pop("names", default_names), + **kwargs, + ) + else: + raise RuntimeError("Unexpected type: [%s]" % type(data)) + + +def plot_histogram(data: Union["pp.DataFrame", "pp.Series"], **kwargs): + import plotly.graph_objs as go + + bins = kwargs.get("bins", 10) + kdf, bins = HistogramPlotBase.prepare_hist_data(data, bins) + assert len(bins) > 2, "the number of buckets must be higher than 2." + output_series = HistogramPlotBase.compute_hist(kdf, bins) + prev = float("%.9f" % bins[0]) # to make it prettier, truncate. + text_bins = [] + for b in bins[1:]: + norm_b = float("%.9f" % b) + text_bins.append("[%s, %s)" % (prev, norm_b)) + prev = norm_b + text_bins[-1] = text_bins[-1][:-1] + "]" # replace ) to ] for the last bucket. + + bins = 0.5 * (bins[:-1] + bins[1:]) + + output_series = list(output_series) + bars = [] + for series in output_series: + bars.append( + go.Bar( + x=bins, + y=series, + name=name_like_string(series.name), + text=text_bins, + hovertemplate=( + "variable=" + name_like_string(series.name) + "
value=%{text}
count=%{y}" + ), + ) + ) + + fig = go.Figure(data=bars, layout=go.Layout(barmode="stack")) + fig["layout"]["xaxis"]["title"] = "value" + fig["layout"]["yaxis"]["title"] = "count" + return fig + + +def plot_box(data: Union["pp.DataFrame", "pp.Series"], **kwargs): + import plotly.graph_objs as go + import pyspark.pandas as pp + + if isinstance(data, pp.DataFrame): + raise RuntimeError( + "plotly does not support a box plot with Koalas DataFrame. Use Series instead." + ) + + # 'whis' isn't actually an argument in plotly (but in matplotlib). But seems like + # plotly doesn't expose the reach of the whiskers to the beyond the first and + # third quartiles (?). Looks they use default 1.5. + whis = kwargs.pop("whis", 1.5) + # 'precision' is Koalas specific to control precision for approx_percentile + precision = kwargs.pop("precision", 0.01) + + # Plotly options + boxpoints = kwargs.pop("boxpoints", "suspectedoutliers") + notched = kwargs.pop("notched", False) + if boxpoints not in ["suspectedoutliers", False]: + raise ValueError( + "plotly plotting backend does not support 'boxpoints' set to '%s'. " + "Set to 'suspectedoutliers' or False." % boxpoints + ) + if notched: + raise ValueError( + "plotly plotting backend does not support 'notched' set to '%s'. " + "Set to False." % notched + ) + + colname = name_like_string(data.name) + spark_column_name = data._internal.spark_column_name_for(data._column_label) + + # Computes mean, median, Q1 and Q3 with approx_percentile and precision + col_stats, col_fences = BoxPlotBase.compute_stats(data, spark_column_name, whis, precision) + + # Creates a column to flag rows as outliers or not + outliers = BoxPlotBase.outliers(data, spark_column_name, *col_fences) + + # Computes min and max values of non-outliers - the whiskers + whiskers = BoxPlotBase.calc_whiskers(spark_column_name, outliers) + + fliers = None + if boxpoints: + fliers = BoxPlotBase.get_fliers(spark_column_name, outliers, whiskers[0]) + fliers = [fliers] if len(fliers) > 0 else None + + fig = go.Figure() + fig.add_trace( + go.Box( + name=colname, + q1=[col_stats["q1"]], + median=[col_stats["med"]], + q3=[col_stats["q3"]], + mean=[col_stats["mean"]], + lowerfence=[whiskers[0]], + upperfence=[whiskers[1]], + y=fliers, + boxpoints=boxpoints, + notched=notched, + **kwargs, # this is for workarounds. Box takes different options from express.box. + ) + ) + fig["layout"]["xaxis"]["title"] = colname + fig["layout"]["yaxis"]["title"] = "value" + return fig + + +def plot_kde(data: Union["pp.DataFrame", "pp.Series"], **kwargs): + from plotly import express + import pyspark.pandas as pp + + if isinstance(data, pp.DataFrame) and "color" not in kwargs: + kwargs["color"] = "names" + + kdf = KdePlotBase.prepare_kde_data(data) + sdf = kdf._internal.spark_frame + data_columns = kdf._internal.data_spark_columns + ind = KdePlotBase.get_ind(sdf.select(*data_columns), kwargs.pop("ind", None)) + bw_method = kwargs.pop("bw_method", None) + + pdfs = [] + for label in kdf._internal.column_labels: + pdfs.append( + pd.DataFrame( + { + "Density": KdePlotBase.compute_kde( + sdf.select(kdf._internal.spark_column_for(label)), + ind=ind, + bw_method=bw_method, + ), + "names": name_like_string(label), + "index": ind, + } + ) + ) + pdf = pd.concat(pdfs) + + fig = express.line(pdf, x="index", y="Density", **kwargs) + fig["layout"]["xaxis"]["title"] = None + return fig diff --git a/python/pyspark/pandas/series.py b/python/pyspark/pandas/series.py new file mode 100644 index 0000000000..4f513a9a27 --- /dev/null +++ b/python/pyspark/pandas/series.py @@ -0,0 +1,6235 @@ +# +# 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. +# + +""" +A wrapper class for Spark Column to behave similar to pandas Series. +""" +import datetime +import re +import inspect +import sys +import warnings +from collections.abc import Mapping +from distutils.version import LooseVersion +from functools import partial, wraps, reduce +from typing import Any, Generic, Iterable, List, Optional, Tuple, TypeVar, Union, cast + +import numpy as np +import pandas as pd +from pandas.core.accessor import CachedAccessor +from pandas.io.formats.printing import pprint_thing +from pandas.api.types import is_list_like, is_hashable +from pandas.api.extensions import ExtensionDtype +from pandas.tseries.frequencies import DateOffset +import pyspark +from pyspark import sql as spark +from pyspark.sql import functions as F, Column +from pyspark.sql.types import ( + BooleanType, + DoubleType, + FloatType, + IntegerType, + LongType, + NumericType, + StructType, + IntegralType, + ArrayType, +) +from pyspark.sql.window import Window + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas.accessors import KoalasSeriesMethods +from pyspark.pandas.categorical import CategoricalAccessor +from pyspark.pandas.config import get_option +from pyspark.pandas.base import IndexOpsMixin +from pyspark.pandas.exceptions import SparkPandasIndexingError +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.generic import Frame +from pyspark.pandas.internal import ( + InternalFrame, + DEFAULT_SERIES_NAME, + NATURAL_ORDER_COLUMN_NAME, + SPARK_DEFAULT_INDEX_NAME, + SPARK_DEFAULT_SERIES_NAME, +) +from pyspark.pandas.missing.series import MissingPandasLikeSeries +from pyspark.pandas.plot import KoalasPlotAccessor +from pyspark.pandas.ml import corr +from pyspark.pandas.utils import ( + combine_frames, + is_name_like_tuple, + is_name_like_value, + name_like_string, + same_anchor, + scol_for, + sql_conf, + validate_arguments_and_invoke_function, + validate_axis, + validate_bool_kwarg, + verify_temp_column_name, + SPARK_CONF_ARROW_ENABLED, +) +from pyspark.pandas.datetimes import DatetimeMethods +from pyspark.pandas.spark import functions as SF +from pyspark.pandas.spark.accessors import SparkSeriesMethods +from pyspark.pandas.strings import StringMethods +from pyspark.pandas.typedef import ( + infer_return_type, + spark_type_to_pandas_dtype, + ScalarType, + Scalar, + SeriesType, +) + + +# This regular expression pattern is complied and defined here to avoid to compile the same +# pattern every time it is used in _repr_ in Series. +# This pattern basically seeks the footer string from pandas' +REPR_PATTERN = re.compile(r"Length: (?P[0-9]+)") + +_flex_doc_SERIES = """ +Return {desc} of series and other, element-wise (binary operator `{op_name}`). + +Equivalent to ``{equiv}`` + +Parameters +---------- +other : Series or scalar value + +Returns +------- +Series + The result of the operation. + +See Also +-------- +Series.{reverse} + +{series_examples} +""" + +_add_example_SERIES = """ +Examples +-------- +>>> df = pp.DataFrame({'a': [2, 2, 4, np.nan], +... 'b': [2, np.nan, 2, np.nan]}, +... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) +>>> df + a b +a 2.0 2.0 +b 2.0 NaN +c 4.0 2.0 +d NaN NaN + +>>> df.a.add(df.b) +a 4.0 +b NaN +c 6.0 +d NaN +dtype: float64 + +>>> df.a.radd(df.b) +a 4.0 +b NaN +c 6.0 +d NaN +dtype: float64 +""" + +_sub_example_SERIES = """ +Examples +-------- +>>> df = pp.DataFrame({'a': [2, 2, 4, np.nan], +... 'b': [2, np.nan, 2, np.nan]}, +... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) +>>> df + a b +a 2.0 2.0 +b 2.0 NaN +c 4.0 2.0 +d NaN NaN + +>>> df.a.subtract(df.b) +a 0.0 +b NaN +c 2.0 +d NaN +dtype: float64 + +>>> df.a.rsub(df.b) +a 0.0 +b NaN +c -2.0 +d NaN +dtype: float64 +""" + +_mul_example_SERIES = """ +Examples +-------- +>>> df = pp.DataFrame({'a': [2, 2, 4, np.nan], +... 'b': [2, np.nan, 2, np.nan]}, +... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) +>>> df + a b +a 2.0 2.0 +b 2.0 NaN +c 4.0 2.0 +d NaN NaN + +>>> df.a.multiply(df.b) +a 4.0 +b NaN +c 8.0 +d NaN +dtype: float64 + +>>> df.a.rmul(df.b) +a 4.0 +b NaN +c 8.0 +d NaN +dtype: float64 +""" + +_div_example_SERIES = """ +Examples +-------- +>>> df = pp.DataFrame({'a': [2, 2, 4, np.nan], +... 'b': [2, np.nan, 2, np.nan]}, +... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) +>>> df + a b +a 2.0 2.0 +b 2.0 NaN +c 4.0 2.0 +d NaN NaN + +>>> df.a.divide(df.b) +a 1.0 +b NaN +c 2.0 +d NaN +dtype: float64 + +>>> df.a.rdiv(df.b) +a 1.0 +b NaN +c 0.5 +d NaN +dtype: float64 +""" + +_pow_example_SERIES = """ +Examples +-------- +>>> df = pp.DataFrame({'a': [2, 2, 4, np.nan], +... 'b': [2, np.nan, 2, np.nan]}, +... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) +>>> df + a b +a 2.0 2.0 +b 2.0 NaN +c 4.0 2.0 +d NaN NaN + +>>> df.a.pow(df.b) +a 4.0 +b NaN +c 16.0 +d NaN +dtype: float64 + +>>> df.a.rpow(df.b) +a 4.0 +b NaN +c 16.0 +d NaN +dtype: float64 +""" + +_mod_example_SERIES = """ +Examples +-------- +>>> df = pp.DataFrame({'a': [2, 2, 4, np.nan], +... 'b': [2, np.nan, 2, np.nan]}, +... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) +>>> df + a b +a 2.0 2.0 +b 2.0 NaN +c 4.0 2.0 +d NaN NaN + +>>> df.a.mod(df.b) +a 0.0 +b NaN +c 0.0 +d NaN +dtype: float64 + +>>> df.a.rmod(df.b) +a 0.0 +b NaN +c 2.0 +d NaN +dtype: float64 +""" + +_floordiv_example_SERIES = """ +Examples +-------- +>>> df = pp.DataFrame({'a': [2, 2, 4, np.nan], +... 'b': [2, np.nan, 2, np.nan]}, +... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) +>>> df + a b +a 2.0 2.0 +b 2.0 NaN +c 4.0 2.0 +d NaN NaN + +>>> df.a.floordiv(df.b) +a 1.0 +b NaN +c 2.0 +d NaN +dtype: float64 + +>>> df.a.rfloordiv(df.b) +a 1.0 +b NaN +c 0.0 +d NaN +dtype: float64 +""" + +T = TypeVar("T") + +# Needed to disambiguate Series.str and str type +str_type = str + + +def _create_type_for_series_type(param): + from pyspark.pandas.typedef import NameTypeHolder + + if isinstance(param, ExtensionDtype): + new_class = type("NameType", (NameTypeHolder,), {}) + new_class.tpe = param + else: + new_class = param.type if isinstance(param, np.dtype) else param + + return SeriesType[new_class] + + +if (3, 5) <= sys.version_info < (3, 7): + from typing import GenericMeta # type: ignore + + old_getitem = GenericMeta.__getitem__ # type: ignore + + def new_getitem(self, params): + if hasattr(self, "is_series"): + return old_getitem(self, _create_type_for_series_type(params)) + else: + return old_getitem(self, params) + + GenericMeta.__getitem__ = new_getitem # type: ignore + + +class Series(Frame, IndexOpsMixin, Generic[T]): + """ + Koalas Series that corresponds to pandas Series logically. This holds Spark Column + internally. + + :ivar _internal: an internal immutable Frame to manage metadata. + :type _internal: InternalFrame + :ivar _kdf: Parent's Koalas DataFrame + :type _kdf: pp.DataFrame + + Parameters + ---------- + data : array-like, dict, or scalar value, pandas Series + Contains data stored in Series + If data is a dict, argument order is maintained for Python 3.6 + and later. + Note that if `data` is a pandas Series, other arguments should not be used. + index : array-like or Index (1d) + Values must be hashable and have the same length as `data`. + Non-unique index values are allowed. Will default to + RangeIndex (0, 1, 2, ..., n) if not provided. If both a dict and index + sequence are used, the index will override the keys found in the + dict. + dtype : numpy.dtype or None + If None, dtype will be inferred + copy : boolean, default False + Copy input data + """ + + def __init__(self, data=None, index=None, dtype=None, name=None, copy=False, fastpath=False): + assert data is not None + + if isinstance(data, DataFrame): + assert dtype is None + assert name is None + assert not copy + assert not fastpath + + self._anchor = data + self._col_label = index + else: + if isinstance(data, pd.Series): + assert index is None + assert dtype is None + assert name is None + assert not copy + assert not fastpath + s = data + else: + s = pd.Series( + data=data, index=index, dtype=dtype, name=name, copy=copy, fastpath=fastpath + ) + internal = InternalFrame.from_pandas(pd.DataFrame(s)) + if s.name is None: + internal = internal.copy(column_labels=[None]) + anchor = DataFrame(internal) + + self._anchor = anchor + self._col_label = anchor._internal.column_labels[0] + object.__setattr__(anchor, "_kseries", {self._column_label: self}) + + @property + def _kdf(self) -> DataFrame: + return self._anchor + + @property + def _internal(self) -> InternalFrame: + return self._kdf._internal.select_column(self._column_label) + + @property + def _column_label(self) -> Tuple: + return self._col_label + + def _update_anchor(self, kdf: DataFrame): + assert kdf._internal.column_labels == [self._column_label], ( + kdf._internal.column_labels, + [self._column_label], + ) + self._anchor = kdf + object.__setattr__(kdf, "_kseries", {self._column_label: self}) + + def _with_new_scol(self, scol: spark.Column, *, dtype=None) -> "Series": + """ + Copy Koalas Series with the new Spark Column. + + :param scol: the new Spark Column + :return: the copied Series + """ + internal = self._internal.copy( + data_spark_columns=[scol.alias(name_like_string(self._column_label))], + data_dtypes=[dtype], + ) + return first_series(DataFrame(internal)) + + spark = CachedAccessor("spark", SparkSeriesMethods) + + @property + def dtypes(self) -> np.dtype: + """Return the dtype object of the underlying data. + + >>> s = pp.Series(list('abc')) + >>> s.dtype == s.dtypes + True + """ + return self.dtype + + @property + def axes(self) -> List: + """ + Return a list of the row axis labels. + + Examples + -------- + + >>> kser = pp.Series([1, 2, 3]) + >>> kser.axes + [Int64Index([0, 1, 2], dtype='int64')] + """ + return [self.index] + + @property + def spark_type(self): + warnings.warn( + "Series.spark_type is deprecated as of Series.spark.data_type. " + "Please use the API instead.", + FutureWarning, + ) + return self.spark.data_type + + spark_type.__doc__ = SparkSeriesMethods.data_type.__doc__ + + # Arithmetic Operators + def add(self, other) -> "Series": + return self + other + + add.__doc__ = _flex_doc_SERIES.format( + desc="Addition", + op_name="+", + equiv="series + other", + reverse="radd", + series_examples=_add_example_SERIES, + ) + + def radd(self, other) -> "Series": + return other + self + + radd.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Addition", + op_name="+", + equiv="other + series", + reverse="add", + series_examples=_add_example_SERIES, + ) + + def div(self, other) -> "Series": + return self / other + + div.__doc__ = _flex_doc_SERIES.format( + desc="Floating division", + op_name="/", + equiv="series / other", + reverse="rdiv", + series_examples=_div_example_SERIES, + ) + + divide = div + + def rdiv(self, other) -> "Series": + return other / self + + rdiv.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Floating division", + op_name="/", + equiv="other / series", + reverse="div", + series_examples=_div_example_SERIES, + ) + + def truediv(self, other) -> "Series": + return self / other + + truediv.__doc__ = _flex_doc_SERIES.format( + desc="Floating division", + op_name="/", + equiv="series / other", + reverse="rtruediv", + series_examples=_div_example_SERIES, + ) + + def rtruediv(self, other) -> "Series": + return other / self + + rtruediv.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Floating division", + op_name="/", + equiv="other / series", + reverse="truediv", + series_examples=_div_example_SERIES, + ) + + def mul(self, other) -> "Series": + return self * other + + mul.__doc__ = _flex_doc_SERIES.format( + desc="Multiplication", + op_name="*", + equiv="series * other", + reverse="rmul", + series_examples=_mul_example_SERIES, + ) + + multiply = mul + + def rmul(self, other) -> "Series": + return other * self + + rmul.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Multiplication", + op_name="*", + equiv="other * series", + reverse="mul", + series_examples=_mul_example_SERIES, + ) + + def sub(self, other) -> "Series": + return self - other + + sub.__doc__ = _flex_doc_SERIES.format( + desc="Subtraction", + op_name="-", + equiv="series - other", + reverse="rsub", + series_examples=_sub_example_SERIES, + ) + + subtract = sub + + def rsub(self, other) -> "Series": + return other - self + + rsub.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Subtraction", + op_name="-", + equiv="other - series", + reverse="sub", + series_examples=_sub_example_SERIES, + ) + + def mod(self, other) -> "Series": + return self % other + + mod.__doc__ = _flex_doc_SERIES.format( + desc="Modulo", + op_name="%", + equiv="series % other", + reverse="rmod", + series_examples=_mod_example_SERIES, + ) + + def rmod(self, other) -> "Series": + return other % self + + rmod.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Modulo", + op_name="%", + equiv="other % series", + reverse="mod", + series_examples=_mod_example_SERIES, + ) + + def pow(self, other) -> "Series": + return self ** other + + pow.__doc__ = _flex_doc_SERIES.format( + desc="Exponential power of series", + op_name="**", + equiv="series ** other", + reverse="rpow", + series_examples=_pow_example_SERIES, + ) + + def rpow(self, other) -> "Series": + return other ** self + + rpow.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Exponential power", + op_name="**", + equiv="other ** series", + reverse="pow", + series_examples=_pow_example_SERIES, + ) + + def floordiv(self, other) -> "Series": + return self // other + + floordiv.__doc__ = _flex_doc_SERIES.format( + desc="Integer division", + op_name="//", + equiv="series // other", + reverse="rfloordiv", + series_examples=_floordiv_example_SERIES, + ) + + def rfloordiv(self, other) -> "Series": + return other // self + + rfloordiv.__doc__ = _flex_doc_SERIES.format( + desc="Reverse Integer division", + op_name="//", + equiv="other // series", + reverse="floordiv", + series_examples=_floordiv_example_SERIES, + ) + + # create accessor for Koalas specific methods. + koalas = CachedAccessor("koalas", KoalasSeriesMethods) + + # Comparison Operators + def eq(self, other) -> bool: + """ + Compare if the current value is equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.a == 1 + a True + b False + c False + d False + Name: a, dtype: bool + + >>> df.b.eq(1) + a True + b False + c True + d False + Name: b, dtype: bool + """ + return self == other + + equals = eq + + def gt(self, other) -> "Series": + """ + Compare if the current value is greater than the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.a > 1 + a False + b True + c True + d True + Name: a, dtype: bool + + >>> df.b.gt(1) + a False + b False + c False + d False + Name: b, dtype: bool + """ + return self > other + + def ge(self, other) -> "Series": + """ + Compare if the current value is greater than or equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.a >= 2 + a False + b True + c True + d True + Name: a, dtype: bool + + >>> df.b.ge(2) + a False + b False + c False + d False + Name: b, dtype: bool + """ + return self >= other + + def lt(self, other) -> "Series": + """ + Compare if the current value is less than the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.a < 1 + a False + b False + c False + d False + Name: a, dtype: bool + + >>> df.b.lt(2) + a True + b False + c True + d False + Name: b, dtype: bool + """ + return self < other + + def le(self, other) -> "Series": + """ + Compare if the current value is less than or equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.a <= 2 + a True + b True + c False + d False + Name: a, dtype: bool + + >>> df.b.le(2) + a True + b False + c True + d False + Name: b, dtype: bool + """ + return self <= other + + def ne(self, other) -> "Series": + """ + Compare if the current value is not equal to the other. + + >>> df = pp.DataFrame({'a': [1, 2, 3, 4], + ... 'b': [1, np.nan, 1, np.nan]}, + ... index=['a', 'b', 'c', 'd'], columns=['a', 'b']) + + >>> df.a != 1 + a False + b True + c True + d True + Name: a, dtype: bool + + >>> df.b.ne(1) + a False + b True + c False + d True + Name: b, dtype: bool + """ + return self != other + + def divmod(self, other) -> Tuple["Series", "Series"]: + """ + Return Integer division and modulo of series and other, element-wise + (binary operator `divmod`). + + Parameters + ---------- + other : Series or scalar value + + Returns + ------- + 2-Tuple of Series + The result of the operation. + + See Also + -------- + Series.rdivmod + """ + return (self.floordiv(other), self.mod(other)) + + def rdivmod(self, other) -> Tuple["Series", "Series"]: + """ + Return Integer division and modulo of series and other, element-wise + (binary operator `rdivmod`). + + Parameters + ---------- + other : Series or scalar value + + Returns + ------- + 2-Tuple of Series + The result of the operation. + + See Also + -------- + Series.divmod + """ + return (self.rfloordiv(other), self.rmod(other)) + + def between(self, left, right, inclusive=True) -> "Series": + """ + Return boolean Series equivalent to left <= series <= right. + This function returns a boolean vector containing `True` wherever the + corresponding Series element is between the boundary values `left` and + `right`. NA values are treated as `False`. + + Parameters + ---------- + left : scalar or list-like + Left boundary. + right : scalar or list-like + Right boundary. + inclusive : bool, default True + Include boundaries. + + Returns + ------- + Series + Series representing whether each element is between left and + right (inclusive). + + See Also + -------- + Series.gt : Greater than of series and other. + Series.lt : Less than of series and other. + + Notes + ----- + This function is equivalent to ``(left <= ser) & (ser <= right)`` + + Examples + -------- + >>> s = pp.Series([2, 0, 4, 8, np.nan]) + + Boundary values are included by default: + + >>> s.between(1, 4) + 0 True + 1 False + 2 True + 3 False + 4 False + dtype: bool + + With `inclusive` set to ``False`` boundary values are excluded: + + >>> s.between(1, 4, inclusive=False) + 0 True + 1 False + 2 False + 3 False + 4 False + dtype: bool + + `left` and `right` can be any scalar value: + + >>> s = pp.Series(['Alice', 'Bob', 'Carol', 'Eve']) + >>> s.between('Anna', 'Daniel') + 0 False + 1 True + 2 True + 3 False + dtype: bool + """ + if inclusive: + lmask = self >= left + rmask = self <= right + else: + lmask = self > left + rmask = self < right + + return lmask & rmask + + # TODO: arg should support Series + # TODO: NaN and None + def map(self, arg) -> "Series": + """ + Map values of Series according to input correspondence. + + Used for substituting each value in a Series with another value, + that may be derived from a function, a ``dict``. + + .. note:: make sure the size of the dictionary is not huge because it could + downgrade the performance or throw OutOfMemoryError due to a huge + expression within Spark. Consider the input as a functions as an + alternative instead in this case. + + Parameters + ---------- + arg : function or dict + Mapping correspondence. + + Returns + ------- + Series + Same index as caller. + + See Also + -------- + Series.apply : For applying more complex functions on a Series. + DataFrame.applymap : Apply a function elementwise on a whole DataFrame. + + Notes + ----- + When ``arg`` is a dictionary, values in Series that are not in the + dictionary (as keys) are converted to ``None``. However, if the + dictionary is a ``dict`` subclass that defines ``__missing__`` (i.e. + provides a method for default values), then this default is used + rather than ``None``. + + Examples + -------- + >>> s = pp.Series(['cat', 'dog', None, 'rabbit']) + >>> s + 0 cat + 1 dog + 2 None + 3 rabbit + dtype: object + + ``map`` accepts a ``dict``. Values that are not found + in the ``dict`` are converted to ``None``, unless the dict has a default + value (e.g. ``defaultdict``): + + >>> s.map({'cat': 'kitten', 'dog': 'puppy'}) + 0 kitten + 1 puppy + 2 None + 3 None + dtype: object + + It also accepts a function: + + >>> def format(x) -> str: + ... return 'I am a {}'.format(x) + + >>> s.map(format) + 0 I am a cat + 1 I am a dog + 2 I am a None + 3 I am a rabbit + dtype: object + """ + if isinstance(arg, dict): + is_start = True + # In case dictionary is empty. + current = F.when(F.lit(False), F.lit(None).cast(self.spark.data_type)) + + for to_replace, value in arg.items(): + if is_start: + current = F.when(self.spark.column == F.lit(to_replace), value) + is_start = False + else: + current = current.when(self.spark.column == F.lit(to_replace), value) + + if hasattr(arg, "__missing__"): + tmp_val = arg[np._NoValue] + del arg[np._NoValue] # Remove in case it's set in defaultdict. + current = current.otherwise(F.lit(tmp_val)) + else: + current = current.otherwise(F.lit(None).cast(self.spark.data_type)) + return self._with_new_scol(current) + else: + return self.apply(arg) + + def alias(self, name) -> "Series": + """An alias for :meth:`Series.rename`.""" + warnings.warn( + "Series.alias is deprecated as of Series.rename. Please use the API instead.", + FutureWarning, + ) + return self.rename(name) + + @property + def shape(self): + """Return a tuple of the shape of the underlying data.""" + return (len(self),) + + @property + def name(self) -> Union[Any, Tuple]: + """Return name of the Series.""" + name = self._column_label + if name is not None and len(name) == 1: + return name[0] + else: + return name + + @name.setter + def name(self, name: Union[Any, Tuple]): + self.rename(name, inplace=True) + + # TODO: Functionality and documentation should be matched. Currently, changing index labels + # taking dictionary and function to change index are not supported. + def rename(self, index=None, **kwargs) -> "Series": + """ + Alter Series name. + + Parameters + ---------- + index : scalar + Scalar will alter the ``Series.name`` attribute. + + inplace : bool, default False + Whether to return a new Series. If True then value of copy is + ignored. + + Returns + ------- + Series + Series with name altered. + + Examples + -------- + + >>> s = pp.Series([1, 2, 3]) + >>> s + 0 1 + 1 2 + 2 3 + dtype: int64 + + >>> s.rename("my_name") # scalar, changes Series.name + 0 1 + 1 2 + 2 3 + Name: my_name, dtype: int64 + """ + if index is None: + pass + elif not is_hashable(index): + raise TypeError("Series.name must be a hashable type") + elif not isinstance(index, tuple): + index = (index,) + scol = self.spark.column.alias(name_like_string(index)) + + internal = self._internal.copy( + column_labels=[index], data_spark_columns=[scol], column_label_names=None + ) + kdf = DataFrame(internal) # type: DataFrame + + if kwargs.get("inplace", False): + self._col_label = index + self._update_anchor(kdf) + return self + else: + return first_series(kdf) + + def rename_axis( + self, mapper: Optional[Any] = None, index: Optional[Any] = None, inplace: bool = False + ) -> Optional["Series"]: + """ + Set the name of the axis for the index or columns. + + Parameters + ---------- + mapper, index : scalar, list-like, dict-like or function, optional + A scalar, list-like, dict-like or functions transformations to + apply to the index values. + inplace : bool, default False + Modifies the object directly, instead of creating a new Series. + + Returns + ------- + Series, or None if `inplace` is True. + + See Also + -------- + Series.rename : Alter Series index labels or name. + DataFrame.rename : Alter DataFrame index labels or name. + Index.rename : Set new names on index. + + Examples + -------- + >>> s = pp.Series(["dog", "cat", "monkey"], name="animal") + >>> s # doctest: +NORMALIZE_WHITESPACE + 0 dog + 1 cat + 2 monkey + Name: animal, dtype: object + >>> s.rename_axis("index").sort_index() # doctest: +NORMALIZE_WHITESPACE + index + 0 dog + 1 cat + 2 monkey + Name: animal, dtype: object + + **MultiIndex** + + >>> index = pd.MultiIndex.from_product([['mammal'], + ... ['dog', 'cat', 'monkey']], + ... names=['type', 'name']) + >>> s = pp.Series([4, 4, 2], index=index, name='num_legs') + >>> s # doctest: +NORMALIZE_WHITESPACE + type name + mammal dog 4 + cat 4 + monkey 2 + Name: num_legs, dtype: int64 + >>> s.rename_axis(index={'type': 'class'}).sort_index() # doctest: +NORMALIZE_WHITESPACE + class name + mammal cat 4 + dog 4 + monkey 2 + Name: num_legs, dtype: int64 + >>> s.rename_axis(index=str.upper).sort_index() # doctest: +NORMALIZE_WHITESPACE + TYPE NAME + mammal cat 4 + dog 4 + monkey 2 + Name: num_legs, dtype: int64 + """ + kdf = self.to_frame().rename_axis(mapper=mapper, index=index, inplace=False) + if inplace: + self._update_anchor(kdf) + return None + else: + return first_series(kdf) + + @property + def index(self) -> "pp.Index": + """The index (axis labels) Column of the Series. + + See Also + -------- + Index + """ + return self._kdf.index + + @property + def is_unique(self) -> bool: + """ + Return boolean if values in the object are unique + + Returns + ------- + is_unique : boolean + + >>> pp.Series([1, 2, 3]).is_unique + True + >>> pp.Series([1, 2, 2]).is_unique + False + >>> pp.Series([1, 2, 3, None]).is_unique + True + """ + scol = self.spark.column + + # Here we check: + # 1. the distinct count without nulls and count without nulls for non-null values + # 2. count null values and see if null is a distinct value. + # + # This workaround is in order to calculate the distinct count including nulls in + # single pass. Note that COUNT(DISTINCT expr) in Spark is designed to ignore nulls. + return self._internal.spark_frame.select( + (F.count(scol) == F.countDistinct(scol)) + & (F.count(F.when(scol.isNull(), 1).otherwise(None)) <= 1) + ).collect()[0][0] + + def reset_index( + self, level=None, drop=False, name=None, inplace=False + ) -> Optional[Union["Series", DataFrame]]: + """ + Generate a new DataFrame or Series with the index reset. + + This is useful when the index needs to be treated as a column, + or when the index is meaningless and needs to be reset + to the default before another operation. + + Parameters + ---------- + level : int, str, tuple, or list, default optional + For a Series with a MultiIndex, only remove the specified levels from the index. + Removes all levels by default. + drop : bool, default False + Just reset the index, without inserting it as a column in the new DataFrame. + name : object, optional + The name to use for the column containing the original Series values. + Uses self.name by default. This argument is ignored when drop is True. + inplace : bool, default False + Modify the Series in place (do not create a new object). + + Returns + ------- + Series or DataFrame + When `drop` is False (the default), a DataFrame is returned. + The newly created columns will come first in the DataFrame, + followed by the original Series values. + When `drop` is True, a `Series` is returned. + In either case, if ``inplace=True``, no value is returned. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4], index=pd.Index(['a', 'b', 'c', 'd'], name='idx')) + + Generate a DataFrame with default index. + + >>> s.reset_index() + idx 0 + 0 a 1 + 1 b 2 + 2 c 3 + 3 d 4 + + To specify the name of the new column use `name`. + + >>> s.reset_index(name='values') + idx values + 0 a 1 + 1 b 2 + 2 c 3 + 3 d 4 + + To generate a new Series with the default set `drop` to True. + + >>> s.reset_index(drop=True) + 0 1 + 1 2 + 2 3 + 3 4 + dtype: int64 + + To update the Series in place, without generating a new one + set `inplace` to True. Note that it also requires ``drop=True``. + + >>> s.reset_index(inplace=True, drop=True) + >>> s + 0 1 + 1 2 + 2 3 + 3 4 + dtype: int64 + """ + inplace = validate_bool_kwarg(inplace, "inplace") + if inplace and not drop: + raise TypeError("Cannot reset_index inplace on a Series to create a DataFrame") + + if drop: + kdf = self._kdf[[self.name]] + else: + kser = self + if name is not None: + kser = kser.rename(name) + kdf = kser.to_frame() + kdf = kdf.reset_index(level=level, drop=drop) + if drop: + if inplace: + self._update_anchor(kdf) + return None + else: + return first_series(kdf) + else: + return kdf + + def to_frame(self, name: Union[Any, Tuple] = None) -> DataFrame: + """ + Convert Series to DataFrame. + + Parameters + ---------- + name : object, default None + The passed name should substitute for the series name (if it has + one). + + Returns + ------- + DataFrame + DataFrame representation of Series. + + Examples + -------- + >>> s = pp.Series(["a", "b", "c"]) + >>> s.to_frame() + 0 + 0 a + 1 b + 2 c + + >>> s = pp.Series(["a", "b", "c"], name="vals") + >>> s.to_frame() + vals + 0 a + 1 b + 2 c + """ + if name is not None: + renamed = self.rename(name) + elif self._column_label is None: + renamed = self.rename(DEFAULT_SERIES_NAME) + else: + renamed = self + return DataFrame(renamed._internal) + + to_dataframe = to_frame + + def to_string( + self, + buf=None, + na_rep="NaN", + float_format=None, + header=True, + index=True, + length=False, + dtype=False, + name=False, + max_rows=None, + ) -> str: + """ + Render a string representation of the Series. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. If the input + is large, set max_rows parameter. + + Parameters + ---------- + buf : StringIO-like, optional + buffer to write to + na_rep : string, optional + string representation of NAN to use, default 'NaN' + float_format : one-parameter function, optional + formatter function to apply to columns' elements if they are floats + default None + header : boolean, default True + Add the Series header (index name) + index : bool, optional + Add index (row) labels, default True + length : boolean, default False + Add the Series length + dtype : boolean, default False + Add the Series dtype + name : boolean, default False + Add the Series name if not None + max_rows : int, optional + Maximum number of rows to show before truncating. If None, show + all. + + Returns + ------- + formatted : string (if not buffer passed) + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], columns=['dogs', 'cats']) + >>> print(df['dogs'].to_string()) + 0 0.2 + 1 0.0 + 2 0.6 + 3 0.2 + + >>> print(df['dogs'].to_string(max_rows=2)) + 0 0.2 + 1 0.0 + """ + # Make sure locals() call is at the top of the function so we don't capture local variables. + args = locals() + if max_rows is not None: + kseries = self.head(max_rows) + else: + kseries = self + + return validate_arguments_and_invoke_function( + kseries._to_internal_pandas(), self.to_string, pd.Series.to_string, args + ) + + def to_clipboard(self, excel=True, sep=None, **kwargs) -> None: + # Docstring defined below by reusing DataFrame.to_clipboard's. + args = locals() + kseries = self + + return validate_arguments_and_invoke_function( + kseries._to_internal_pandas(), self.to_clipboard, pd.Series.to_clipboard, args + ) + + to_clipboard.__doc__ = DataFrame.to_clipboard.__doc__ + + def to_dict(self, into=dict) -> Mapping: + """ + Convert Series to {label -> value} dict or dict-like object. + + .. note:: This method should only be used if the resulting pandas DataFrame is expected + to be small, as all the data is loaded into the driver's memory. + + Parameters + ---------- + into : class, default dict + The collections.abc.Mapping subclass to use as the return + object. Can be the actual class or an empty + instance of the mapping type you want. If you want a + collections.defaultdict, you must pass it initialized. + + Returns + ------- + collections.abc.Mapping + Key-value representation of Series. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4]) + >>> s_dict = s.to_dict() + >>> sorted(s_dict.items()) + [(0, 1), (1, 2), (2, 3), (3, 4)] + + >>> from collections import OrderedDict, defaultdict + >>> s.to_dict(OrderedDict) + OrderedDict([(0, 1), (1, 2), (2, 3), (3, 4)]) + + >>> dd = defaultdict(list) + >>> s.to_dict(dd) # doctest: +ELLIPSIS + defaultdict(, {...}) + """ + # Make sure locals() call is at the top of the function so we don't capture local variables. + args = locals() + kseries = self + return validate_arguments_and_invoke_function( + kseries._to_internal_pandas(), self.to_dict, pd.Series.to_dict, args + ) + + def to_latex( + self, + buf=None, + columns=None, + col_space=None, + header=True, + index=True, + na_rep="NaN", + formatters=None, + float_format=None, + sparsify=None, + index_names=True, + bold_rows=False, + column_format=None, + longtable=None, + escape=None, + encoding=None, + decimal=".", + multicolumn=None, + multicolumn_format=None, + multirow=None, + ) -> Optional[str]: + + args = locals() + kseries = self + return validate_arguments_and_invoke_function( + kseries._to_internal_pandas(), self.to_latex, pd.Series.to_latex, args + ) + + to_latex.__doc__ = DataFrame.to_latex.__doc__ + + def to_pandas(self) -> pd.Series: + """ + Return a pandas Series. + + .. note:: This method should only be used if the resulting pandas object is expected + to be small, as all the data is loaded into the driver's memory. + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], columns=['dogs', 'cats']) + >>> df['dogs'].to_pandas() + 0 0.2 + 1 0.0 + 2 0.6 + 3 0.2 + Name: dogs, dtype: float64 + """ + return self._to_internal_pandas().copy() + + # Alias to maintain backward compatibility with Spark + def toPandas(self) -> pd.Series: + warnings.warn( + "Series.toPandas is deprecated as of Series.to_pandas. Please use the API instead.", + FutureWarning, + ) + return self.to_pandas() + + toPandas.__doc__ = to_pandas.__doc__ + + def to_list(self) -> List: + """ + Return a list of the values. + + These are each a scalar type, which is a Python scalar + (for str, int, float) or a pandas scalar + (for Timestamp/Timedelta/Interval/Period) + + .. note:: This method should only be used if the resulting list is expected + to be small, as all the data is loaded into the driver's memory. + + """ + return self._to_internal_pandas().tolist() + + tolist = to_list + + def drop_duplicates(self, keep="first", inplace=False) -> Optional["Series"]: + """ + Return Series with duplicate values removed. + + Parameters + ---------- + keep : {'first', 'last', ``False``}, default 'first' + Method to handle dropping duplicates: + - 'first' : Drop duplicates except for the first occurrence. + - 'last' : Drop duplicates except for the last occurrence. + - ``False`` : Drop all duplicates. + inplace : bool, default ``False`` + If ``True``, performs operation inplace and returns None. + + Returns + ------- + Series + Series with duplicates dropped. + + Examples + -------- + Generate a Series with duplicated entries. + + >>> s = pp.Series(['lama', 'cow', 'lama', 'beetle', 'lama', 'hippo'], + ... name='animal') + >>> s.sort_index() + 0 lama + 1 cow + 2 lama + 3 beetle + 4 lama + 5 hippo + Name: animal, dtype: object + + With the 'keep' parameter, the selection behaviour of duplicated values + can be changed. The value 'first' keeps the first occurrence for each + set of duplicated entries. The default value of keep is 'first'. + + >>> s.drop_duplicates().sort_index() + 0 lama + 1 cow + 3 beetle + 5 hippo + Name: animal, dtype: object + + The value 'last' for parameter 'keep' keeps the last occurrence for + each set of duplicated entries. + + >>> s.drop_duplicates(keep='last').sort_index() + 1 cow + 3 beetle + 4 lama + 5 hippo + Name: animal, dtype: object + + The value ``False`` for parameter 'keep' discards all sets of + duplicated entries. Setting the value of 'inplace' to ``True`` performs + the operation inplace and returns ``None``. + + >>> s.drop_duplicates(keep=False, inplace=True) + >>> s.sort_index() + 1 cow + 3 beetle + 5 hippo + Name: animal, dtype: object + """ + inplace = validate_bool_kwarg(inplace, "inplace") + kdf = self._kdf[[self.name]].drop_duplicates(keep=keep) + + if inplace: + self._update_anchor(kdf) + return None + else: + return first_series(kdf) + + def reindex(self, index: Optional[Any] = None, fill_value: Optional[Any] = None,) -> "Series": + """ + Conform Series to new index with optional filling logic, placing + NA/NaN in locations having no value in the previous index. A new object + is produced. + + Parameters + ---------- + index: array-like, optional + New labels / index to conform to, should be specified using keywords. + Preferably an Index object to avoid duplicating data + fill_value : scalar, default np.NaN + Value to use for missing values. Defaults to NaN, but can be any + "compatible" value. + + Returns + ------- + Series with changed index. + + See Also + -------- + Series.reset_index : Remove row labels or move them to new columns. + + Examples + -------- + + Create a series with some fictional data. + + >>> index = ['Firefox', 'Chrome', 'Safari', 'IE10', 'Konqueror'] + >>> ser = pp.Series([200, 200, 404, 404, 301], + ... index=index, name='http_status') + >>> ser + Firefox 200 + Chrome 200 + Safari 404 + IE10 404 + Konqueror 301 + Name: http_status, dtype: int64 + + Create a new index and reindex the Series. By default + values in the new index that do not have corresponding + records in the Series are assigned ``NaN``. + + >>> new_index= ['Safari', 'Iceweasel', 'Comodo Dragon', 'IE10', + ... 'Chrome'] + >>> ser.reindex(new_index).sort_index() + Chrome 200.0 + Comodo Dragon NaN + IE10 404.0 + Iceweasel NaN + Safari 404.0 + Name: http_status, dtype: float64 + + We can fill in the missing values by passing a value to + the keyword ``fill_value``. + + >>> ser.reindex(new_index, fill_value=0).sort_index() + Chrome 200 + Comodo Dragon 0 + IE10 404 + Iceweasel 0 + Safari 404 + Name: http_status, dtype: int64 + + To further illustrate the filling functionality in + ``reindex``, we will create a Series with a + monotonically increasing index (for example, a sequence + of dates). + + >>> date_index = pd.date_range('1/1/2010', periods=6, freq='D') + >>> ser2 = pp.Series([100, 101, np.nan, 100, 89, 88], + ... name='prices', index=date_index) + >>> ser2.sort_index() + 2010-01-01 100.0 + 2010-01-02 101.0 + 2010-01-03 NaN + 2010-01-04 100.0 + 2010-01-05 89.0 + 2010-01-06 88.0 + Name: prices, dtype: float64 + + Suppose we decide to expand the series to cover a wider + date range. + + >>> date_index2 = pd.date_range('12/29/2009', periods=10, freq='D') + >>> ser2.reindex(date_index2).sort_index() + 2009-12-29 NaN + 2009-12-30 NaN + 2009-12-31 NaN + 2010-01-01 100.0 + 2010-01-02 101.0 + 2010-01-03 NaN + 2010-01-04 100.0 + 2010-01-05 89.0 + 2010-01-06 88.0 + 2010-01-07 NaN + Name: prices, dtype: float64 + """ + + return first_series(self.to_frame().reindex(index=index, fill_value=fill_value)).rename( + self.name + ) + + def reindex_like(self, other: Union["Series", "DataFrame"]) -> "Series": + """ + Return a Series with matching indices as other object. + + Conform the object to the same index on all axes. Places NA/NaN in locations + having no value in the previous index. + + Parameters + ---------- + other : Series or DataFrame + Its row and column indices are used to define the new indices + of this object. + + Returns + ------- + Series + Series with changed indices on each axis. + + See Also + -------- + DataFrame.set_index : Set row labels. + DataFrame.reset_index : Remove row labels or move them to new columns. + DataFrame.reindex : Change to new indices or expand indices. + + Notes + ----- + Same as calling + ``.reindex(index=other.index, ...)``. + + Examples + -------- + + >>> s1 = pp.Series([24.3, 31.0, 22.0, 35.0], + ... index=pd.date_range(start='2014-02-12', + ... end='2014-02-15', freq='D'), + ... name="temp_celsius") + >>> s1 + 2014-02-12 24.3 + 2014-02-13 31.0 + 2014-02-14 22.0 + 2014-02-15 35.0 + Name: temp_celsius, dtype: float64 + + >>> s2 = pp.Series(["low", "low", "medium"], + ... index=pd.DatetimeIndex(['2014-02-12', '2014-02-13', + ... '2014-02-15']), + ... name="winspeed") + >>> s2 + 2014-02-12 low + 2014-02-13 low + 2014-02-15 medium + Name: winspeed, dtype: object + + >>> s2.reindex_like(s1).sort_index() + 2014-02-12 low + 2014-02-13 low + 2014-02-14 None + 2014-02-15 medium + Name: winspeed, dtype: object + """ + if isinstance(other, (Series, DataFrame)): + return self.reindex(index=other.index) + else: + raise TypeError("other must be a Koalas Series or DataFrame") + + def fillna( + self, value=None, method=None, axis=None, inplace=False, limit=None + ) -> Optional["Series"]: + """Fill NA/NaN values. + + .. note:: the current implementation of 'method' parameter in fillna uses Spark's Window + without specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + value : scalar, dict, Series + Value to use to fill holes. alternately a dict/Series of values + specifying which value to use for each column. + DataFrame is not supported. + method : {'backfill', 'bfill', 'pad', 'ffill', None}, default None + Method to use for filling holes in reindexed Series pad / ffill: propagate last valid + observation forward to next valid backfill / bfill: + use NEXT valid observation to fill gap + axis : {0 or `index`} + 1 and `columns` are not supported. + inplace : boolean, default False + Fill in place (do not create a new object) + limit : int, default None + If method is specified, this is the maximum number of consecutive NaN values to + forward/backward fill. In other words, if there is a gap with more than this number of + consecutive NaNs, it will only be partially filled. If method is not specified, + this is the maximum number of entries along the entire axis where NaNs will be filled. + Must be greater than 0 if not None + + Returns + ------- + Series + Series with NA entries filled. + + Examples + -------- + >>> s = pp.Series([np.nan, 2, 3, 4, np.nan, 6], name='x') + >>> s + 0 NaN + 1 2.0 + 2 3.0 + 3 4.0 + 4 NaN + 5 6.0 + Name: x, dtype: float64 + + Replace all NaN elements with 0s. + + >>> s.fillna(0) + 0 0.0 + 1 2.0 + 2 3.0 + 3 4.0 + 4 0.0 + 5 6.0 + Name: x, dtype: float64 + + We can also propagate non-null values forward or backward. + + >>> s.fillna(method='ffill') + 0 NaN + 1 2.0 + 2 3.0 + 3 4.0 + 4 4.0 + 5 6.0 + Name: x, dtype: float64 + + >>> s = pp.Series([np.nan, 'a', 'b', 'c', np.nan], name='x') + >>> s.fillna(method='ffill') + 0 None + 1 a + 2 b + 3 c + 4 c + Name: x, dtype: object + """ + kser = self._fillna(value=value, method=method, axis=axis, limit=limit) + + if method is not None: + kser = DataFrame(kser._kdf._internal.resolved_copy)._kser_for(self._column_label) + + inplace = validate_bool_kwarg(inplace, "inplace") + if inplace: + self._kdf._update_internal_frame(kser._kdf._internal, requires_same_anchor=False) + return None + else: + return kser._with_new_scol(kser.spark.column) # TODO: dtype? + + def _fillna(self, value=None, method=None, axis=None, limit=None, part_cols=()): + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError("fillna currently only works for axis=0 or axis='index'") + if (value is None) and (method is None): + raise ValueError("Must specify a fillna 'value' or 'method' parameter.") + if (method is not None) and (method not in ["ffill", "pad", "backfill", "bfill"]): + raise ValueError("Expecting 'pad', 'ffill', 'backfill' or 'bfill'.") + + scol = self.spark.column + + if isinstance(self.spark.data_type, (FloatType, DoubleType)): + cond = scol.isNull() | F.isnan(scol) + else: + if not self.spark.nullable: + return self.copy() + cond = scol.isNull() + + if value is not None: + if not isinstance(value, (float, int, str, bool)): + raise TypeError("Unsupported type %s" % type(value).__name__) + if limit is not None: + raise ValueError("limit parameter for value is not support now") + scol = F.when(cond, value).otherwise(scol) + else: + if method in ["ffill", "pad"]: + func = F.last + end = Window.currentRow - 1 + if limit is not None: + begin = Window.currentRow - limit + else: + begin = Window.unboundedPreceding + elif method in ["bfill", "backfill"]: + func = F.first + begin = Window.currentRow + 1 + if limit is not None: + end = Window.currentRow + limit + else: + end = Window.unboundedFollowing + + window = ( + Window.partitionBy(*part_cols) + .orderBy(NATURAL_ORDER_COLUMN_NAME) + .rowsBetween(begin, end) + ) + scol = F.when(cond, func(scol, True).over(window)).otherwise(scol) + + return DataFrame( + self._kdf._internal.with_new_spark_column( + self._column_label, scol.alias(name_like_string(self.name)) # TODO: dtype? + ) + )._kser_for(self._column_label) + + def dropna(self, axis=0, inplace=False, **kwargs) -> Optional["Series"]: + """ + Return a new Series with missing values removed. + + Parameters + ---------- + axis : {0 or 'index'}, default 0 + There is only one axis to drop values from. + inplace : bool, default False + If True, do operation inplace and return None. + **kwargs + Not in use. + + Returns + ------- + Series + Series with NA entries dropped from it. + + Examples + -------- + >>> ser = pp.Series([1., 2., np.nan]) + >>> ser + 0 1.0 + 1 2.0 + 2 NaN + dtype: float64 + + Drop NA values from a Series. + + >>> ser.dropna() + 0 1.0 + 1 2.0 + dtype: float64 + + Keep the Series with valid entries in the same variable. + + >>> ser.dropna(inplace=True) + >>> ser + 0 1.0 + 1 2.0 + dtype: float64 + """ + inplace = validate_bool_kwarg(inplace, "inplace") + # TODO: last two examples from pandas produce different results. + kdf = self._kdf[[self.name]].dropna(axis=axis, inplace=False) + if inplace: + self._update_anchor(kdf) + return None + else: + return first_series(kdf) + + def clip(self, lower: Union[float, int] = None, upper: Union[float, int] = None) -> "Series": + """ + Trim values at input threshold(s). + + Assigns values outside boundary to boundary values. + + Parameters + ---------- + lower : float or int, default None + Minimum threshold value. All values below this threshold will be set to it. + upper : float or int, default None + Maximum threshold value. All values above this threshold will be set to it. + + Returns + ------- + Series + Series with the values outside the clip boundaries replaced + + Examples + -------- + >>> pp.Series([0, 2, 4]).clip(1, 3) + 0 1 + 1 2 + 2 3 + dtype: int64 + + Notes + ----- + One difference between this implementation and pandas is that running + `pd.Series(['a', 'b']).clip(0, 1)` will crash with "TypeError: '<=' not supported between + instances of 'str' and 'int'" while `pp.Series(['a', 'b']).clip(0, 1)` will output the + original Series, simply ignoring the incompatible types. + """ + if is_list_like(lower) or is_list_like(upper): + raise ValueError( + "List-like value are not supported for 'lower' and 'upper' at the " + "moment" + ) + + if lower is None and upper is None: + return self + + if isinstance(self.spark.data_type, NumericType): + scol = self.spark.column + if lower is not None: + scol = F.when(scol < lower, lower).otherwise(scol) + if upper is not None: + scol = F.when(scol > upper, upper).otherwise(scol) + return self._with_new_scol(scol, dtype=self.dtype) + else: + return self + + def drop( + self, labels=None, index: Union[Any, Tuple, List[Any], List[Tuple]] = None, level=None + ) -> "Series": + """ + Return Series with specified index labels removed. + + Remove elements of a Series based on specifying the index labels. + When using a multi-index, labels on different levels can be removed by specifying the level. + + Parameters + ---------- + labels : single label or list-like + Index labels to drop. + index : None + Redundant for application on Series, but index can be used instead of labels. + level : int or level name, optional + For MultiIndex, level for which the labels will be removed. + + Returns + ------- + Series + Series with specified index labels removed. + + See Also + -------- + Series.dropna + + Examples + -------- + >>> s = pp.Series(data=np.arange(3), index=['A', 'B', 'C']) + >>> s + A 0 + B 1 + C 2 + dtype: int64 + + Drop single label A + + >>> s.drop('A') + B 1 + C 2 + dtype: int64 + + Drop labels B and C + + >>> s.drop(labels=['B', 'C']) + A 0 + dtype: int64 + + With 'index' rather than 'labels' returns exactly same result. + + >>> s.drop(index='A') + B 1 + C 2 + dtype: int64 + + >>> s.drop(index=['B', 'C']) + A 0 + dtype: int64 + + Also support for MultiIndex + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 1, 2]]) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], + ... index=midx) + >>> s + lama speed 45.0 + weight 200.0 + length 1.2 + cow speed 30.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + >>> s.drop(labels='weight', level=1) + lama speed 45.0 + length 1.2 + cow speed 30.0 + length 1.5 + falcon speed 320.0 + length 0.3 + dtype: float64 + + >>> s.drop(('lama', 'weight')) + lama speed 45.0 + length 1.2 + cow speed 30.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + >>> s.drop([('lama', 'speed'), ('falcon', 'weight')]) + lama weight 200.0 + length 1.2 + cow speed 30.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + length 0.3 + dtype: float64 + """ + return first_series(self._drop(labels=labels, index=index, level=level)) + + def _drop( + self, labels=None, index: Union[Any, Tuple, List[Any], List[Tuple]] = None, level=None + ): + if labels is not None: + if index is not None: + raise ValueError("Cannot specify both 'labels' and 'index'") + return self._drop(index=labels, level=level) + if index is not None: + internal = self._internal + if level is None: + level = 0 + if level >= internal.index_level: + raise ValueError("'level' should be less than the number of indexes") + + if is_name_like_tuple(index): # type: ignore + index = [index] + elif is_name_like_value(index): + index = [(index,)] + elif all(is_name_like_value(idxes, allow_tuple=False) for idxes in index): + index = [(idex,) for idex in index] + elif not all(is_name_like_tuple(idxes) for idxes in index): + raise ValueError( + "If the given index is a list, it " + "should only contains names as all tuples or all non tuples " + "that contain index names" + ) + + drop_index_scols = [] + for idxes in index: + try: + index_scols = [ + internal.index_spark_columns[lvl] == idx + for lvl, idx in enumerate(idxes, level) + ] + except IndexError: + raise KeyError( + "Key length ({}) exceeds index depth ({})".format( + internal.index_level, len(idxes) + ) + ) + drop_index_scols.append(reduce(lambda x, y: x & y, index_scols)) + + cond = ~reduce(lambda x, y: x | y, drop_index_scols) + + return DataFrame(internal.with_filter(cond)) + else: + raise ValueError("Need to specify at least one of 'labels' or 'index'") + + def head(self, n: int = 5) -> "Series": + """ + Return the first n rows. + + This function returns the first n rows for the object based on position. + It is useful for quickly testing if your object has the right type of data in it. + + Parameters + ---------- + n : Integer, default = 5 + + Returns + ------- + The first n rows of the caller object. + + Examples + -------- + >>> df = pp.DataFrame({'animal':['alligator', 'bee', 'falcon', 'lion']}) + >>> df.animal.head(2) # doctest: +NORMALIZE_WHITESPACE + 0 alligator + 1 bee + Name: animal, dtype: object + """ + return first_series(self.to_frame().head(n)).rename(self.name) + + def last(self, offset: Union[str, DateOffset]) -> "Series": + """ + Select final periods of time series data based on a date offset. + + When having a Series with dates as index, this function can + select the last few elements based on a date offset. + + Parameters + ---------- + offset : str or DateOffset + The offset length of the data that will be selected. For instance, + '3D' will display all the rows having their index within the last 3 days. + + Returns + ------- + Series + A subset of the caller. + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + Examples + -------- + >>> index = pd.date_range('2018-04-09', periods=4, freq='2D') + >>> kser = pp.Series([1, 2, 3, 4], index=index) + >>> kser + 2018-04-09 1 + 2018-04-11 2 + 2018-04-13 3 + 2018-04-15 4 + dtype: int64 + + Get the rows for the last 3 days: + + >>> kser.last('3D') + 2018-04-13 3 + 2018-04-15 4 + dtype: int64 + + Notice the data for 3 last calendar days were returned, not the last + 3 observed days in the dataset, and therefore data for 2018-04-11 was + not returned. + """ + return first_series(self.to_frame().last(offset)).rename(self.name) + + def first(self, offset: Union[str, DateOffset]) -> "Series": + """ + Select first periods of time series data based on a date offset. + + When having a Series with dates as index, this function can + select the first few elements based on a date offset. + + Parameters + ---------- + offset : str or DateOffset + The offset length of the data that will be selected. For instance, + '3D' will display all the rows having their index within the first 3 days. + + Returns + ------- + Series + A subset of the caller. + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + Examples + -------- + >>> index = pd.date_range('2018-04-09', periods=4, freq='2D') + >>> kser = pp.Series([1, 2, 3, 4], index=index) + >>> kser + 2018-04-09 1 + 2018-04-11 2 + 2018-04-13 3 + 2018-04-15 4 + dtype: int64 + + Get the rows for the first 3 days: + + >>> kser.first('3D') + 2018-04-09 1 + 2018-04-11 2 + dtype: int64 + + Notice the data for 3 first calendar days were returned, not the first + 3 observed days in the dataset, and therefore data for 2018-04-13 was + not returned. + """ + return first_series(self.to_frame().first(offset)).rename(self.name) + + # TODO: Categorical type isn't supported (due to PySpark's limitation) and + # some doctests related with timestamps were not added. + def unique(self) -> "Series": + """ + Return unique values of Series object. + + Uniques are returned in order of appearance. Hash table-based unique, + therefore does NOT sort. + + .. note:: This method returns newly created Series whereas pandas returns + the unique values as a NumPy array. + + Returns + ------- + Returns the unique values as a Series. + + See Also + -------- + Index.unique + groupby.SeriesGroupBy.unique + + Examples + -------- + >>> kser = pp.Series([2, 1, 3, 3], name='A') + >>> kser.unique().sort_values() # doctest: +NORMALIZE_WHITESPACE, +ELLIPSIS + + ... 1 + ... 2 + ... 3 + Name: A, dtype: int64 + + >>> pp.Series([pd.Timestamp('2016-01-01') for _ in range(3)]).unique() + 0 2016-01-01 + dtype: datetime64[ns] + + >>> kser.name = ('x', 'a') + >>> kser.unique().sort_values() # doctest: +NORMALIZE_WHITESPACE, +ELLIPSIS + + ... 1 + ... 2 + ... 3 + Name: (x, a), dtype: int64 + """ + sdf = self._internal.spark_frame.select(self.spark.column).distinct() + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=None, + column_labels=[self._column_label], + data_spark_columns=[scol_for(sdf, self._internal.data_spark_column_names[0])], + data_dtypes=[self.dtype], + column_label_names=self._internal.column_label_names, + ) + return first_series(DataFrame(internal)) + + def sort_values( + self, ascending: bool = True, inplace: bool = False, na_position: str = "last" + ) -> Optional["Series"]: + """ + Sort by the values. + + Sort a Series in ascending or descending order by some criterion. + + Parameters + ---------- + ascending : bool or list of bool, default True + Sort ascending vs. descending. Specify list for multiple sort + orders. If this is a list of bools, must match the length of + the by. + inplace : bool, default False + if True, perform operation in-place + na_position : {'first', 'last'}, default 'last' + `first` puts NaNs at the beginning, `last` puts NaNs at the end + + Returns + ------- + sorted_obj : Series ordered by values. + + Examples + -------- + >>> s = pp.Series([np.nan, 1, 3, 10, 5]) + >>> s + 0 NaN + 1 1.0 + 2 3.0 + 3 10.0 + 4 5.0 + dtype: float64 + + Sort values ascending order (default behaviour) + + >>> s.sort_values(ascending=True) + 1 1.0 + 2 3.0 + 4 5.0 + 3 10.0 + 0 NaN + dtype: float64 + + Sort values descending order + + >>> s.sort_values(ascending=False) + 3 10.0 + 4 5.0 + 2 3.0 + 1 1.0 + 0 NaN + dtype: float64 + + Sort values inplace + + >>> s.sort_values(ascending=False, inplace=True) + >>> s + 3 10.0 + 4 5.0 + 2 3.0 + 1 1.0 + 0 NaN + dtype: float64 + + Sort values putting NAs first + + >>> s.sort_values(na_position='first') + 0 NaN + 1 1.0 + 2 3.0 + 4 5.0 + 3 10.0 + dtype: float64 + + Sort a series of strings + + >>> s = pp.Series(['z', 'b', 'd', 'a', 'c']) + >>> s + 0 z + 1 b + 2 d + 3 a + 4 c + dtype: object + + >>> s.sort_values() + 3 a + 1 b + 4 c + 2 d + 0 z + dtype: object + """ + inplace = validate_bool_kwarg(inplace, "inplace") + kdf = self._kdf[[self.name]]._sort( + by=[self.spark.column], ascending=ascending, inplace=False, na_position=na_position + ) + + if inplace: + self._update_anchor(kdf) + return None + else: + return first_series(kdf) + + def sort_index( + self, + axis: int = 0, + level: Optional[Union[int, List[int]]] = None, + ascending: bool = True, + inplace: bool = False, + kind: str = None, + na_position: str = "last", + ) -> Optional["Series"]: + """ + Sort object by labels (along an axis) + + Parameters + ---------- + axis : index, columns to direct sorting. Currently, only axis = 0 is supported. + level : int or level name or list of ints or list of level names + if not None, sort on values in specified index level(s) + ascending : boolean, default True + Sort ascending vs. descending + inplace : bool, default False + if True, perform operation in-place + kind : str, default None + Koalas does not allow specifying the sorting algorithm at the moment, default None + na_position : {‘first’, ‘last’}, default ‘last’ + first puts NaNs at the beginning, last puts NaNs at the end. Not implemented for + MultiIndex. + + Returns + ------- + sorted_obj : Series + + Examples + -------- + >>> df = pp.Series([2, 1, np.nan], index=['b', 'a', np.nan]) + + >>> df.sort_index() + a 1.0 + b 2.0 + NaN NaN + dtype: float64 + + >>> df.sort_index(ascending=False) + b 2.0 + a 1.0 + NaN NaN + dtype: float64 + + >>> df.sort_index(na_position='first') + NaN NaN + a 1.0 + b 2.0 + dtype: float64 + + >>> df.sort_index(inplace=True) + >>> df + a 1.0 + b 2.0 + NaN NaN + dtype: float64 + + >>> df = pp.Series(range(4), index=[['b', 'b', 'a', 'a'], [1, 0, 1, 0]], name='0') + + >>> df.sort_index() + a 0 3 + 1 2 + b 0 1 + 1 0 + Name: 0, dtype: int64 + + >>> df.sort_index(level=1) # doctest: +SKIP + a 0 3 + b 0 1 + a 1 2 + b 1 0 + Name: 0, dtype: int64 + + >>> df.sort_index(level=[1, 0]) + a 0 3 + b 0 1 + a 1 2 + b 1 0 + Name: 0, dtype: int64 + """ + inplace = validate_bool_kwarg(inplace, "inplace") + kdf = self._kdf[[self.name]].sort_index( + axis=axis, level=level, ascending=ascending, kind=kind, na_position=na_position + ) + + if inplace: + self._update_anchor(kdf) + return None + else: + return first_series(kdf) + + def swaplevel(self, i=-2, j=-1, copy: bool = True) -> "Series": + """ + Swap levels i and j in a MultiIndex. + Default is to swap the two innermost levels of the index. + + Parameters + ---------- + i, j : int, str + Level of the indices to be swapped. Can pass level name as string. + copy : bool, default True + Whether to copy underlying data. Must be True. + + Returns + ------- + Series + Series with levels swapped in MultiIndex. + + Examples + -------- + >>> midx = pd.MultiIndex.from_arrays([['a', 'b'], [1, 2]], names = ['word', 'number']) + >>> midx # doctest: +SKIP + MultiIndex([('a', 1), + ('b', 2)], + names=['word', 'number']) + >>> kser = pp.Series(['x', 'y'], index=midx) + >>> kser + word number + a 1 x + b 2 y + dtype: object + >>> kser.swaplevel() + number word + 1 a x + 2 b y + dtype: object + >>> kser.swaplevel(0, 1) + number word + 1 a x + 2 b y + dtype: object + >>> kser.swaplevel('number', 'word') + number word + 1 a x + 2 b y + dtype: object + """ + assert copy is True + + return first_series(self.to_frame().swaplevel(i, j, axis=0)).rename(self.name) + + def swapaxes(self, i: Union[str, int], j: Union[str, int], copy: bool = True) -> "Series": + """ + Interchange axes and swap values axes appropriately. + + Parameters + ---------- + i: {0 or 'index', 1 or 'columns'}. The axis to swap. + j: {0 or 'index', 1 or 'columns'}. The axis to swap. + copy : bool, default True. + + Returns + ------- + Series + + Examples + -------- + >>> kser = pp.Series([1, 2, 3], index=["x", "y", "z"]) + >>> kser + x 1 + y 2 + z 3 + dtype: int64 + >>> + >>> kser.swapaxes(0, 0) + x 1 + y 2 + z 3 + dtype: int64 + """ + assert copy is True + + i = validate_axis(i) + j = validate_axis(j) + if not i == j == 0: + raise ValueError("Axis must be 0 for Series") + + return self.copy() + + def add_prefix(self, prefix) -> "Series": + """ + Prefix labels with string `prefix`. + + For Series, the row labels are prefixed. + For DataFrame, the column labels are prefixed. + + Parameters + ---------- + prefix : str + The string to add before each label. + + Returns + ------- + Series + New Series with updated labels. + + See Also + -------- + Series.add_suffix: Suffix column labels with string `suffix`. + DataFrame.add_suffix: Suffix column labels with string `suffix`. + DataFrame.add_prefix: Prefix column labels with string `prefix`. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4]) + >>> s + 0 1 + 1 2 + 2 3 + 3 4 + dtype: int64 + + >>> s.add_prefix('item_') + item_0 1 + item_1 2 + item_2 3 + item_3 4 + dtype: int64 + """ + assert isinstance(prefix, str) + internal = self._internal.resolved_copy + sdf = internal.spark_frame.select( + [ + F.concat(F.lit(prefix), index_spark_column).alias(index_spark_column_name) + for index_spark_column, index_spark_column_name in zip( + internal.index_spark_columns, internal.index_spark_column_names + ) + ] + + internal.data_spark_columns + ) + return first_series( + DataFrame(internal.with_new_sdf(sdf, index_dtypes=([None] * internal.index_level))) + ) + + def add_suffix(self, suffix) -> "Series": + """ + Suffix labels with string suffix. + + For Series, the row labels are suffixed. + For DataFrame, the column labels are suffixed. + + Parameters + ---------- + suffix : str + The string to add after each label. + + Returns + ------- + Series + New Series with updated labels. + + See Also + -------- + Series.add_prefix: Prefix row labels with string `prefix`. + DataFrame.add_prefix: Prefix column labels with string `prefix`. + DataFrame.add_suffix: Suffix column labels with string `suffix`. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4]) + >>> s + 0 1 + 1 2 + 2 3 + 3 4 + dtype: int64 + + >>> s.add_suffix('_item') + 0_item 1 + 1_item 2 + 2_item 3 + 3_item 4 + dtype: int64 + """ + assert isinstance(suffix, str) + internal = self._internal.resolved_copy + sdf = internal.spark_frame.select( + [ + F.concat(index_spark_column, F.lit(suffix)).alias(index_spark_column_name) + for index_spark_column, index_spark_column_name in zip( + internal.index_spark_columns, internal.index_spark_column_names + ) + ] + + internal.data_spark_columns + ) + return first_series( + DataFrame(internal.with_new_sdf(sdf, index_dtypes=([None] * internal.index_level))) + ) + + def corr(self, other, method="pearson") -> float: + """ + Compute correlation with `other` Series, excluding missing values. + + Parameters + ---------- + other : Series + method : {'pearson', 'spearman'} + * pearson : standard correlation coefficient + * spearman : Spearman rank correlation + + Returns + ------- + correlation : float + + Examples + -------- + >>> df = pp.DataFrame({'s1': [.2, .0, .6, .2], + ... 's2': [.3, .6, .0, .1]}) + >>> s1 = df.s1 + >>> s2 = df.s2 + >>> s1.corr(s2, method='pearson') # doctest: +ELLIPSIS + -0.851064... + + >>> s1.corr(s2, method='spearman') # doctest: +ELLIPSIS + -0.948683... + + Notes + ----- + There are behavior differences between Koalas and pandas. + + * the `method` argument only accepts 'pearson', 'spearman' + * the data should not contain NaNs. Koalas will return an error. + * Koalas doesn't support the following argument(s). + + * `min_periods` argument is not supported + """ + # This implementation is suboptimal because it computes more than necessary, + # but it should be a start + columns = ["__corr_arg1__", "__corr_arg2__"] + kdf = self._kdf.assign(__corr_arg1__=self, __corr_arg2__=other)[columns] + kdf.columns = columns + c = corr(kdf, method=method) + return c.loc[tuple(columns)] + + def nsmallest(self, n: int = 5) -> "Series": + """ + Return the smallest `n` elements. + + Parameters + ---------- + n : int, default 5 + Return this many ascending sorted values. + + Returns + ------- + Series + The `n` smallest values in the Series, sorted in increasing order. + + See Also + -------- + Series.nlargest: Get the `n` largest elements. + Series.sort_values: Sort Series by values. + Series.head: Return the first `n` rows. + + Notes + ----- + Faster than ``.sort_values().head(n)`` for small `n` relative to + the size of the ``Series`` object. + In Koalas, thanks to Spark's lazy execution and query optimizer, + the two would have same performance. + + Examples + -------- + >>> data = [1, 2, 3, 4, np.nan ,6, 7, 8] + >>> s = pp.Series(data) + >>> s + 0 1.0 + 1 2.0 + 2 3.0 + 3 4.0 + 4 NaN + 5 6.0 + 6 7.0 + 7 8.0 + dtype: float64 + + The `n` largest elements where ``n=5`` by default. + + >>> s.nsmallest() + 0 1.0 + 1 2.0 + 2 3.0 + 3 4.0 + 5 6.0 + dtype: float64 + + >>> s.nsmallest(3) + 0 1.0 + 1 2.0 + 2 3.0 + dtype: float64 + """ + return self.sort_values(ascending=True).head(n) + + def nlargest(self, n: int = 5) -> "Series": + """ + Return the largest `n` elements. + + Parameters + ---------- + n : int, default 5 + + Returns + ------- + Series + The `n` largest values in the Series, sorted in decreasing order. + + See Also + -------- + Series.nsmallest: Get the `n` smallest elements. + Series.sort_values: Sort Series by values. + Series.head: Return the first `n` rows. + + Notes + ----- + Faster than ``.sort_values(ascending=False).head(n)`` for small `n` + relative to the size of the ``Series`` object. + + In Koalas, thanks to Spark's lazy execution and query optimizer, + the two would have same performance. + + Examples + -------- + >>> data = [1, 2, 3, 4, np.nan ,6, 7, 8] + >>> s = pp.Series(data) + >>> s + 0 1.0 + 1 2.0 + 2 3.0 + 3 4.0 + 4 NaN + 5 6.0 + 6 7.0 + 7 8.0 + dtype: float64 + + The `n` largest elements where ``n=5`` by default. + + >>> s.nlargest() + 7 8.0 + 6 7.0 + 5 6.0 + 3 4.0 + 2 3.0 + dtype: float64 + + >>> s.nlargest(n=3) + 7 8.0 + 6 7.0 + 5 6.0 + dtype: float64 + + + """ + return self.sort_values(ascending=False).head(n) + + def append( + self, to_append: "Series", ignore_index: bool = False, verify_integrity: bool = False + ) -> "Series": + """ + Concatenate two or more Series. + + Parameters + ---------- + to_append : Series or list/tuple of Series + ignore_index : boolean, default False + If True, do not use the index labels. + verify_integrity : boolean, default False + If True, raise Exception on creating index with duplicates + + Returns + ------- + appended : Series + + Examples + -------- + >>> s1 = pp.Series([1, 2, 3]) + >>> s2 = pp.Series([4, 5, 6]) + >>> s3 = pp.Series([4, 5, 6], index=[3,4,5]) + + >>> s1.append(s2) + 0 1 + 1 2 + 2 3 + 0 4 + 1 5 + 2 6 + dtype: int64 + + >>> s1.append(s3) + 0 1 + 1 2 + 2 3 + 3 4 + 4 5 + 5 6 + dtype: int64 + + With ignore_index set to True: + + >>> s1.append(s2, ignore_index=True) + 0 1 + 1 2 + 2 3 + 3 4 + 4 5 + 5 6 + dtype: int64 + """ + return first_series( + self.to_frame().append(to_append.to_frame(), ignore_index, verify_integrity) + ).rename(self.name) + + def sample( + self, + n: Optional[int] = None, + frac: Optional[float] = None, + replace: bool = False, + random_state: Optional[int] = None, + ) -> "Series": + return first_series( + self.to_frame().sample(n=n, frac=frac, replace=replace, random_state=random_state) + ).rename(self.name) + + sample.__doc__ = DataFrame.sample.__doc__ + + def hist(self, bins=10, **kwds): + return self.plot.hist(bins, **kwds) + + hist.__doc__ = KoalasPlotAccessor.hist.__doc__ + + def apply(self, func, args=(), **kwds) -> "Series": + """ + Invoke function on values of Series. + + Can be a Python function that only works on the Series. + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def square(x) -> np.int32: + ... return x ** 2 + + Koalas uses return type hint and does not try to infer the type. + + Parameters + ---------- + func : function + Python function to apply. Note that type hint for return type is required. + args : tuple + Positional arguments passed to func after the series value. + **kwds + Additional keyword arguments passed to func. + + Returns + ------- + Series + + See Also + -------- + Series.aggregate : Only perform aggregating type operations. + Series.transform : Only perform transforming type operations. + DataFrame.apply : The equivalent function for DataFrame. + + Examples + -------- + Create a Series with typical summer temperatures for each city. + + >>> s = pp.Series([20, 21, 12], + ... index=['London', 'New York', 'Helsinki']) + >>> s + London 20 + New York 21 + Helsinki 12 + dtype: int64 + + + Square the values by defining a function and passing it as an + argument to ``apply()``. + + >>> def square(x) -> np.int64: + ... return x ** 2 + >>> s.apply(square) + London 400 + New York 441 + Helsinki 144 + dtype: int64 + + + Define a custom function that needs additional positional + arguments and pass these additional arguments using the + ``args`` keyword + + >>> def subtract_custom_value(x, custom_value) -> np.int64: + ... return x - custom_value + + >>> s.apply(subtract_custom_value, args=(5,)) + London 15 + New York 16 + Helsinki 7 + dtype: int64 + + + Define a custom function that takes keyword arguments + and pass these arguments to ``apply`` + + >>> def add_custom_values(x, **kwargs) -> np.int64: + ... for month in kwargs: + ... x += kwargs[month] + ... return x + + >>> s.apply(add_custom_values, june=30, july=20, august=25) + London 95 + New York 96 + Helsinki 87 + dtype: int64 + + + Use a function from the Numpy library + + >>> def numpy_log(col) -> np.float64: + ... return np.log(col) + >>> s.apply(numpy_log) + London 2.995732 + New York 3.044522 + Helsinki 2.484907 + dtype: float64 + + + You can omit the type hint and let Koalas infer its type. + + >>> s.apply(np.log) + London 2.995732 + New York 3.044522 + Helsinki 2.484907 + dtype: float64 + + """ + assert callable(func), "the first argument should be a callable function." + try: + spec = inspect.getfullargspec(func) + return_sig = spec.annotations.get("return", None) + should_infer_schema = return_sig is None + except TypeError: + # Falls back to schema inference if it fails to get signature. + should_infer_schema = True + + apply_each = wraps(func)(lambda s: s.apply(func, args=args, **kwds)) + + if should_infer_schema: + return self.koalas._transform_batch(apply_each, None) + else: + sig_return = infer_return_type(func) + if not isinstance(sig_return, ScalarType): + raise ValueError( + "Expected the return type of this function to be of scalar type, " + "but found type {}".format(sig_return) + ) + return_type = cast(ScalarType, sig_return) + return self.koalas._transform_batch(apply_each, return_type) + + # TODO: not all arguments are implemented comparing to pandas' for now. + def aggregate(self, func: Union[str, List[str]]) -> Union[Scalar, "Series"]: + """Aggregate using one or more operations over the specified axis. + + Parameters + ---------- + func : str or a list of str + function name(s) as string apply to series. + + Returns + ------- + scalar, Series + The return can be: + - scalar : when Series.agg is called with single function + - Series : when Series.agg is called with several functions + + Notes + ----- + `agg` is an alias for `aggregate`. Use the alias. + + See Also + -------- + Series.apply : Invoke function on a Series. + Series.transform : Only perform transforming type operations. + Series.groupby : Perform operations over groups. + DataFrame.aggregate : The equivalent function for DataFrame. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4]) + >>> s.agg('min') + 1 + + >>> s.agg(['min', 'max']).sort_index() + max 4 + min 1 + dtype: int64 + """ + if isinstance(func, list): + return first_series(self.to_frame().aggregate(func)).rename(self.name) + elif isinstance(func, str): + return getattr(self, func)() + else: + raise ValueError("func must be a string or list of strings") + + agg = aggregate + + def transpose(self, *args, **kwargs) -> "Series": + """ + Return the transpose, which is by definition self. + + Examples + -------- + It returns the same object as the transpose of the given series object, which is by + definition self. + + >>> s = pp.Series([1, 2, 3]) + >>> s + 0 1 + 1 2 + 2 3 + dtype: int64 + + >>> s.transpose() + 0 1 + 1 2 + 2 3 + dtype: int64 + """ + return self.copy() + + T = property(transpose) + + def transform(self, func, axis=0, *args, **kwargs) -> Union["Series", DataFrame]: + """ + Call ``func`` producing the same type as `self` with transformed values + and that has the same axis length as input. + + .. note:: this API executes the function once to infer the type which is + potentially expensive, for instance, when the dataset is created after + aggregations or sorting. + + To avoid this, specify return type in ``func``, for instance, as below: + + >>> def square(x) -> np.int32: + ... return x ** 2 + + Koalas uses return type hint and does not try to infer the type. + + Parameters + ---------- + func : function or list + A function or a list of functions to use for transforming the data. + axis : int, default 0 or 'index' + Can only be set to 0 at the moment. + *args + Positional arguments to pass to `func`. + **kwargs + Keyword arguments to pass to `func`. + + Returns + ------- + An instance of the same type with `self` that must have the same length as input. + + See Also + -------- + Series.aggregate : Only perform aggregating type operations. + Series.apply : Invoke function on Series. + DataFrame.transform : The equivalent function for DataFrame. + + Examples + -------- + + >>> s = pp.Series(range(3)) + >>> s + 0 0 + 1 1 + 2 2 + dtype: int64 + + >>> def sqrt(x) -> float: + ... return np.sqrt(x) + >>> s.transform(sqrt) + 0 0.000000 + 1 1.000000 + 2 1.414214 + dtype: float64 + + Even though the resulting instance must have the same length as the + input, it is possible to provide several input functions: + + >>> def exp(x) -> float: + ... return np.exp(x) + >>> s.transform([sqrt, exp]) + sqrt exp + 0 0.000000 1.000000 + 1 1.000000 2.718282 + 2 1.414214 7.389056 + + You can omit the type hint and let Koalas infer its type. + + >>> s.transform([np.sqrt, np.exp]) + sqrt exp + 0 0.000000 1.000000 + 1 1.000000 2.718282 + 2 1.414214 7.389056 + """ + axis = validate_axis(axis) + if axis != 0: + raise NotImplementedError('axis should be either 0 or "index" currently.') + + if isinstance(func, list): + applied = [] + for f in func: + applied.append(self.apply(f, args=args, **kwargs).rename(f.__name__)) + + internal = self._internal.with_new_columns(applied) + return DataFrame(internal) + else: + return self.apply(func, args=args, **kwargs) + + def transform_batch(self, func, *args, **kwargs) -> "pp.Series": + warnings.warn( + "Series.transform_batch is deprecated as of Series.koalas.transform_batch. " + "Please use the API instead.", + FutureWarning, + ) + return self.koalas.transform_batch(func, *args, **kwargs) + + transform_batch.__doc__ = KoalasSeriesMethods.transform_batch.__doc__ + + def round(self, decimals=0) -> "Series": + """ + Round each value in a Series to the given number of decimals. + + Parameters + ---------- + decimals : int + Number of decimal places to round to (default: 0). + If decimals is negative, it specifies the number of + positions to the left of the decimal point. + + Returns + ------- + Series object + + See Also + -------- + DataFrame.round + + Examples + -------- + >>> df = pp.Series([0.028208, 0.038683, 0.877076], name='x') + >>> df + 0 0.028208 + 1 0.038683 + 2 0.877076 + Name: x, dtype: float64 + + >>> df.round(2) + 0 0.03 + 1 0.04 + 2 0.88 + Name: x, dtype: float64 + """ + if not isinstance(decimals, int): + raise ValueError("decimals must be an integer") + scol = F.round(self.spark.column, decimals) + return self._with_new_scol(scol) + + # TODO: add 'interpolation' parameter. + def quantile( + self, q: Union[float, Iterable[float]] = 0.5, accuracy: int = 10000 + ) -> Union[Scalar, "Series"]: + """ + Return value at the given quantile. + + .. note:: Unlike pandas', the quantile in Koalas is an approximated quantile based upon + approximate percentile computation because computing quantile across a large dataset + is extremely expensive. + + Parameters + ---------- + q : float or array-like, default 0.5 (50% quantile) + 0 <= q <= 1, the quantile(s) to compute. + accuracy : int, optional + Default accuracy of approximation. Larger value means better accuracy. + The relative error can be deduced by 1.0 / accuracy. + + Returns + ------- + float or Series + If the current object is a Series and ``q`` is an array, a Series will be + returned where the index is ``q`` and the values are the quantiles, otherwise + a float will be returned. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4, 5]) + >>> s.quantile(.5) + 3.0 + + >>> (s + 1).quantile(.5) + 4.0 + + >>> s.quantile([.25, .5, .75]) + 0.25 2.0 + 0.50 3.0 + 0.75 4.0 + dtype: float64 + + >>> (s + 1).quantile([.25, .5, .75]) + 0.25 3.0 + 0.50 4.0 + 0.75 5.0 + dtype: float64 + """ + if isinstance(q, Iterable): + return first_series( + self.to_frame().quantile(q=q, axis=0, numeric_only=False, accuracy=accuracy) + ).rename(self.name) + else: + if not isinstance(accuracy, int): + raise ValueError( + "accuracy must be an integer; however, got [%s]" % type(accuracy).__name__ + ) + + if not isinstance(q, float): + raise ValueError( + "q must be a float or an array of floats; however, [%s] found." % type(q) + ) + if q < 0.0 or q > 1.0: + raise ValueError("percentiles should all be in the interval [0, 1].") + + def quantile(spark_column, spark_type): + if isinstance(spark_type, (BooleanType, NumericType)): + return SF.percentile_approx(spark_column.cast(DoubleType()), q, accuracy) + else: + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(spark_type), spark_type.simpleString() + ) + ) + + return self._reduce_for_stat_function(quantile, name="quantile") + + # TODO: add axis, numeric_only, pct, na_option parameter + def rank(self, method="average", ascending=True) -> "Series": + """ + Compute numerical data ranks (1 through n) along axis. Equal values are + assigned a rank that is the average of the ranks of those values. + + .. note:: the current implementation of rank uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + method : {'average', 'min', 'max', 'first', 'dense'} + * average: average rank of group + * min: lowest rank in group + * max: highest rank in group + * first: ranks assigned in order they appear in the array + * dense: like 'min', but rank always increases by 1 between groups + ascending : boolean, default True + False for ranks by high (1) to low (N) + + Returns + ------- + ranks : same type as caller + + Examples + -------- + >>> s = pp.Series([1, 2, 2, 3], name='A') + >>> s + 0 1 + 1 2 + 2 2 + 3 3 + Name: A, dtype: int64 + + >>> s.rank() + 0 1.0 + 1 2.5 + 2 2.5 + 3 4.0 + Name: A, dtype: float64 + + If method is set to 'min', it use lowest rank in group. + + >>> s.rank(method='min') + 0 1.0 + 1 2.0 + 2 2.0 + 3 4.0 + Name: A, dtype: float64 + + If method is set to 'max', it use highest rank in group. + + >>> s.rank(method='max') + 0 1.0 + 1 3.0 + 2 3.0 + 3 4.0 + Name: A, dtype: float64 + + If method is set to 'first', it is assigned rank in order without groups. + + >>> s.rank(method='first') + 0 1.0 + 1 2.0 + 2 3.0 + 3 4.0 + Name: A, dtype: float64 + + If method is set to 'dense', it leaves no gaps in group. + + >>> s.rank(method='dense') + 0 1.0 + 1 2.0 + 2 2.0 + 3 3.0 + Name: A, dtype: float64 + """ + return self._rank(method, ascending).spark.analyzed + + def _rank(self, method="average", ascending=True, *, part_cols=()): + if method not in ["average", "min", "max", "first", "dense"]: + msg = "method must be one of 'average', 'min', 'max', 'first', 'dense'" + raise ValueError(msg) + + if self._internal.index_level > 1: + raise ValueError("rank do not support index now") + + if ascending: + asc_func = lambda scol: scol.asc() + else: + asc_func = lambda scol: scol.desc() + + if method == "first": + window = ( + Window.orderBy( + asc_func(self.spark.column), asc_func(F.col(NATURAL_ORDER_COLUMN_NAME)), + ) + .partitionBy(*part_cols) + .rowsBetween(Window.unboundedPreceding, Window.currentRow) + ) + scol = F.row_number().over(window) + elif method == "dense": + window = ( + Window.orderBy(asc_func(self.spark.column)) + .partitionBy(*part_cols) + .rowsBetween(Window.unboundedPreceding, Window.currentRow) + ) + scol = F.dense_rank().over(window) + else: + if method == "average": + stat_func = F.mean + elif method == "min": + stat_func = F.min + elif method == "max": + stat_func = F.max + window1 = ( + Window.orderBy(asc_func(self.spark.column)) + .partitionBy(*part_cols) + .rowsBetween(Window.unboundedPreceding, Window.currentRow) + ) + window2 = Window.partitionBy([self.spark.column] + list(part_cols)).rowsBetween( + Window.unboundedPreceding, Window.unboundedFollowing + ) + scol = stat_func(F.row_number().over(window1)).over(window2) + kser = self._with_new_scol(scol) + return kser.astype(np.float64) + + def filter(self, items=None, like=None, regex=None, axis=None) -> "Series": + axis = validate_axis(axis) + if axis == 1: + raise ValueError("Series does not support columns axis.") + return first_series( + self.to_frame().filter(items=items, like=like, regex=regex, axis=axis) + ).rename(self.name) + + filter.__doc__ = DataFrame.filter.__doc__ + + def describe(self, percentiles: Optional[List[float]] = None) -> "Series": + return first_series(self.to_frame().describe(percentiles)).rename(self.name) + + describe.__doc__ = DataFrame.describe.__doc__ + + def diff(self, periods=1) -> "Series": + """ + First discrete difference of element. + + Calculates the difference of a Series element compared with another element in the + DataFrame (default is the element in the same column of the previous row). + + .. note:: the current implementation of diff uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + periods : int, default 1 + Periods to shift for calculating difference, accepts negative values. + + Returns + ------- + diffed : Series + + Examples + -------- + >>> df = pp.DataFrame({'a': [1, 2, 3, 4, 5, 6], + ... 'b': [1, 1, 2, 3, 5, 8], + ... 'c': [1, 4, 9, 16, 25, 36]}, columns=['a', 'b', 'c']) + >>> df + a b c + 0 1 1 1 + 1 2 1 4 + 2 3 2 9 + 3 4 3 16 + 4 5 5 25 + 5 6 8 36 + + >>> df.b.diff() + 0 NaN + 1 0.0 + 2 1.0 + 3 1.0 + 4 2.0 + 5 3.0 + Name: b, dtype: float64 + + Difference with previous value + + >>> df.c.diff(periods=3) + 0 NaN + 1 NaN + 2 NaN + 3 15.0 + 4 21.0 + 5 27.0 + Name: c, dtype: float64 + + Difference with following value + + >>> df.c.diff(periods=-1) + 0 -3.0 + 1 -5.0 + 2 -7.0 + 3 -9.0 + 4 -11.0 + 5 NaN + Name: c, dtype: float64 + """ + return self._diff(periods).spark.analyzed + + def _diff(self, periods, *, part_cols=()): + if not isinstance(periods, int): + raise ValueError("periods should be an int; however, got [%s]" % type(periods).__name__) + window = ( + Window.partitionBy(*part_cols) + .orderBy(NATURAL_ORDER_COLUMN_NAME) + .rowsBetween(-periods, -periods) + ) + scol = self.spark.column - F.lag(self.spark.column, periods).over(window) + return self._with_new_scol(scol, dtype=self.dtype) + + def idxmax(self, skipna=True) -> Union[Tuple, Any]: + """ + Return the row label of the maximum value. + + If multiple values equal the maximum, the first row label with that + value is returned. + + Parameters + ---------- + skipna : bool, default True + Exclude NA/null values. If the entire Series is NA, the result + will be NA. + + Returns + ------- + Index + Label of the maximum value. + + Raises + ------ + ValueError + If the Series is empty. + + See Also + -------- + Series.idxmin : Return index *label* of the first occurrence + of minimum of values. + + Examples + -------- + >>> s = pp.Series(data=[1, None, 4, 3, 5], + ... index=['A', 'B', 'C', 'D', 'E']) + >>> s + A 1.0 + B NaN + C 4.0 + D 3.0 + E 5.0 + dtype: float64 + + >>> s.idxmax() + 'E' + + If `skipna` is False and there is an NA value in the data, + the function returns ``nan``. + + >>> s.idxmax(skipna=False) + nan + + In case of multi-index, you get a tuple: + + >>> index = pd.MultiIndex.from_arrays([ + ... ['a', 'a', 'b', 'b'], ['c', 'd', 'e', 'f']], names=('first', 'second')) + >>> s = pp.Series(data=[1, None, 4, 5], index=index) + >>> s + first second + a c 1.0 + d NaN + b e 4.0 + f 5.0 + dtype: float64 + + >>> s.idxmax() + ('b', 'f') + + If multiple values equal the maximum, the first row label with that + value is returned. + + >>> s = pp.Series([1, 100, 1, 100, 1, 100], index=[10, 3, 5, 2, 1, 8]) + >>> s + 10 1 + 3 100 + 5 1 + 2 100 + 1 1 + 8 100 + dtype: int64 + + >>> s.idxmax() + 3 + """ + sdf = self._internal.spark_frame + scol = self.spark.column + index_scols = self._internal.index_spark_columns + # desc_nulls_(last|first) is used via Py4J directly because + # it's not supported in Spark 2.3. + if skipna: + sdf = sdf.orderBy(Column(scol._jc.desc_nulls_last()), NATURAL_ORDER_COLUMN_NAME) + else: + sdf = sdf.orderBy(Column(scol._jc.desc_nulls_first()), NATURAL_ORDER_COLUMN_NAME) + results = sdf.select([scol] + index_scols).take(1) + if len(results) == 0: + raise ValueError("attempt to get idxmin of an empty sequence") + if results[0][0] is None: + # This will only happens when skipna is False because we will + # place nulls first. + return np.nan + values = list(results[0][1:]) + if len(values) == 1: + return values[0] + else: + return tuple(values) + + def idxmin(self, skipna=True) -> Union[Tuple, Any]: + """ + Return the row label of the minimum value. + + If multiple values equal the minimum, the first row label with that + value is returned. + + Parameters + ---------- + skipna : bool, default True + Exclude NA/null values. If the entire Series is NA, the result + will be NA. + + Returns + ------- + Index + Label of the minimum value. + + Raises + ------ + ValueError + If the Series is empty. + + See Also + -------- + Series.idxmax : Return index *label* of the first occurrence + of maximum of values. + + Notes + ----- + This method is the Series version of ``ndarray.argmin``. This method + returns the label of the minimum, while ``ndarray.argmin`` returns + the position. To get the position, use ``series.values.argmin()``. + + Examples + -------- + >>> s = pp.Series(data=[1, None, 4, 0], + ... index=['A', 'B', 'C', 'D']) + >>> s + A 1.0 + B NaN + C 4.0 + D 0.0 + dtype: float64 + + >>> s.idxmin() + 'D' + + If `skipna` is False and there is an NA value in the data, + the function returns ``nan``. + + >>> s.idxmin(skipna=False) + nan + + In case of multi-index, you get a tuple: + + >>> index = pd.MultiIndex.from_arrays([ + ... ['a', 'a', 'b', 'b'], ['c', 'd', 'e', 'f']], names=('first', 'second')) + >>> s = pp.Series(data=[1, None, 4, 0], index=index) + >>> s + first second + a c 1.0 + d NaN + b e 4.0 + f 0.0 + dtype: float64 + + >>> s.idxmin() + ('b', 'f') + + If multiple values equal the minimum, the first row label with that + value is returned. + + >>> s = pp.Series([1, 100, 1, 100, 1, 100], index=[10, 3, 5, 2, 1, 8]) + >>> s + 10 1 + 3 100 + 5 1 + 2 100 + 1 1 + 8 100 + dtype: int64 + + >>> s.idxmin() + 10 + """ + sdf = self._internal.spark_frame + scol = self.spark.column + index_scols = self._internal.index_spark_columns + # asc_nulls_(last|first)is used via Py4J directly because + # it's not supported in Spark 2.3. + if skipna: + sdf = sdf.orderBy(Column(scol._jc.asc_nulls_last()), NATURAL_ORDER_COLUMN_NAME) + else: + sdf = sdf.orderBy(Column(scol._jc.asc_nulls_first()), NATURAL_ORDER_COLUMN_NAME) + results = sdf.select([scol] + index_scols).take(1) + if len(results) == 0: + raise ValueError("attempt to get idxmin of an empty sequence") + if results[0][0] is None: + # This will only happens when skipna is False because we will + # place nulls first. + return np.nan + values = list(results[0][1:]) + if len(values) == 1: + return values[0] + else: + return tuple(values) + + def pop(self, item) -> Union["Series", Scalar]: + """ + Return item and drop from series. + + Parameters + ---------- + item : str + Label of index to be popped. + + Returns + ------- + Value that is popped from series. + + Examples + -------- + >>> s = pp.Series(data=np.arange(3), index=['A', 'B', 'C']) + >>> s + A 0 + B 1 + C 2 + dtype: int64 + + >>> s.pop('A') + 0 + + >>> s + B 1 + C 2 + dtype: int64 + + >>> s = pp.Series(data=np.arange(3), index=['A', 'A', 'C']) + >>> s + A 0 + A 1 + C 2 + dtype: int64 + + >>> s.pop('A') + A 0 + A 1 + dtype: int64 + + >>> s + C 2 + dtype: int64 + + Also support for MultiIndex + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 1, 2]]) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], + ... index=midx) + >>> s + lama speed 45.0 + weight 200.0 + length 1.2 + cow speed 30.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + >>> s.pop('lama') + speed 45.0 + weight 200.0 + length 1.2 + dtype: float64 + + >>> s + cow speed 30.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + Also support for MultiIndex with several indexs. + + >>> midx = pd.MultiIndex([['a', 'b', 'c'], + ... ['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 0, 0, 0, 1, 1, 1], + ... [0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 0, 2]] + ... ) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], + ... index=midx) + >>> s + a lama speed 45.0 + weight 200.0 + length 1.2 + cow speed 30.0 + weight 250.0 + length 1.5 + b falcon speed 320.0 + speed 1.0 + length 0.3 + dtype: float64 + + >>> s.pop(('a', 'lama')) + speed 45.0 + weight 200.0 + length 1.2 + dtype: float64 + + >>> s + a cow speed 30.0 + weight 250.0 + length 1.5 + b falcon speed 320.0 + speed 1.0 + length 0.3 + dtype: float64 + + >>> s.pop(('b', 'falcon', 'speed')) + (b, falcon, speed) 320.0 + (b, falcon, speed) 1.0 + dtype: float64 + """ + if not is_name_like_value(item): + raise ValueError("'key' should be string or tuple that contains strings") + if not is_name_like_tuple(item): + item = (item,) + if self._internal.index_level < len(item): + raise KeyError( + "Key length ({}) exceeds index depth ({})".format( + len(item), self._internal.index_level + ) + ) + + internal = self._internal + scols = internal.index_spark_columns[len(item):] + [self.spark.column] + rows = [internal.spark_columns[level] == index for level, index in enumerate(item)] + sdf = internal.spark_frame.filter(reduce(lambda x, y: x & y, rows)).select(scols) + + kdf = self._drop(item) + self._update_anchor(kdf) + + if self._internal.index_level == len(item): + # if spark_frame has one column and one data, return data only without frame + pdf = sdf.limit(2).toPandas() + length = len(pdf) + if length == 1: + return pdf[internal.data_spark_column_names[0]].iloc[0] + + item_string = name_like_string(item) + sdf = sdf.withColumn(SPARK_DEFAULT_INDEX_NAME, F.lit(str(item_string))) + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, SPARK_DEFAULT_INDEX_NAME)], + column_labels=[self._column_label], + data_dtypes=[self.dtype], + ) + return first_series(DataFrame(internal)) + else: + internal = internal.copy( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in internal.index_spark_column_names[len(item):] + ], + index_dtypes=internal.index_dtypes[len(item):], + index_names=self._internal.index_names[len(item):], + data_spark_columns=[scol_for(sdf, internal.data_spark_column_names[0])], + ) + return first_series(DataFrame(internal)) + + def copy(self, deep=None) -> "Series": + """ + Make a copy of this object's indices and data. + + Parameters + ---------- + deep : None + this parameter is not supported but just dummy parameter to match pandas. + + Returns + ------- + copy : Series + + Examples + -------- + >>> s = pp.Series([1, 2], index=["a", "b"]) + >>> s + a 1 + b 2 + dtype: int64 + >>> s_copy = s.copy() + >>> s_copy + a 1 + b 2 + dtype: int64 + """ + return self._kdf.copy()._kser_for(self._column_label) + + def mode(self, dropna=True) -> "Series": + """ + Return the mode(s) of the dataset. + + Always returns Series even if only one value is returned. + + Parameters + ---------- + dropna : bool, default True + Don't consider counts of NaN/NaT. + + Returns + ------- + Series + Modes of the Series. + + Examples + -------- + >>> s = pp.Series([0, 0, 1, 1, 1, np.nan, np.nan, np.nan]) + >>> s + 0 0.0 + 1 0.0 + 2 1.0 + 3 1.0 + 4 1.0 + 5 NaN + 6 NaN + 7 NaN + dtype: float64 + + >>> s.mode() + 0 1.0 + dtype: float64 + + If there are several same modes, all items are shown + + >>> s = pp.Series([0, 0, 1, 1, 1, 2, 2, 2, 3, 3, 3, + ... np.nan, np.nan, np.nan]) + >>> s + 0 0.0 + 1 0.0 + 2 1.0 + 3 1.0 + 4 1.0 + 5 2.0 + 6 2.0 + 7 2.0 + 8 3.0 + 9 3.0 + 10 3.0 + 11 NaN + 12 NaN + 13 NaN + dtype: float64 + + >>> s.mode().sort_values() # doctest: +NORMALIZE_WHITESPACE, +ELLIPSIS + + ... 1.0 + ... 2.0 + ... 3.0 + dtype: float64 + + With 'dropna' set to 'False', we can also see NaN in the result + + >>> s.mode(False).sort_values() # doctest: +NORMALIZE_WHITESPACE, +ELLIPSIS + + ... 1.0 + ... 2.0 + ... 3.0 + ... NaN + dtype: float64 + """ + ser_count = self.value_counts(dropna=dropna, sort=False) + sdf_count = ser_count._internal.spark_frame + most_value = ser_count.max() + sdf_most_value = sdf_count.filter("count == {}".format(most_value)) + sdf = sdf_most_value.select( + F.col(SPARK_DEFAULT_INDEX_NAME).alias(SPARK_DEFAULT_SERIES_NAME) + ) + internal = InternalFrame(spark_frame=sdf, index_spark_columns=None, column_labels=[None]) + + return first_series(DataFrame(internal)) + + def keys(self) -> "pp.Index": + """ + Return alias for index. + + Returns + ------- + Index + Index of the Series. + + Examples + -------- + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 1, 2]]) + >>> kser = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], index=midx) + + >>> kser.keys() # doctest: +SKIP + MultiIndex([( 'lama', 'speed'), + ( 'lama', 'weight'), + ( 'lama', 'length'), + ( 'cow', 'speed'), + ( 'cow', 'weight'), + ( 'cow', 'length'), + ('falcon', 'speed'), + ('falcon', 'weight'), + ('falcon', 'length')], + ) + """ + return self.index + + # TODO: 'regex', 'method' parameter + def replace(self, to_replace=None, value=None, regex=False) -> "Series": + """ + Replace values given in to_replace with value. + Values of the Series are replaced with other values dynamically. + + Parameters + ---------- + to_replace : str, list, tuple, dict, Series, int, float, or None + How to find the values that will be replaced. + * numeric, str: + + - numeric: numeric values equal to to_replace will be replaced with value + - str: string exactly matching to_replace will be replaced with value + + * list of str or numeric: + + - if to_replace and value are both lists or tuples, they must be the same length. + - str and numeric rules apply as above. + + * dict: + + - Dicts can be used to specify different replacement values for different + existing values. + For example, {'a': 'b', 'y': 'z'} replaces the value ‘a’ with ‘b’ and ‘y’ + with ‘z’. To use a dict in this way the value parameter should be None. + - For a DataFrame a dict can specify that different values should be replaced + in different columns. For example, {'a': 1, 'b': 'z'} looks for the value 1 + in column ‘a’ and the value ‘z’ in column ‘b’ and replaces these values with + whatever is specified in value. + The value parameter should not be None in this case. + You can treat this as a special case of passing two lists except that you are + specifying the column to search in. + + See the examples section for examples of each of these. + + value : scalar, dict, list, tuple, str default None + Value to replace any values matching to_replace with. + For a DataFrame a dict of values can be used to specify which value to use + for each column (columns not in the dict will not be filled). + Regular expressions, strings and lists or dicts of such objects are also allowed. + + Returns + ------- + Series + Object after replacement. + + Examples + -------- + + Scalar `to_replace` and `value` + + >>> s = pp.Series([0, 1, 2, 3, 4]) + >>> s + 0 0 + 1 1 + 2 2 + 3 3 + 4 4 + dtype: int64 + + >>> s.replace(0, 5) + 0 5 + 1 1 + 2 2 + 3 3 + 4 4 + dtype: int64 + + List-like `to_replace` + + >>> s.replace([0, 4], 5000) + 0 5000 + 1 1 + 2 2 + 3 3 + 4 5000 + dtype: int64 + + >>> s.replace([1, 2, 3], [10, 20, 30]) + 0 0 + 1 10 + 2 20 + 3 30 + 4 4 + dtype: int64 + + Dict-like `to_replace` + + >>> s.replace({1: 1000, 2: 2000, 3: 3000, 4: 4000}) + 0 0 + 1 1000 + 2 2000 + 3 3000 + 4 4000 + dtype: int64 + + Also support for MultiIndex + + >>> midx = pd.MultiIndex([['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 1, 2]]) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], + ... index=midx) + >>> s + lama speed 45.0 + weight 200.0 + length 1.2 + cow speed 30.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + >>> s.replace(45, 450) + lama speed 450.0 + weight 200.0 + length 1.2 + cow speed 30.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + >>> s.replace([45, 30, 320], 500) + lama speed 500.0 + weight 200.0 + length 1.2 + cow speed 500.0 + weight 250.0 + length 1.5 + falcon speed 500.0 + weight 1.0 + length 0.3 + dtype: float64 + + >>> s.replace({45: 450, 30: 300}) + lama speed 450.0 + weight 200.0 + length 1.2 + cow speed 300.0 + weight 250.0 + length 1.5 + falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + """ + if to_replace is None: + return self.fillna(method="ffill") + if not isinstance(to_replace, (str, list, tuple, dict, int, float)): + raise ValueError("'to_replace' should be one of str, list, tuple, dict, int, float") + if regex: + raise NotImplementedError("replace currently not support for regex") + to_replace = list(to_replace) if isinstance(to_replace, tuple) else to_replace + value = list(value) if isinstance(value, tuple) else value + if isinstance(to_replace, list) and isinstance(value, list): + if not len(to_replace) == len(value): + raise ValueError( + "Replacement lists must match in length. Expecting {} got {}".format( + len(to_replace), len(value) + ) + ) + to_replace = {k: v for k, v in zip(to_replace, value)} + if isinstance(to_replace, dict): + is_start = True + if len(to_replace) == 0: + current = self.spark.column + else: + for to_replace_, value in to_replace.items(): + cond = ( + (F.isnan(self.spark.column) | self.spark.column.isNull()) + if pd.isna(to_replace_) + else (self.spark.column == F.lit(to_replace_)) + ) + if is_start: + current = F.when(cond, value) + is_start = False + else: + current = current.when(cond, value) + current = current.otherwise(self.spark.column) + else: + cond = self.spark.column.isin(to_replace) + # to_replace may be a scalar + if np.array(pd.isna(to_replace)).any(): + cond = cond | F.isnan(self.spark.column) | self.spark.column.isNull() + current = F.when(cond, value).otherwise(self.spark.column) + + return self._with_new_scol(current) # TODO: dtype? + + def update(self, other) -> None: + """ + Modify Series in place using non-NA values from passed Series. Aligns on index. + + Parameters + ---------- + other : Series + + Examples + -------- + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + >>> s = pp.Series([1, 2, 3]) + >>> s.update(pp.Series([4, 5, 6])) + >>> s.sort_index() + 0 4 + 1 5 + 2 6 + dtype: int64 + + >>> s = pp.Series(['a', 'b', 'c']) + >>> s.update(pp.Series(['d', 'e'], index=[0, 2])) + >>> s.sort_index() + 0 d + 1 b + 2 e + dtype: object + + >>> s = pp.Series([1, 2, 3]) + >>> s.update(pp.Series([4, 5, 6, 7, 8])) + >>> s.sort_index() + 0 4 + 1 5 + 2 6 + dtype: int64 + + >>> s = pp.Series([1, 2, 3], index=[10, 11, 12]) + >>> s + 10 1 + 11 2 + 12 3 + dtype: int64 + + >>> s.update(pp.Series([4, 5, 6])) + >>> s.sort_index() + 10 1 + 11 2 + 12 3 + dtype: int64 + + >>> s.update(pp.Series([4, 5, 6], index=[11, 12, 13])) + >>> s.sort_index() + 10 1 + 11 4 + 12 5 + dtype: int64 + + If ``other`` contains NaNs the corresponding values are not updated + in the original Series. + + >>> s = pp.Series([1, 2, 3]) + >>> s.update(pp.Series([4, np.nan, 6])) + >>> s.sort_index() + 0 4.0 + 1 2.0 + 2 6.0 + dtype: float64 + + >>> reset_option("compute.ops_on_diff_frames") + """ + if not isinstance(other, Series): + raise ValueError("'other' must be a Series") + + combined = combine_frames(self._kdf, other._kdf, how="leftouter") + + this_scol = combined["this"]._internal.spark_column_for(self._column_label) + that_scol = combined["that"]._internal.spark_column_for(other._column_label) + + scol = ( + F.when(that_scol.isNotNull(), that_scol) + .otherwise(this_scol) + .alias(self._kdf._internal.spark_column_name_for(self._column_label)) + ) + + internal = combined["this"]._internal.with_new_spark_column( + self._column_label, scol # TODO: dtype? + ) + + self._kdf._update_internal_frame(internal.resolved_copy, requires_same_anchor=False) + + def where(self, cond, other=np.nan) -> "Series": + """ + Replace values where the condition is False. + + Parameters + ---------- + cond : boolean Series + Where cond is True, keep the original value. Where False, + replace with corresponding value from other. + other : scalar, Series + Entries where cond is False are replaced with corresponding value from other. + + Returns + ------- + Series + + Examples + -------- + + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + >>> s1 = pp.Series([0, 1, 2, 3, 4]) + >>> s2 = pp.Series([100, 200, 300, 400, 500]) + >>> s1.where(s1 > 0).sort_index() + 0 NaN + 1 1.0 + 2 2.0 + 3 3.0 + 4 4.0 + dtype: float64 + + >>> s1.where(s1 > 1, 10).sort_index() + 0 10 + 1 10 + 2 2 + 3 3 + 4 4 + dtype: int64 + + >>> s1.where(s1 > 1, s1 + 100).sort_index() + 0 100 + 1 101 + 2 2 + 3 3 + 4 4 + dtype: int64 + + >>> s1.where(s1 > 1, s2).sort_index() + 0 100 + 1 200 + 2 2 + 3 3 + 4 4 + dtype: int64 + + >>> reset_option("compute.ops_on_diff_frames") + """ + assert isinstance(cond, Series) + + # We should check the DataFrame from both `cond` and `other`. + should_try_ops_on_diff_frame = not same_anchor(cond, self) or ( + isinstance(other, Series) and not same_anchor(other, self) + ) + + if should_try_ops_on_diff_frame: + # Try to perform it with 'compute.ops_on_diff_frame' option. + kdf = self.to_frame() + tmp_cond_col = verify_temp_column_name(kdf, "__tmp_cond_col__") + tmp_other_col = verify_temp_column_name(kdf, "__tmp_other_col__") + + kdf[tmp_cond_col] = cond + kdf[tmp_other_col] = other + + # above logic makes a Spark DataFrame looks like below: + # +-----------------+---+----------------+-----------------+ + # |__index_level_0__| 0|__tmp_cond_col__|__tmp_other_col__| + # +-----------------+---+----------------+-----------------+ + # | 0| 0| false| 100| + # | 1| 1| false| 200| + # | 3| 3| true| 400| + # | 2| 2| true| 300| + # | 4| 4| true| 500| + # +-----------------+---+----------------+-----------------+ + condition = ( + F.when( + kdf[tmp_cond_col].spark.column, + kdf._kser_for(kdf._internal.column_labels[0]).spark.column, + ) + .otherwise(kdf[tmp_other_col].spark.column) + .alias(kdf._internal.data_spark_column_names[0]) + ) + + internal = kdf._internal.with_new_columns( + [condition], column_labels=self._internal.column_labels + ) + return first_series(DataFrame(internal)) + else: + if isinstance(other, Series): + other = other.spark.column + condition = ( + F.when(cond.spark.column, self.spark.column) + .otherwise(other) + .alias(self._internal.data_spark_column_names[0]) + ) + return self._with_new_scol(condition) + + def mask(self, cond, other=np.nan) -> "Series": + """ + Replace values where the condition is True. + + Parameters + ---------- + cond : boolean Series + Where cond is False, keep the original value. Where True, + replace with corresponding value from other. + other : scalar, Series + Entries where cond is True are replaced with corresponding value from other. + + Returns + ------- + Series + + Examples + -------- + + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + >>> s1 = pp.Series([0, 1, 2, 3, 4]) + >>> s2 = pp.Series([100, 200, 300, 400, 500]) + >>> s1.mask(s1 > 0).sort_index() + 0 0.0 + 1 NaN + 2 NaN + 3 NaN + 4 NaN + dtype: float64 + + >>> s1.mask(s1 > 1, 10).sort_index() + 0 0 + 1 1 + 2 10 + 3 10 + 4 10 + dtype: int64 + + >>> s1.mask(s1 > 1, s1 + 100).sort_index() + 0 0 + 1 1 + 2 102 + 3 103 + 4 104 + dtype: int64 + + >>> s1.mask(s1 > 1, s2).sort_index() + 0 0 + 1 1 + 2 300 + 3 400 + 4 500 + dtype: int64 + + >>> reset_option("compute.ops_on_diff_frames") + """ + return self.where(~cond, other) + + def xs(self, key, level=None) -> "Series": + """ + Return cross-section from the Series. + + This method takes a `key` argument to select data at a particular + level of a MultiIndex. + + Parameters + ---------- + key : label or tuple of label + Label contained in the index, or partially in a MultiIndex. + level : object, defaults to first n levels (n=1 or len(key)) + In case of a key partially contained in a MultiIndex, indicate + which levels are used. Levels can be referred by label or position. + + Returns + ------- + Series + Cross-section from the original Series + corresponding to the selected index levels. + + Examples + -------- + >>> midx = pd.MultiIndex([['a', 'b', 'c'], + ... ['lama', 'cow', 'falcon'], + ... ['speed', 'weight', 'length']], + ... [[0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 0, 0, 1, 1, 1, 2, 2, 2], + ... [0, 1, 2, 0, 1, 2, 0, 1, 2]]) + >>> s = pp.Series([45, 200, 1.2, 30, 250, 1.5, 320, 1, 0.3], + ... index=midx) + >>> s + a lama speed 45.0 + weight 200.0 + length 1.2 + b cow speed 30.0 + weight 250.0 + length 1.5 + c falcon speed 320.0 + weight 1.0 + length 0.3 + dtype: float64 + + Get values at specified index + + >>> s.xs('a') + lama speed 45.0 + weight 200.0 + length 1.2 + dtype: float64 + + Get values at several indexes + + >>> s.xs(('a', 'lama')) + speed 45.0 + weight 200.0 + length 1.2 + dtype: float64 + + Get values at specified index and level + + >>> s.xs('lama', level=1) + a speed 45.0 + weight 200.0 + length 1.2 + dtype: float64 + """ + if not isinstance(key, tuple): + key = (key,) + if level is None: + level = 0 + + internal = self._internal + scols = ( + internal.index_spark_columns[:level] + + internal.index_spark_columns[level + len(key):] + + [self.spark.column] + ) + rows = [internal.spark_columns[lvl] == index for lvl, index in enumerate(key, level)] + sdf = internal.spark_frame.filter(reduce(lambda x, y: x & y, rows)).select(scols) + + if internal.index_level == len(key): + # if spark_frame has one column and one data, return data only without frame + pdf = sdf.limit(2).toPandas() + length = len(pdf) + if length == 1: + return pdf[self._internal.data_spark_column_names[0]].iloc[0] + + index_spark_column_names = ( + internal.index_spark_column_names[:level] + + internal.index_spark_column_names[level + len(key):] + ) + index_names = internal.index_names[:level] + internal.index_names[level + len(key):] + index_dtypes = internal.index_dtypes[:level] + internal.index_dtypes[level + len(key):] + + internal = internal.copy( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_spark_column_names], + index_names=index_names, + index_dtypes=index_dtypes, + data_spark_columns=[scol_for(sdf, internal.data_spark_column_names[0])], + ) + return first_series(DataFrame(internal)) + + def pct_change(self, periods=1) -> "Series": + """ + Percentage change between the current and a prior element. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Parameters + ---------- + periods : int, default 1 + Periods to shift for forming percent change. + + Returns + ------- + Series + + Examples + -------- + + >>> kser = pp.Series([90, 91, 85], index=[2, 4, 1]) + >>> kser + 2 90 + 4 91 + 1 85 + dtype: int64 + + >>> kser.pct_change() + 2 NaN + 4 0.011111 + 1 -0.065934 + dtype: float64 + + >>> kser.sort_index().pct_change() + 1 NaN + 2 0.058824 + 4 0.011111 + dtype: float64 + + >>> kser.pct_change(periods=2) + 2 NaN + 4 NaN + 1 -0.055556 + dtype: float64 + """ + scol = self.spark.column + + window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween(-periods, -periods) + prev_row = F.lag(scol, periods).over(window) + + return self._with_new_scol((scol - prev_row) / prev_row).spark.analyzed + + def combine_first(self, other) -> "Series": + """ + Combine Series values, choosing the calling Series's values first. + + Parameters + ---------- + other : Series + The value(s) to be combined with the `Series`. + + Returns + ------- + Series + The result of combining the Series with the other object. + + See Also + -------- + Series.combine : Perform elementwise operation on two Series + using a given function. + + Notes + ----- + Result index will be the union of the two indexes. + + Examples + -------- + >>> s1 = pp.Series([1, np.nan]) + >>> s2 = pp.Series([3, 4]) + >>> with pp.option_context("compute.ops_on_diff_frames", True): + ... s1.combine_first(s2) + 0 1.0 + 1 4.0 + dtype: float64 + """ + if not isinstance(other, pp.Series): + raise ValueError("`combine_first` only allows `Series` for parameter `other`") + if same_anchor(self, other): + this = self.spark.column + that = other.spark.column + combined = self._kdf + else: + combined = combine_frames(self._kdf, other._kdf) + this = combined["this"]._internal.spark_column_for(self._column_label) + that = combined["that"]._internal.spark_column_for(other._column_label) + # If `self` has missing value, use value of `other` + cond = F.when(this.isNull(), that).otherwise(this) + # If `self` and `other` come from same frame, the anchor should be kept + if same_anchor(self, other): + return self._with_new_scol(cond) # TODO: dtype? + index_scols = combined._internal.index_spark_columns + sdf = combined._internal.spark_frame.select( + *index_scols, cond.alias(self._internal.data_spark_column_names[0]) + ).distinct() + internal = self._internal.with_new_sdf(sdf, data_dtypes=[None]) # TODO: dtype? + return first_series(DataFrame(internal)) + + def dot(self, other: Union["Series", DataFrame]) -> Union[Scalar, "Series"]: + """ + Compute the dot product between the Series and the columns of other. + + This method computes the dot product between the Series and another + one, or the Series and each columns of a DataFrame. + + It can also be called using `self @ other` in Python >= 3.5. + + .. note:: This API is slightly different from pandas when indexes from both Series + are not aligned. To match with pandas', it requires to read the whole data for, + for example, counting. pandas raises an exception; however, Koalas just proceeds + and performs by ignoring mismatches with NaN permissively. + + >>> pdf1 = pd.Series([1, 2, 3], index=[0, 1, 2]) + >>> pdf2 = pd.Series([1, 2, 3], index=[0, 1, 3]) + >>> pdf1.dot(pdf2) # doctest: +SKIP + ... + ValueError: matrices are not aligned + + >>> kdf1 = pp.Series([1, 2, 3], index=[0, 1, 2]) + >>> kdf2 = pp.Series([1, 2, 3], index=[0, 1, 3]) + >>> kdf1.dot(kdf2) # doctest: +SKIP + 5 + + Parameters + ---------- + other : Series, DataFrame. + The other object to compute the dot product with its columns. + + Returns + ------- + scalar, Series + Return the dot product of the Series and other if other is a + Series, the Series of the dot product of Series and each rows of + other if other is a DataFrame. + + Notes + ----- + The Series and other has to share the same index if other is a Series + or a DataFrame. + + Examples + -------- + >>> s = pp.Series([0, 1, 2, 3]) + + >>> s.dot(s) + 14 + + >>> s @ s + 14 + + >>> kdf = pp.DataFrame({'x': [0, 1, 2, 3], 'y': [0, -1, -2, -3]}) + >>> kdf + x y + 0 0 0 + 1 1 -1 + 2 2 -2 + 3 3 -3 + + >>> with pp.option_context("compute.ops_on_diff_frames", True): + ... s.dot(kdf) + ... + x 14 + y -14 + dtype: int64 + """ + if isinstance(other, DataFrame): + if not same_anchor(self, other): + if not self.index.sort_values().equals(other.index.sort_values()): + raise ValueError("matrices are not aligned") + + other = other.copy() + column_labels = other._internal.column_labels + + self_column_label = verify_temp_column_name(other, "__self_column__") + other[self_column_label] = self + self_kser = other._kser_for(self_column_label) + + product_ksers = [other._kser_for(label) * self_kser for label in column_labels] + + dot_product_kser = DataFrame( + other._internal.with_new_columns(product_ksers, column_labels=column_labels) + ).sum() + + return cast(Series, dot_product_kser).rename(self.name) + + else: + assert isinstance(other, Series) + if not same_anchor(self, other): + if len(self.index) != len(other.index): + raise ValueError("matrices are not aligned") + return (self * other).sum() + + def __matmul__(self, other): + """ + Matrix multiplication using binary `@` operator in Python>=3.5. + """ + return self.dot(other) + + def repeat(self, repeats: Union[int, "Series"]) -> "Series": + """ + Repeat elements of a Series. + + Returns a new Series where each element of the current Series + is repeated consecutively a given number of times. + + Parameters + ---------- + repeats : int or Series + The number of repetitions for each element. This should be a + non-negative integer. Repeating 0 times will return an empty + Series. + + Returns + ------- + Series + Newly created Series with repeated elements. + + See Also + -------- + Index.repeat : Equivalent function for Index. + + Examples + -------- + >>> s = pp.Series(['a', 'b', 'c']) + >>> s + 0 a + 1 b + 2 c + dtype: object + >>> s.repeat(2) + 0 a + 1 b + 2 c + 0 a + 1 b + 2 c + dtype: object + >>> pp.Series([1, 2, 3]).repeat(0) + Series([], dtype: int64) + """ + if not isinstance(repeats, (int, Series)): + raise ValueError( + "`repeats` argument must be integer or Series, but got {}".format(type(repeats)) + ) + + if isinstance(repeats, Series): + if LooseVersion(pyspark.__version__) < LooseVersion("2.4"): + raise ValueError( + "`repeats` argument must be integer with Spark<2.4, but got {}".format( + type(repeats) + ) + ) + if not same_anchor(self, repeats): + kdf = self.to_frame() + temp_repeats = verify_temp_column_name(kdf, "__temp_repeats__") + kdf[temp_repeats] = repeats + return ( + kdf._kser_for(kdf._internal.column_labels[0]) + .repeat(kdf[temp_repeats]) + .rename(self.name) + ) + else: + scol = F.explode( + SF.array_repeat(self.spark.column, repeats.astype("int32").spark.column) + ).alias(name_like_string(self.name)) + sdf = self._internal.spark_frame.select(self._internal.index_spark_columns + [scol]) + internal = self._internal.copy( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + data_spark_columns=[scol_for(sdf, name_like_string(self.name))], + ) + return first_series(DataFrame(internal)) + else: + if repeats < 0: + raise ValueError("negative dimensions are not allowed") + + kdf = self._kdf[[self.name]] + if repeats == 0: + return first_series(DataFrame(kdf._internal.with_filter(F.lit(False)))) + else: + return first_series(pp.concat([kdf] * repeats)) + + def asof(self, where) -> Union[Scalar, "Series"]: + """ + Return the last row(s) without any NaNs before `where`. + + The last row (for each element in `where`, if list) without any + NaN is taken. + + If there is no good value, NaN is returned. + + .. note:: This API is dependent on :meth:`Index.is_monotonic_increasing` + which can be expensive. + + Parameters + ---------- + where : index or array-like of indices + + Returns + ------- + scalar or Series + + The return can be: + + * scalar : when `self` is a Series and `where` is a scalar + * Series: when `self` is a Series and `where` is an array-like + + Return scalar or Series + + Notes + ----- + Indices are assumed to be sorted. Raises if this is not the case. + + Examples + -------- + >>> s = pp.Series([1, 2, np.nan, 4], index=[10, 20, 30, 40]) + >>> s + 10 1.0 + 20 2.0 + 30 NaN + 40 4.0 + dtype: float64 + + A scalar `where`. + + >>> s.asof(20) + 2.0 + + For a sequence `where`, a Series is returned. The first value is + NaN, because the first element of `where` is before the first + index value. + + >>> s.asof([5, 20]).sort_index() + 5 NaN + 20 2.0 + dtype: float64 + + Missing values are not considered. The following is ``2.0``, not + NaN, even though NaN is at the index location for ``30``. + + >>> s.asof(30) + 2.0 + """ + should_return_series = True + if isinstance(self.index, pp.MultiIndex): + raise ValueError("asof is not supported for a MultiIndex") + if isinstance(where, (pp.Index, pp.Series, DataFrame)): + raise ValueError("where cannot be an Index, Series or a DataFrame") + if not self.index.is_monotonic_increasing: + raise ValueError("asof requires a sorted index") + if not is_list_like(where): + should_return_series = False + where = [where] + index_scol = self._internal.index_spark_columns[0] + index_type = self._internal.spark_type_for(index_scol) + cond = [ + F.max(F.when(index_scol <= F.lit(index).cast(index_type), self.spark.column)) + for index in where + ] + sdf = self._internal.spark_frame.select(cond) + if not should_return_series: + with sql_conf({SPARK_CONF_ARROW_ENABLED: False}): + # Disable Arrow to keep row ordering. + result = sdf.limit(1).toPandas().iloc[0, 0] + return result if result is not None else np.nan + + # The data is expected to be small so it's fine to transpose/use default index. + with pp.option_context("compute.default_index_type", "distributed", "compute.max_rows", 1): + kdf = pp.DataFrame(sdf) # type: DataFrame + kdf.columns = pd.Index(where) + return first_series(kdf.transpose()).rename(self.name) + + def mad(self) -> float: + """ + Return the mean absolute deviation of values. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4]) + >>> s + 0 1 + 1 2 + 2 3 + 3 4 + dtype: int64 + + >>> s.mad() + 1.0 + """ + + sdf = self._internal.spark_frame + spark_column = self.spark.column + avg = unpack_scalar(sdf.select(F.avg(spark_column))) + mad = unpack_scalar(sdf.select(F.avg(F.abs(spark_column - avg)))) + + return mad + + def unstack(self, level=-1) -> DataFrame: + """ + Unstack, a.k.a. pivot, Series with MultiIndex to produce DataFrame. + The level involved will automatically get sorted. + + Notes + ----- + Unlike pandas, Koalas doesn't check whether an index is duplicated or not + because the checking of duplicated index requires scanning whole data which + can be quite expensive. + + Parameters + ---------- + level : int, str, or list of these, default last level + Level(s) to unstack, can pass level name. + + Returns + ------- + DataFrame + Unstacked Series. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4], + ... index=pd.MultiIndex.from_product([['one', 'two'], + ... ['a', 'b']])) + >>> s + one a 1 + b 2 + two a 3 + b 4 + dtype: int64 + + >>> s.unstack(level=-1).sort_index() + a b + one 1 2 + two 3 4 + + >>> s.unstack(level=0).sort_index() + one two + a 1 3 + b 2 4 + """ + if not isinstance(self.index, pp.MultiIndex): + raise ValueError("Series.unstack only support for a MultiIndex") + index_nlevels = self.index.nlevels + if level > 0 and (level > index_nlevels - 1): + raise IndexError( + "Too many levels: Index has only {} levels, not {}".format(index_nlevels, level + 1) + ) + elif level < 0 and (level < -index_nlevels): + raise IndexError( + "Too many levels: Index has only {} levels, {} is not a valid level number".format( + index_nlevels, level + ) + ) + + internal = self._internal.resolved_copy + + index_map = list(zip(internal.index_spark_column_names, internal.index_names)) + pivot_col, column_label_names = index_map.pop(level) + index_scol_names, index_names = zip(*index_map) + col = internal.data_spark_column_names[0] + + sdf = internal.spark_frame + sdf = sdf.groupby(list(index_scol_names)).pivot(pivot_col).agg(F.first(scol_for(sdf, col))) + internal = InternalFrame( # TODO: dtypes? + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in index_scol_names], + index_names=list(index_names), + column_label_names=[column_label_names], + ) + return DataFrame(internal) + + def item(self) -> Scalar: + """ + Return the first element of the underlying data as a Python scalar. + + Returns + ------- + scalar + The first element of Series. + + Raises + ------ + ValueError + If the data is not length-1. + + Examples + -------- + >>> kser = pp.Series([10]) + >>> kser.item() + 10 + """ + return self.head(2)._to_internal_pandas().item() + + def iteritems(self) -> Iterable: + """ + Lazily iterate over (index, value) tuples. + + This method returns an iterable tuple (index, value). This is + convenient if you want to create a lazy iterator. + + .. note:: Unlike pandas', the iteritems in Koalas returns generator rather zip object + + Returns + ------- + iterable + Iterable of tuples containing the (index, value) pairs from a + Series. + + See Also + -------- + DataFrame.items : Iterate over (column name, Series) pairs. + DataFrame.iterrows : Iterate over DataFrame rows as (index, Series) pairs. + + Examples + -------- + >>> s = pp.Series(['A', 'B', 'C']) + >>> for index, value in s.items(): + ... print("Index : {}, Value : {}".format(index, value)) + Index : 0, Value : A + Index : 1, Value : B + Index : 2, Value : C + """ + internal_index_columns = self._internal.index_spark_column_names + internal_data_column = self._internal.data_spark_column_names[0] + + def extract_kv_from_spark_row(row): + k = ( + row[internal_index_columns[0]] + if len(internal_index_columns) == 1 + else tuple(row[c] for c in internal_index_columns) + ) + v = row[internal_data_column] + return k, v + + for k, v in map( + extract_kv_from_spark_row, self._internal.resolved_copy.spark_frame.toLocalIterator() + ): + yield k, v + + def items(self) -> Iterable: + """This is an alias of ``iteritems``.""" + return self.iteritems() + + def droplevel(self, level) -> "Series": + """ + Return Series with requested index level(s) removed. + + Parameters + ---------- + level : int, str, or list-like + If a string is given, must be the name of a level + If list-like, elements must be names or positional indexes + of levels. + + Returns + ------- + Series + Series with requested index level(s) removed. + + Examples + -------- + >>> kser = pp.Series( + ... [1, 2, 3], + ... index=pd.MultiIndex.from_tuples( + ... [("x", "a"), ("x", "b"), ("y", "c")], names=["level_1", "level_2"] + ... ), + ... ) + >>> kser + level_1 level_2 + x a 1 + b 2 + y c 3 + dtype: int64 + + Removing specific index level by level + + >>> kser.droplevel(0) + level_2 + a 1 + b 2 + c 3 + dtype: int64 + + Removing specific index level by name + + >>> kser.droplevel("level_2") + level_1 + x 1 + x 2 + y 3 + dtype: int64 + """ + return first_series(self.to_frame().droplevel(level=level, axis=0)).rename(self.name) + + def tail(self, n=5) -> "Series": + """ + Return the last `n` rows. + + This function returns last `n` rows from the object based on + position. It is useful for quickly verifying data, for example, + after sorting or appending rows. + + For negative values of `n`, this function returns all rows except + the first `n` rows, equivalent to ``df[n:]``. + + Parameters + ---------- + n : int, default 5 + Number of rows to select. + + Returns + ------- + type of caller + The last `n` rows of the caller object. + + See Also + -------- + DataFrame.head : The first `n` rows of the caller object. + + Examples + -------- + >>> kser = pp.Series([1, 2, 3, 4, 5]) + >>> kser + 0 1 + 1 2 + 2 3 + 3 4 + 4 5 + dtype: int64 + + >>> kser.tail(3) # doctest: +SKIP + 2 3 + 3 4 + 4 5 + dtype: int64 + """ + return first_series(self.to_frame().tail(n=n)).rename(self.name) + + def explode(self) -> "Series": + """ + Transform each element of a list-like to a row. + + Returns + ------- + Series + Exploded lists to rows; index will be duplicated for these rows. + + See Also + -------- + Series.str.split : Split string values on specified separator. + Series.unstack : Unstack, a.k.a. pivot, Series with MultiIndex + to produce DataFrame. + DataFrame.melt : Unpivot a DataFrame from wide format to long format. + DataFrame.explode : Explode a DataFrame from list-like + columns to long format. + + Examples + -------- + >>> kser = pp.Series([[1, 2, 3], [], [3, 4]]) + >>> kser + 0 [1, 2, 3] + 1 [] + 2 [3, 4] + dtype: object + + >>> kser.explode() # doctest: +SKIP + 0 1.0 + 0 2.0 + 0 3.0 + 1 NaN + 2 3.0 + 2 4.0 + dtype: float64 + """ + if not isinstance(self.spark.data_type, ArrayType): + return self.copy() + + scol = F.explode_outer(self.spark.column).alias(name_like_string(self._column_label)) + + internal = self._internal.with_new_columns([scol], keep_order=False) + return first_series(DataFrame(internal)) + + def argsort(self) -> "Series": + """ + Return the integer indices that would sort the Series values. + Unlike pandas, the index order is not preserved in the result. + + Returns + ------- + Series + Positions of values within the sort order with -1 indicating + nan values. + + Examples + -------- + >>> kser = pp.Series([3, 3, 4, 1, 6, 2, 3, 7, 8, 7, 10]) + >>> kser + 0 3 + 1 3 + 2 4 + 3 1 + 4 6 + 5 2 + 6 3 + 7 7 + 8 8 + 9 7 + 10 10 + dtype: int64 + + >>> kser.argsort().sort_index() + 0 3 + 1 5 + 2 0 + 3 1 + 4 6 + 5 2 + 6 4 + 7 7 + 8 9 + 9 8 + 10 10 + dtype: int64 + """ + notnull = self.loc[self.notnull()] + + sdf_for_index = notnull._internal.spark_frame.select(notnull._internal.index_spark_columns) + + tmp_join_key = verify_temp_column_name(sdf_for_index, "__tmp_join_key__") + sdf_for_index = InternalFrame.attach_distributed_sequence_column( + sdf_for_index, tmp_join_key + ) + # sdf_for_index: + # +----------------+-----------------+ + # |__tmp_join_key__|__index_level_0__| + # +----------------+-----------------+ + # | 0| 0| + # | 1| 1| + # | 2| 2| + # | 3| 3| + # | 4| 4| + # +----------------+-----------------+ + + sdf_for_data = notnull._internal.spark_frame.select( + notnull.spark.column.alias("values"), NATURAL_ORDER_COLUMN_NAME + ) + sdf_for_data = InternalFrame.attach_distributed_sequence_column( + sdf_for_data, SPARK_DEFAULT_SERIES_NAME + ) + # sdf_for_data: + # +---+------+-----------------+ + # | 0|values|__natural_order__| + # +---+------+-----------------+ + # | 0| 3| 25769803776| + # | 1| 3| 51539607552| + # | 2| 4| 77309411328| + # | 3| 1| 103079215104| + # | 4| 2| 128849018880| + # +---+------+-----------------+ + + sdf_for_data = sdf_for_data.sort( + scol_for(sdf_for_data, "values"), NATURAL_ORDER_COLUMN_NAME + ).drop("values", NATURAL_ORDER_COLUMN_NAME) + + tmp_join_key = verify_temp_column_name(sdf_for_data, "__tmp_join_key__") + sdf_for_data = InternalFrame.attach_distributed_sequence_column(sdf_for_data, tmp_join_key) + # sdf_for_index: sdf_for_data: + # +----------------+-----------------+ +----------------+---+ + # |__tmp_join_key__|__index_level_0__| |__tmp_join_key__| 0| + # +----------------+-----------------+ +----------------+---+ + # | 0| 0| | 0| 3| + # | 1| 1| | 1| 4| + # | 2| 2| | 2| 0| + # | 3| 3| | 3| 1| + # | 4| 4| | 4| 2| + # +----------------+-----------------+ +----------------+---+ + + sdf = sdf_for_index.join(sdf_for_data, on=tmp_join_key).drop(tmp_join_key) + + internal = self._internal.with_new_sdf( + spark_frame=sdf, data_columns=[SPARK_DEFAULT_SERIES_NAME], data_dtypes=[None] + ) + kser = first_series(DataFrame(internal)) + + return cast( + Series, pp.concat([kser, self.loc[self.isnull()].spark.transform(lambda _: F.lit(-1))]) + ) + + def argmax(self) -> int: + """ + Return int position of the largest value in the Series. + + If the maximum is achieved in multiple locations, + the first row position is returned. + + Returns + ------- + int + Row position of the maximum value. + + Examples + -------- + Consider dataset containing cereal calories + + >>> s = pp.Series({'Corn Flakes': 100.0, 'Almond Delight': 110.0, + ... 'Cinnamon Toast Crunch': 120.0, 'Cocoa Puff': 110.0}) + >>> s # doctest: +SKIP + Corn Flakes 100.0 + Almond Delight 110.0 + Cinnamon Toast Crunch 120.0 + Cocoa Puff 110.0 + dtype: float64 + + >>> s.argmax() # doctest: +SKIP + 2 + """ + sdf = self._internal.spark_frame.select(self.spark.column, NATURAL_ORDER_COLUMN_NAME) + max_value = sdf.select( + F.max(scol_for(sdf, self._internal.data_spark_column_names[0])), + F.first(NATURAL_ORDER_COLUMN_NAME), + ).head() + if max_value[1] is None: + raise ValueError("attempt to get argmax of an empty sequence") + elif max_value[0] is None: + return -1 + # We should remember the natural sequence started from 0 + seq_col_name = verify_temp_column_name(sdf, "__distributed_sequence_column__") + sdf = InternalFrame.attach_distributed_sequence_column( + sdf.drop(NATURAL_ORDER_COLUMN_NAME), seq_col_name + ) + # If the maximum is achieved in multiple locations, the first row position is returned. + return sdf.filter( + scol_for(sdf, self._internal.data_spark_column_names[0]) == max_value[0] + ).head()[0] + + def argmin(self) -> int: + """ + Return int position of the smallest value in the Series. + + If the minimum is achieved in multiple locations, + the first row position is returned. + + Returns + ------- + int + Row position of the minimum value. + + Examples + -------- + Consider dataset containing cereal calories + + >>> s = pp.Series({'Corn Flakes': 100.0, 'Almond Delight': 110.0, + ... 'Cinnamon Toast Crunch': 120.0, 'Cocoa Puff': 110.0}) + >>> s # doctest: +SKIP + Corn Flakes 100.0 + Almond Delight 110.0 + Cinnamon Toast Crunch 120.0 + Cocoa Puff 110.0 + dtype: float64 + + >>> s.argmin() # doctest: +SKIP + 0 + """ + sdf = self._internal.spark_frame.select(self.spark.column, NATURAL_ORDER_COLUMN_NAME) + min_value = sdf.select( + F.min(scol_for(sdf, self._internal.data_spark_column_names[0])), + F.first(NATURAL_ORDER_COLUMN_NAME), + ).head() + if min_value[1] is None: + raise ValueError("attempt to get argmin of an empty sequence") + elif min_value[0] is None: + return -1 + # We should remember the natural sequence started from 0 + seq_col_name = verify_temp_column_name(sdf, "__distributed_sequence_column__") + sdf = InternalFrame.attach_distributed_sequence_column( + sdf.drop(NATURAL_ORDER_COLUMN_NAME), seq_col_name + ) + # If the minimum is achieved in multiple locations, the first row position is returned. + return sdf.filter( + scol_for(sdf, self._internal.data_spark_column_names[0]) == min_value[0] + ).head()[0] + + def compare( + self, other: "Series", keep_shape: bool = False, keep_equal: bool = False + ) -> DataFrame: + """ + Compare to another Series and show the differences. + + Parameters + ---------- + other : Series + Object to compare with. + keep_shape : bool, default False + If true, all rows and columns are kept. + Otherwise, only the ones with different values are kept. + keep_equal : bool, default False + If true, the result keeps values that are equal. + Otherwise, equal values are shown as NaNs. + + Returns + ------- + DataFrame + + Notes + ----- + Matching NaNs will not appear as a difference. + + Examples + -------- + + >>> from pyspark.pandas.config import set_option, reset_option + >>> set_option("compute.ops_on_diff_frames", True) + >>> s1 = pp.Series(["a", "b", "c", "d", "e"]) + >>> s2 = pp.Series(["a", "a", "c", "b", "e"]) + + Align the differences on columns + + >>> s1.compare(s2).sort_index() + self other + 1 b a + 3 d b + + Keep all original rows + + >>> s1.compare(s2, keep_shape=True).sort_index() + self other + 0 None None + 1 b a + 2 None None + 3 d b + 4 None None + + Keep all original rows and also all original values + + >>> s1.compare(s2, keep_shape=True, keep_equal=True).sort_index() + self other + 0 a a + 1 b a + 2 c c + 3 d b + 4 e e + + >>> reset_option("compute.ops_on_diff_frames") + """ + if same_anchor(self, other): + self_column_label = verify_temp_column_name(other.to_frame(), "__self_column__") + other_column_label = verify_temp_column_name(self.to_frame(), "__other_column__") + combined = DataFrame( + self._internal.with_new_columns( + [self.rename(self_column_label), other.rename(other_column_label)] + ) + ) # type: DataFrame + else: + if not self.index.equals(other.index): + raise ValueError("Can only compare identically-labeled Series objects") + + combined = combine_frames(self.to_frame(), other.to_frame()) + + this_column_label = "self" + that_column_label = "other" + if keep_equal and keep_shape: + combined.columns = pd.Index([this_column_label, that_column_label]) + return combined + + this_data_scol = combined._internal.data_spark_columns[0] + that_data_scol = combined._internal.data_spark_columns[1] + index_scols = combined._internal.index_spark_columns + sdf = combined._internal.spark_frame + if keep_shape: + this_scol = ( + F.when(this_data_scol == that_data_scol, None) + .otherwise(this_data_scol) + .alias(this_column_label) + ) + that_scol = ( + F.when(this_data_scol == that_data_scol, None) + .otherwise(that_data_scol) + .alias(that_column_label) + ) + else: + sdf = sdf.filter(~this_data_scol.eqNullSafe(that_data_scol)) + this_scol = this_data_scol.alias(this_column_label) + that_scol = that_data_scol.alias(that_column_label) + + sdf = sdf.select(index_scols + [this_scol, that_scol, NATURAL_ORDER_COLUMN_NAME]) + internal = InternalFrame( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, col) for col in self._internal.index_spark_column_names + ], + index_names=self._internal.index_names, + index_dtypes=self._internal.index_dtypes, + column_labels=[(this_column_label,), (that_column_label,)], + data_spark_columns=[scol_for(sdf, this_column_label), scol_for(sdf, that_column_label)], + column_label_names=[None], + ) + return DataFrame(internal) + + def align( + self, + other: Union[DataFrame, "Series"], + join: str = "outer", + axis: Optional[Union[int, str]] = None, + copy: bool = True, + ) -> Tuple["Series", Union[DataFrame, "Series"]]: + """ + Align two objects on their axes with the specified join method. + + Join method is specified for each axis Index. + + Parameters + ---------- + other : DataFrame or Series + join : {{'outer', 'inner', 'left', 'right'}}, default 'outer' + axis : allowed axis of the other object, default None + Align on index (0), columns (1), or both (None). + copy : bool, default True + Always returns new objects. If copy=False and no reindexing is + required then original objects are returned. + + Returns + ------- + (left, right) : (Series, type of other) + Aligned objects. + + Examples + -------- + >>> pp.set_option("compute.ops_on_diff_frames", True) + >>> s1 = pp.Series([7, 8, 9], index=[10, 11, 12]) + >>> s2 = pp.Series(["g", "h", "i"], index=[10, 20, 30]) + + >>> aligned_l, aligned_r = s1.align(s2) + >>> aligned_l.sort_index() + 10 7.0 + 11 8.0 + 12 9.0 + 20 NaN + 30 NaN + dtype: float64 + >>> aligned_r.sort_index() + 10 g + 11 None + 12 None + 20 h + 30 i + dtype: object + + Align with the join type "inner": + + >>> aligned_l, aligned_r = s1.align(s2, join="inner") + >>> aligned_l.sort_index() + 10 7 + dtype: int64 + >>> aligned_r.sort_index() + 10 g + dtype: object + + Align with a DataFrame: + + >>> df = pp.DataFrame({"a": [1, 2, 3], "b": ["a", "b", "c"]}, index=[10, 20, 30]) + >>> aligned_l, aligned_r = s1.align(df) + >>> aligned_l.sort_index() + 10 7.0 + 11 8.0 + 12 9.0 + 20 NaN + 30 NaN + dtype: float64 + >>> aligned_r.sort_index() + a b + 10 1.0 a + 11 NaN None + 12 NaN None + 20 2.0 b + 30 3.0 c + + >>> pp.reset_option("compute.ops_on_diff_frames") + """ + axis = validate_axis(axis) + if axis == 1: + raise ValueError("Series does not support columns axis.") + + self_df = self.to_frame() + left, right = self_df.align(other, join=join, axis=axis, copy=False) + + if left is self_df: + left_ser = self + else: + left_ser = first_series(left).rename(self.name) + + return (left_ser.copy(), right.copy()) if copy else (left_ser, right) + + def between_time( + self, + start_time: Union[datetime.time, str], + end_time: Union[datetime.time, str], + include_start: bool = True, + include_end: bool = True, + axis: Union[int, str] = 0, + ) -> "Series": + """ + Select values between particular times of the day (e.g., 9:00-9:30 AM). + + By setting ``start_time`` to be later than ``end_time``, + you can get the times that are *not* between the two times. + + Parameters + ---------- + start_time : datetime.time or str + Initial time as a time filter limit. + end_time : datetime.time or str + End time as a time filter limit. + include_start : bool, default True + Whether the start time needs to be included in the result. + include_end : bool, default True + Whether the end time needs to be included in the result. + axis : {0 or 'index', 1 or 'columns'}, default 0 + Determine range time on index or columns value. + + Returns + ------- + Series + Data from the original object filtered to the specified dates range. + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + See Also + -------- + at_time : Select values at a particular time of the day. + last : Select final periods of time series based on a date offset. + DatetimeIndex.indexer_between_time : Get just the index locations for + values between particular times of the day. + + Examples + -------- + >>> idx = pd.date_range('2018-04-09', periods=4, freq='1D20min') + >>> kser = pp.Series([1, 2, 3, 4], index=idx) + >>> kser + 2018-04-09 00:00:00 1 + 2018-04-10 00:20:00 2 + 2018-04-11 00:40:00 3 + 2018-04-12 01:00:00 4 + dtype: int64 + + >>> kser.between_time('0:15', '0:45') + 2018-04-10 00:20:00 2 + 2018-04-11 00:40:00 3 + dtype: int64 + """ + return first_series( + self.to_frame().between_time(start_time, end_time, include_start, include_end, axis) + ).rename(self.name) + + def at_time( + self, time: Union[datetime.time, str], asof: bool = False, axis: Union[int, str] = 0 + ) -> "Series": + """ + Select values at particular time of day (e.g., 9:30AM). + + Parameters + ---------- + time : datetime.time or str + axis : {0 or 'index', 1 or 'columns'}, default 0 + + Returns + ------- + Series + + Raises + ------ + TypeError + If the index is not a :class:`DatetimeIndex` + + See Also + -------- + between_time : Select values between particular times of the day. + DatetimeIndex.indexer_at_time : Get just the index locations for + values at particular time of the day. + + Examples + -------- + >>> idx = pd.date_range('2018-04-09', periods=4, freq='12H') + >>> kser = pp.Series([1, 2, 3, 4], index=idx) + >>> kser + 2018-04-09 00:00:00 1 + 2018-04-09 12:00:00 2 + 2018-04-10 00:00:00 3 + 2018-04-10 12:00:00 4 + dtype: int64 + + >>> kser.at_time('12:00') + 2018-04-09 12:00:00 2 + 2018-04-10 12:00:00 4 + dtype: int64 + """ + return first_series(self.to_frame().at_time(time, asof, axis)).rename(self.name) + + def _cum(self, func, skipna, part_cols=(), ascending=True): + # This is used to cummin, cummax, cumsum, etc. + + if ascending: + window = ( + Window.orderBy(F.asc(NATURAL_ORDER_COLUMN_NAME)) + .partitionBy(*part_cols) + .rowsBetween(Window.unboundedPreceding, Window.currentRow) + ) + else: + window = ( + Window.orderBy(F.desc(NATURAL_ORDER_COLUMN_NAME)) + .partitionBy(*part_cols) + .rowsBetween(Window.unboundedPreceding, Window.currentRow) + ) + + if skipna: + # There is a behavior difference between pandas and PySpark. In case of cummax, + # + # Input: + # A B + # 0 2.0 1.0 + # 1 5.0 NaN + # 2 1.0 0.0 + # 3 2.0 4.0 + # 4 4.0 9.0 + # + # pandas: + # A B + # 0 2.0 1.0 + # 1 5.0 NaN + # 2 5.0 1.0 + # 3 5.0 4.0 + # 4 5.0 9.0 + # + # PySpark: + # A B + # 0 2.0 1.0 + # 1 5.0 1.0 + # 2 5.0 1.0 + # 3 5.0 4.0 + # 4 5.0 9.0 + + scol = F.when( + # Manually sets nulls given the column defined above. + self.spark.column.isNull(), + F.lit(None), + ).otherwise(func(self.spark.column).over(window)) + else: + # Here, we use two Windows. + # One for real data. + # The other one for setting nulls after the first null it meets. + # + # There is a behavior difference between pandas and PySpark. In case of cummax, + # + # Input: + # A B + # 0 2.0 1.0 + # 1 5.0 NaN + # 2 1.0 0.0 + # 3 2.0 4.0 + # 4 4.0 9.0 + # + # pandas: + # A B + # 0 2.0 1.0 + # 1 5.0 NaN + # 2 5.0 NaN + # 3 5.0 NaN + # 4 5.0 NaN + # + # PySpark: + # A B + # 0 2.0 1.0 + # 1 5.0 1.0 + # 2 5.0 1.0 + # 3 5.0 4.0 + # 4 5.0 9.0 + scol = F.when( + # By going through with max, it sets True after the first time it meets null. + F.max(self.spark.column.isNull()).over(window), + # Manually sets nulls given the column defined above. + F.lit(None), + ).otherwise(func(self.spark.column).over(window)) + + return self._with_new_scol(scol) + + def _cumsum(self, skipna, part_cols=()): + kser = self + if isinstance(kser.spark.data_type, BooleanType): + kser = kser.spark.transform(lambda scol: scol.cast(LongType())) + elif not isinstance(kser.spark.data_type, NumericType): + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(kser.spark.data_type), + kser.spark.data_type.simpleString(), + ) + ) + return kser._cum(F.sum, skipna, part_cols) + + def _cumprod(self, skipna, part_cols=()): + if isinstance(self.spark.data_type, BooleanType): + scol = self._cum( + lambda scol: F.min(F.coalesce(scol, F.lit(True))), skipna, part_cols + ).spark.column.cast(LongType()) + elif isinstance(self.spark.data_type, NumericType): + num_zeros = self._cum( + lambda scol: F.sum(F.when(scol == 0, 1).otherwise(0)), skipna, part_cols + ).spark.column + num_negatives = self._cum( + lambda scol: F.sum(F.when(scol < 0, 1).otherwise(0)), skipna, part_cols + ).spark.column + sign = F.when(num_negatives % 2 == 0, 1).otherwise(-1) + + abs_prod = F.exp( + self._cum(lambda scol: F.sum(F.log(F.abs(scol))), skipna, part_cols).spark.column + ) + + scol = F.when(num_zeros > 0, 0).otherwise(sign * abs_prod) + + if isinstance(self.spark.data_type, IntegralType): + scol = F.round(scol).cast(LongType()) + else: + raise TypeError( + "Could not convert {} ({}) to numeric".format( + spark_type_to_pandas_dtype(self.spark.data_type), + self.spark.data_type.simpleString(), + ) + ) + + return self._with_new_scol(scol) + + # ---------------------------------------------------------------------- + # Accessor Methods + # ---------------------------------------------------------------------- + dt = CachedAccessor("dt", DatetimeMethods) + str = CachedAccessor("str", StringMethods) + cat = CachedAccessor("cat", CategoricalAccessor) + plot = CachedAccessor("plot", KoalasPlotAccessor) + + # ---------------------------------------------------------------------- + + def _apply_series_op(self, op, should_resolve: bool = False): + kser = op(self) + if should_resolve: + internal = kser._internal.resolved_copy + return first_series(DataFrame(internal)) + else: + return kser + + def _reduce_for_stat_function(self, sfun, name, axis=None, numeric_only=None, **kwargs): + """ + Applies sfun to the column and returns a scalar + + Parameters + ---------- + sfun : the stats function to be used for aggregation + name : original pandas API name. + axis : used only for sanity check because series only support index axis. + numeric_only : not used by this implementation, but passed down by stats functions + """ + from inspect import signature + + axis = validate_axis(axis) + if axis == 1: + raise ValueError("Series does not support columns axis.") + num_args = len(signature(sfun).parameters) + spark_column = self.spark.column + spark_type = self.spark.data_type + + if num_args == 1: + # Only pass in the column if sfun accepts only one arg + scol = sfun(spark_column) + else: # must be 2 + assert num_args == 2 + # Pass in both the column and its data type if sfun accepts two args + scol = sfun(spark_column, spark_type) + + min_count = kwargs.get("min_count", 0) + if min_count > 0: + scol = F.when(Frame._count_expr(spark_column, spark_type) >= min_count, scol) + + result = unpack_scalar(self._internal.spark_frame.select(scol)) + return result if result is not None else np.nan + + def __getitem__(self, key): + try: + if (isinstance(key, slice) and any(type(n) == int for n in [key.start, key.stop])) or ( + type(key) == int + and not isinstance(self.index.spark.data_type, (IntegerType, LongType)) + ): + # Seems like pandas Series always uses int as positional search when slicing + # with ints, searches based on index values when the value is int. + return self.iloc[key] + return self.loc[key] + except SparkPandasIndexingError: + raise KeyError( + "Key length ({}) exceeds index depth ({})".format( + len(key), self._internal.index_level + ) + ) + + def __getattr__(self, item: str_type) -> Any: + if item.startswith("__"): + raise AttributeError(item) + if hasattr(MissingPandasLikeSeries, item): + property_or_func = getattr(MissingPandasLikeSeries, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError("'Series' object has no attribute '{}'".format(item)) + + def _to_internal_pandas(self): + """ + Return a pandas Series directly from _internal to avoid overhead of copy. + + This method is for internal use only. + """ + return self._kdf._internal.to_pandas_frame[self.name] + + def __repr__(self): + max_display_count = get_option("display.max_rows") + if max_display_count is None: + return self._to_internal_pandas().to_string(name=self.name, dtype=self.dtype) + + pser = self._kdf._get_or_create_repr_pandas_cache(max_display_count)[self.name] + pser_length = len(pser) + pser = pser.iloc[:max_display_count] + if pser_length > max_display_count: + repr_string = pser.to_string(length=True) + rest, prev_footer = repr_string.rsplit("\n", 1) + match = REPR_PATTERN.search(prev_footer) + if match is not None: + length = match.group("length") + dtype_name = str(self.dtype.name) + if self.name is None: + footer = "\ndtype: {dtype}\nShowing only the first {length}".format( + length=length, dtype=pprint_thing(dtype_name) + ) + else: + footer = ( + "\nName: {name}, dtype: {dtype}" + "\nShowing only the first {length}".format( + length=length, name=self.name, dtype=pprint_thing(dtype_name) + ) + ) + return rest + footer + return pser.to_string(name=self.name, dtype=self.dtype) + + def __dir__(self): + if not isinstance(self.spark.data_type, StructType): + fields = [] + else: + fields = [f for f in self.spark.data_type.fieldNames() if " " not in f] + return super().__dir__() + fields + + def __iter__(self): + return MissingPandasLikeSeries.__iter__(self) + + if sys.version_info >= (3, 7): + # In order to support the type hints such as Series[...]. See DataFrame.__class_getitem__. + def __class_getitem__(cls, params): + return _create_type_for_series_type(params) + + elif (3, 5) <= sys.version_info < (3, 7): + # The implementation is in its metaclass so this flag is needed to distinguish + # Koalas Series. + is_series = None + + +def unpack_scalar(sdf): + """ + Takes a dataframe that is supposed to contain a single row with a single scalar value, + and returns this value. + """ + l = sdf.limit(2).toPandas() + assert len(l) == 1, (sdf, l) + row = l.iloc[0] + l2 = list(row) + assert len(l2) == 1, (row, l2) + return l2[0] + + +def first_series(df) -> Union["Series", pd.Series]: + """ + Takes a DataFrame and returns the first column of the DataFrame as a Series + """ + assert isinstance(df, (DataFrame, pd.DataFrame)), type(df) + if isinstance(df, DataFrame): + return df._kser_for(df._internal.column_labels[0]) + else: + return df[df.columns[0]] diff --git a/python/pyspark/pandas/spark/__init__.py b/python/pyspark/pandas/spark/__init__.py new file mode 100644 index 0000000000..cce3acad34 --- /dev/null +++ b/python/pyspark/pandas/spark/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/python/pyspark/pandas/spark/accessors.py b/python/pyspark/pandas/spark/accessors.py new file mode 100644 index 0000000000..543e3f4a75 --- /dev/null +++ b/python/pyspark/pandas/spark/accessors.py @@ -0,0 +1,1249 @@ +# +# 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. +# + +""" +Spark related features. Usually, the features here are missing in pandas +but Spark has it. +""" +from abc import ABCMeta, abstractmethod +from distutils.version import LooseVersion +from typing import TYPE_CHECKING, Optional, Union, List, cast + +import pyspark +from pyspark import StorageLevel +from pyspark.sql import Column, DataFrame as SparkDataFrame +from pyspark.sql.types import DataType, StructType + +if TYPE_CHECKING: + import pyspark.pandas as pp # noqa: F401 (SPARK-34943) + from pyspark.pandas.base import IndexOpsMixin # noqa: F401 (SPARK-34943) + from pyspark.pandas.frame import CachedDataFrame # noqa: F401 (SPARK-34943) + + +class SparkIndexOpsMethods(object, metaclass=ABCMeta): + """Spark related features. Usually, the features here are missing in pandas + but Spark has it.""" + + def __init__(self, data: Union["IndexOpsMixin"]): + self._data = data + + @property + def data_type(self) -> DataType: + """ Returns the data type as defined by Spark, as a Spark DataType object.""" + return self._data._internal.spark_type_for(self._data._column_label) + + @property + def nullable(self) -> bool: + """ Returns the nullability as defined by Spark. """ + return self._data._internal.spark_column_nullable_for(self._data._column_label) + + @property + def column(self) -> Column: + """ + Spark Column object representing the Series/Index. + + .. note:: This Spark Column object is strictly stick to its base DataFrame the Series/Index + was derived from. + """ + return self._data._internal.spark_column_for(self._data._column_label) + + def transform(self, func) -> Union["pp.Series", "pp.Index"]: + """ + Applies a function that takes and returns a Spark column. It allows to natively + apply a Spark function and column APIs with the Spark column internally used + in Series or Index. The output length of the Spark column should be same as input's. + + .. note:: It requires to have the same input and output length; therefore, + the aggregate Spark functions such as count does not work. + + Parameters + ---------- + func : function + Function to use for transforming the data by using Spark columns. + + Returns + ------- + Series or Index + + Raises + ------ + ValueError : If the output from the function is not a Spark column. + + Examples + -------- + >>> from pyspark.sql.functions import log + >>> df = pp.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}, columns=["a", "b"]) + >>> df + a b + 0 1 4 + 1 2 5 + 2 3 6 + + >>> df.a.spark.transform(lambda c: log(c)) + 0 0.000000 + 1 0.693147 + 2 1.098612 + Name: a, dtype: float64 + + >>> df.index.spark.transform(lambda c: c + 10) + Int64Index([10, 11, 12], dtype='int64') + + >>> df.a.spark.transform(lambda c: c + df.b.spark.column) + 0 5 + 1 7 + 2 9 + Name: a, dtype: int64 + """ + from pyspark.pandas import MultiIndex + + if isinstance(self._data, MultiIndex): + raise NotImplementedError("MultiIndex does not support spark.transform yet.") + output = func(self._data.spark.column) + if not isinstance(output, Column): + raise ValueError( + "The output of the function [%s] should be of a " + "pyspark.sql.Column; however, got [%s]." % (func, type(output)) + ) + new_ser = self._data._with_new_scol(scol=output) + # Trigger the resolution so it throws an exception if anything does wrong + # within the function, for example, + # `df1.a.spark.transform(lambda _: F.col("non-existent"))`. + new_ser._internal.to_internal_spark_frame + return new_ser + + @property + @abstractmethod + def analyzed(self) -> Union["pp.Series", "pp.Index"]: + pass + + +class SparkSeriesMethods(SparkIndexOpsMethods): + def transform(self, func) -> "pp.Series": + return cast("pp.Series", super().transform(func)) + + transform.__doc__ = SparkIndexOpsMethods.transform.__doc__ + + def apply(self, func) -> "pp.Series": + """ + Applies a function that takes and returns a Spark column. It allows to natively + apply a Spark function and column APIs with the Spark column internally used + in Series or Index. + + .. note:: It forces to lose the index and end up with using default index. It is + preferred to use :meth:`Series.spark.transform` or `:meth:`DataFrame.spark.apply` + with specifying the `inedx_col`. + + .. note:: It does not require to have the same length of the input and output. + However, it requires to create a new DataFrame internally which will require + to set `compute.ops_on_diff_frames` to compute even with the same origin + DataFrame that is expensive, whereas :meth:`Series.spark.transform` does not + require it. + + Parameters + ---------- + func : function + Function to apply the function against the data by using Spark columns. + + Returns + ------- + Series + + Raises + ------ + ValueError : If the output from the function is not a Spark column. + + Examples + -------- + >>> from pyspark import pandas as pp + >>> from pyspark.sql.functions import count, lit + >>> df = pp.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}, columns=["a", "b"]) + >>> df + a b + 0 1 4 + 1 2 5 + 2 3 6 + + >>> df.a.spark.apply(lambda c: count(c)) + 0 3 + Name: a, dtype: int64 + + >>> df.a.spark.apply(lambda c: c + df.b.spark.column) + 0 5 + 1 7 + 2 9 + Name: a, dtype: int64 + """ + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.series import Series, first_series + from pyspark.pandas.internal import HIDDEN_COLUMNS + + output = func(self._data.spark.column) + if not isinstance(output, Column): + raise ValueError( + "The output of the function [%s] should be of a " + "pyspark.sql.Column; however, got [%s]." % (func, type(output)) + ) + assert isinstance(self._data, Series) + + sdf = self._data._internal.spark_frame.drop(*HIDDEN_COLUMNS).select(output) + # Lose index. + return first_series(DataFrame(sdf)).rename(self._data.name) + + @property + def analyzed(self) -> "pp.Series": + """ + Returns a new Series with the analyzed Spark DataFrame. + + After multiple operations, the underlying Spark plan could grow huge + and make the Spark planner take a long time to finish the planning. + + This function is for the workaround to avoid it. + + .. note:: After analyzed, operations between the analyzed Series and the original one + will **NOT** work without setting a config `compute.ops_on_diff_frames` to `True`. + + Returns + ------- + Series + + Examples + -------- + >>> ser = pp.Series([1, 2, 3]) + >>> ser + 0 1 + 1 2 + 2 3 + dtype: int64 + + The analyzed one should return the same value. + + >>> ser.spark.analyzed + 0 1 + 1 2 + 2 3 + dtype: int64 + + However, it won't work with the same anchor Series. + + >>> ser + ser.spark.analyzed + Traceback (most recent call last): + ... + ValueError: ... enable 'compute.ops_on_diff_frames' option. + + >>> with pp.option_context('compute.ops_on_diff_frames', True): + ... (ser + ser.spark.analyzed).sort_index() + 0 2 + 1 4 + 2 6 + dtype: int64 + """ + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.series import first_series + + return first_series(DataFrame(self._data._internal.resolved_copy)) + + +class SparkIndexMethods(SparkIndexOpsMethods): + def transform(self, func) -> "pp.Index": + return cast("pp.Index", super().transform(func)) + + transform.__doc__ = SparkIndexOpsMethods.transform.__doc__ + + @property + def analyzed(self) -> "pp.Index": + """ + Returns a new Index with the analyzed Spark DataFrame. + + After multiple operations, the underlying Spark plan could grow huge + and make the Spark planner take a long time to finish the planning. + + This function is for the workaround to avoid it. + + .. note:: After analyzed, operations between the analyzed Series and the original one + will **NOT** work without setting a config `compute.ops_on_diff_frames` to `True`. + + Returns + ------- + Index + + Examples + -------- + >>> idx = pp.Index([1, 2, 3]) + >>> idx + Int64Index([1, 2, 3], dtype='int64') + + The analyzed one should return the same value. + + >>> idx.spark.analyzed + Int64Index([1, 2, 3], dtype='int64') + + However, it won't work with the same anchor Index. + + >>> idx + idx.spark.analyzed + Traceback (most recent call last): + ... + ValueError: ... enable 'compute.ops_on_diff_frames' option. + + >>> with pp.option_context('compute.ops_on_diff_frames', True): + ... (idx + idx.spark.analyzed).sort_values() + Int64Index([2, 4, 6], dtype='int64') + """ + from pyspark.pandas.frame import DataFrame + + return DataFrame(self._data._internal.resolved_copy).index + + +class SparkFrameMethods(object): + """Spark related features. Usually, the features here are missing in pandas + but Spark has it.""" + + def __init__(self, frame: "pp.DataFrame"): + self._kdf = frame + + def schema(self, index_col: Optional[Union[str, List[str]]] = None) -> StructType: + """ + Returns the underlying Spark schema. + + Returns + ------- + pyspark.sql.types.StructType + The underlying Spark schema. + + Parameters + ---------- + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + + Examples + -------- + >>> df = pp.DataFrame({'a': list('abc'), + ... 'b': list(range(1, 4)), + ... 'c': np.arange(3, 6).astype('i1'), + ... 'd': np.arange(4.0, 7.0, dtype='float64'), + ... 'e': [True, False, True], + ... 'f': pd.date_range('20130101', periods=3)}, + ... columns=['a', 'b', 'c', 'd', 'e', 'f']) + >>> df.spark.schema().simpleString() + 'struct' + >>> df.spark.schema(index_col='index').simpleString() + 'struct' + """ + return self.frame(index_col).schema + + def print_schema(self, index_col: Optional[Union[str, List[str]]] = None) -> None: + """ + Prints out the underlying Spark schema in the tree format. + + Parameters + ---------- + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + + Returns + ------- + None + + Examples + -------- + >>> df = pp.DataFrame({'a': list('abc'), + ... 'b': list(range(1, 4)), + ... 'c': np.arange(3, 6).astype('i1'), + ... 'd': np.arange(4.0, 7.0, dtype='float64'), + ... 'e': [True, False, True], + ... 'f': pd.date_range('20130101', periods=3)}, + ... columns=['a', 'b', 'c', 'd', 'e', 'f']) + >>> df.spark.print_schema() # doctest: +NORMALIZE_WHITESPACE + root + |-- a: string (nullable = false) + |-- b: long (nullable = false) + |-- c: byte (nullable = false) + |-- d: double (nullable = false) + |-- e: boolean (nullable = false) + |-- f: timestamp (nullable = false) + >>> df.spark.print_schema(index_col='index') # doctest: +NORMALIZE_WHITESPACE + root + |-- index: long (nullable = false) + |-- a: string (nullable = false) + |-- b: long (nullable = false) + |-- c: byte (nullable = false) + |-- d: double (nullable = false) + |-- e: boolean (nullable = false) + |-- f: timestamp (nullable = false) + """ + self.frame(index_col).printSchema() + + def frame(self, index_col: Optional[Union[str, List[str]]] = None) -> SparkDataFrame: + """ + Return the current DataFrame as a Spark DataFrame. :meth:`DataFrame.spark.frame` is an + alias of :meth:`DataFrame.to_spark`. + + Parameters + ---------- + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + + See Also + -------- + DataFrame.to_spark + DataFrame.to_koalas + DataFrame.spark.frame + + Examples + -------- + By default, this method loses the index as below. + + >>> df = pp.DataFrame({'a': [1, 2, 3], 'b': [4, 5, 6], 'c': [7, 8, 9]}) + >>> df.to_spark().show() # doctest: +NORMALIZE_WHITESPACE + +---+---+---+ + | a| b| c| + +---+---+---+ + | 1| 4| 7| + | 2| 5| 8| + | 3| 6| 9| + +---+---+---+ + + >>> df = pp.DataFrame({'a': [1, 2, 3], 'b': [4, 5, 6], 'c': [7, 8, 9]}) + >>> df.spark.frame().show() # doctest: +NORMALIZE_WHITESPACE + +---+---+---+ + | a| b| c| + +---+---+---+ + | 1| 4| 7| + | 2| 5| 8| + | 3| 6| 9| + +---+---+---+ + + If `index_col` is set, it keeps the index column as specified. + + >>> df.to_spark(index_col="index").show() # doctest: +NORMALIZE_WHITESPACE + +-----+---+---+---+ + |index| a| b| c| + +-----+---+---+---+ + | 0| 1| 4| 7| + | 1| 2| 5| 8| + | 2| 3| 6| 9| + +-----+---+---+---+ + + Keeping index column is useful when you want to call some Spark APIs and + convert it back to Koalas DataFrame without creating a default index, which + can affect performance. + + >>> spark_df = df.to_spark(index_col="index") + >>> spark_df = spark_df.filter("a == 2") + >>> spark_df.to_koalas(index_col="index") # doctest: +NORMALIZE_WHITESPACE + a b c + index + 1 2 5 8 + + In case of multi-index, specify a list to `index_col`. + + >>> new_df = df.set_index("a", append=True) + >>> new_spark_df = new_df.to_spark(index_col=["index_1", "index_2"]) + >>> new_spark_df.show() # doctest: +NORMALIZE_WHITESPACE + +-------+-------+---+---+ + |index_1|index_2| b| c| + +-------+-------+---+---+ + | 0| 1| 4| 7| + | 1| 2| 5| 8| + | 2| 3| 6| 9| + +-------+-------+---+---+ + + Likewise, can be converted to back to Koalas DataFrame. + + >>> new_spark_df.to_koalas( + ... index_col=["index_1", "index_2"]) # doctest: +NORMALIZE_WHITESPACE + b c + index_1 index_2 + 0 1 4 7 + 1 2 5 8 + 2 3 6 9 + """ + from pyspark.pandas.utils import name_like_string + + kdf = self._kdf + + data_column_names = [] + data_columns = [] + for i, (label, spark_column, column_name) in enumerate( + zip( + kdf._internal.column_labels, + kdf._internal.data_spark_columns, + kdf._internal.data_spark_column_names, + ) + ): + name = str(i) if label is None else name_like_string(label) + data_column_names.append(name) + if column_name != name: + spark_column = spark_column.alias(name) + data_columns.append(spark_column) + + if index_col is None: + return kdf._internal.spark_frame.select(data_columns) + else: + if isinstance(index_col, str): + index_col = [index_col] + + old_index_scols = kdf._internal.index_spark_columns + + if len(index_col) != len(old_index_scols): + raise ValueError( + "length of index columns is %s; however, the length of the given " + "'index_col' is %s." % (len(old_index_scols), len(index_col)) + ) + + if any(col in data_column_names for col in index_col): + raise ValueError("'index_col' cannot be overlapped with other columns.") + + new_index_scols = [ + index_scol.alias(col) for index_scol, col in zip(old_index_scols, index_col) + ] + return kdf._internal.spark_frame.select(new_index_scols + data_columns) + + def cache(self) -> "CachedDataFrame": + """ + Yields and caches the current DataFrame. + + The Koalas DataFrame is yielded as a protected resource and its corresponding + data is cached which gets uncached after execution goes of the context. + + If you want to specify the StorageLevel manually, use :meth:`DataFrame.spark.persist` + + See Also + -------- + DataFrame.spark.persist + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats']) + >>> df + dogs cats + 0 0.2 0.3 + 1 0.0 0.6 + 2 0.6 0.0 + 3 0.2 0.1 + + >>> with df.spark.cache() as cached_df: + ... print(cached_df.count()) + ... + dogs 4 + cats 4 + dtype: int64 + + >>> df = df.spark.cache() + >>> df.to_pandas().mean(axis=1) + 0 0.25 + 1 0.30 + 2 0.30 + 3 0.15 + dtype: float64 + + To uncache the dataframe, use `unpersist` function + + >>> df.spark.unpersist() + """ + from pyspark.pandas.frame import CachedDataFrame + + self._kdf._update_internal_frame( + self._kdf._internal.resolved_copy, requires_same_anchor=False + ) + return CachedDataFrame(self._kdf._internal) + + def persist( + self, storage_level: StorageLevel = StorageLevel.MEMORY_AND_DISK + ) -> "CachedDataFrame": + """ + Yields and caches the current DataFrame with a specific StorageLevel. + If a StogeLevel is not given, the `MEMORY_AND_DISK` level is used by default like PySpark. + + The Koalas DataFrame is yielded as a protected resource and its corresponding + data is cached which gets uncached after execution goes of the context. + + See Also + -------- + DataFrame.spark.cache + + Examples + -------- + >>> import pyspark + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats']) + >>> df + dogs cats + 0 0.2 0.3 + 1 0.0 0.6 + 2 0.6 0.0 + 3 0.2 0.1 + + Set the StorageLevel to `MEMORY_ONLY`. + + >>> with df.spark.persist(pyspark.StorageLevel.MEMORY_ONLY) as cached_df: + ... print(cached_df.spark.storage_level) + ... print(cached_df.count()) + ... + Memory Serialized 1x Replicated + dogs 4 + cats 4 + dtype: int64 + + Set the StorageLevel to `DISK_ONLY`. + + >>> with df.spark.persist(pyspark.StorageLevel.DISK_ONLY) as cached_df: + ... print(cached_df.spark.storage_level) + ... print(cached_df.count()) + ... + Disk Serialized 1x Replicated + dogs 4 + cats 4 + dtype: int64 + + If a StorageLevel is not given, it uses `MEMORY_AND_DISK` by default. + + >>> with df.spark.persist() as cached_df: + ... print(cached_df.spark.storage_level) + ... print(cached_df.count()) + ... + Disk Memory Serialized 1x Replicated + dogs 4 + cats 4 + dtype: int64 + + >>> df = df.spark.persist() + >>> df.to_pandas().mean(axis=1) + 0 0.25 + 1 0.30 + 2 0.30 + 3 0.15 + dtype: float64 + + To uncache the dataframe, use `unpersist` function + + >>> df.spark.unpersist() + """ + from pyspark.pandas.frame import CachedDataFrame + + self._kdf._update_internal_frame( + self._kdf._internal.resolved_copy, requires_same_anchor=False + ) + return CachedDataFrame(self._kdf._internal, storage_level=storage_level) + + def hint(self, name: str, *parameters) -> "pp.DataFrame": + """ + Specifies some hint on the current DataFrame. + + Parameters + ---------- + name : A name of the hint. + parameters : Optional parameters. + + Returns + ------- + ret : DataFrame with the hint. + + See Also + -------- + broadcast : Marks a DataFrame as small enough for use in broadcast joins. + + Examples + -------- + >>> df1 = pp.DataFrame({'lkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [1, 2, 3, 5]}, + ... columns=['lkey', 'value']).set_index('lkey') + >>> df2 = pp.DataFrame({'rkey': ['foo', 'bar', 'baz', 'foo'], + ... 'value': [5, 6, 7, 8]}, + ... columns=['rkey', 'value']).set_index('rkey') + >>> merged = df1.merge(df2.spark.hint("broadcast"), left_index=True, right_index=True) + >>> merged.spark.explain() # doctest: +ELLIPSIS + == Physical Plan == + ... + ...BroadcastHashJoin... + ... + """ + from pyspark.pandas.frame import DataFrame + + internal = self._kdf._internal.resolved_copy + return DataFrame(internal.with_new_sdf(internal.spark_frame.hint(name, *parameters))) + + def to_table( + self, + name: str, + format: Optional[str] = None, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> None: + """ + Write the DataFrame into a Spark table. :meth:`DataFrame.spark.to_table` + is an alias of :meth:`DataFrame.to_table`. + + Parameters + ---------- + name : str, required + Table name in Spark. + format : string, optional + Specifies the output data source format. Some common ones are: + + - 'delta' + - 'parquet' + - 'orc' + - 'json' + - 'csv' + + mode : str {'append', 'overwrite', 'ignore', 'error', 'errorifexists'}, default + 'overwrite'. Specifies the behavior of the save operation when the table exists + already. + + - 'append': Append the new data to existing data. + - 'overwrite': Overwrite existing data. + - 'ignore': Silently ignore this operation if data already exists. + - 'error' or 'errorifexists': Throw an exception if data already exists. + + partition_cols : str or list of str, optional, default None + Names of partitioning columns + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + options + Additional options passed directly to Spark. + + Returns + ------- + None + + See Also + -------- + read_table + DataFrame.to_spark_io + DataFrame.spark.to_spark_io + DataFrame.to_parquet + + Examples + -------- + >>> df = pp.DataFrame(dict( + ... date=list(pd.date_range('2012-1-1 12:00:00', periods=3, freq='M')), + ... country=['KR', 'US', 'JP'], + ... code=[1, 2 ,3]), columns=['date', 'country', 'code']) + >>> df + date country code + 0 2012-01-31 12:00:00 KR 1 + 1 2012-02-29 12:00:00 US 2 + 2 2012-03-31 12:00:00 JP 3 + + >>> df.to_table('%s.my_table' % db, partition_cols='date') + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + self._kdf.spark.frame(index_col=index_col).write.saveAsTable( + name=name, format=format, mode=mode, partitionBy=partition_cols, **options + ) + + def to_spark_io( + self, + path: Optional[str] = None, + format: Optional[str] = None, + mode: str = "overwrite", + partition_cols: Optional[Union[str, List[str]]] = None, + index_col: Optional[Union[str, List[str]]] = None, + **options + ) -> None: + """Write the DataFrame out to a Spark data source. :meth:`DataFrame.spark.to_spark_io` + is an alias of :meth:`DataFrame.to_spark_io`. + + Parameters + ---------- + path : string, optional + Path to the data source. + format : string, optional + Specifies the output data source format. Some common ones are: + + - 'delta' + - 'parquet' + - 'orc' + - 'json' + - 'csv' + mode : str {'append', 'overwrite', 'ignore', 'error', 'errorifexists'}, default + 'overwrite'. Specifies the behavior of the save operation when data already. + + - 'append': Append the new data to existing data. + - 'overwrite': Overwrite existing data. + - 'ignore': Silently ignore this operation if data already exists. + - 'error' or 'errorifexists': Throw an exception if data already exists. + partition_cols : str or list of str, optional + Names of partitioning columns + index_col: str or list of str, optional, default: None + Column names to be used in Spark to represent Koalas' index. The index name + in Koalas is ignored. By default, the index is always lost. + options : dict + All other options passed directly into Spark's data source. + + Returns + ------- + None + + See Also + -------- + read_spark_io + DataFrame.to_delta + DataFrame.to_parquet + DataFrame.to_table + DataFrame.to_spark_io + DataFrame.spark.to_spark_io + + Examples + -------- + >>> df = pp.DataFrame(dict( + ... date=list(pd.date_range('2012-1-1 12:00:00', periods=3, freq='M')), + ... country=['KR', 'US', 'JP'], + ... code=[1, 2 ,3]), columns=['date', 'country', 'code']) + >>> df + date country code + 0 2012-01-31 12:00:00 KR 1 + 1 2012-02-29 12:00:00 US 2 + 2 2012-03-31 12:00:00 JP 3 + + >>> df.to_spark_io(path='%s/to_spark_io/foo.json' % path, format='json') + """ + if "options" in options and isinstance(options.get("options"), dict) and len(options) == 1: + options = options.get("options") # type: ignore + + self._kdf.spark.frame(index_col=index_col).write.save( + path=path, format=format, mode=mode, partitionBy=partition_cols, **options + ) + + def explain(self, extended: Optional[bool] = None, mode: Optional[str] = None) -> None: + """ + Prints the underlying (logical and physical) Spark plans to the console for debugging + purpose. + + Parameters + ---------- + extended : boolean, default ``False``. + If ``False``, prints only the physical plan. + mode : string, default ``None``. + The expected output format of plans. + + Returns + ------- + None + + Examples + -------- + >>> df = pp.DataFrame({'id': range(10)}) + >>> df.spark.explain() # doctest: +ELLIPSIS + == Physical Plan == + ... + + >>> df.spark.explain(True) # doctest: +ELLIPSIS + == Parsed Logical Plan == + ... + == Analyzed Logical Plan == + ... + == Optimized Logical Plan == + ... + == Physical Plan == + ... + + >>> df.spark.explain("extended") # doctest: +ELLIPSIS + == Parsed Logical Plan == + ... + == Analyzed Logical Plan == + ... + == Optimized Logical Plan == + ... + == Physical Plan == + ... + + >>> df.spark.explain(mode="extended") # doctest: +ELLIPSIS + == Parsed Logical Plan == + ... + == Analyzed Logical Plan == + ... + == Optimized Logical Plan == + ... + == Physical Plan == + ... + """ + if LooseVersion(pyspark.__version__) < LooseVersion("3.0"): + if mode is not None and extended is not None: + raise Exception("extended and mode should not be set together.") + + if extended is not None and isinstance(extended, str): + mode = extended + + if mode is not None: + if mode == "simple": + extended = False + elif mode == "extended": + extended = True + else: + raise ValueError( + "Unknown spark.explain mode: {}. Accepted spark.explain modes are " + "'simple', 'extended'.".format(mode) + ) + if extended is None: + extended = False + self._kdf._internal.to_internal_spark_frame.explain(extended) + else: + self._kdf._internal.to_internal_spark_frame.explain(extended, mode) + + def apply(self, func, index_col: Optional[Union[str, List[str]]] = None) -> "pp.DataFrame": + """ + Applies a function that takes and returns a Spark DataFrame. It allows natively + apply a Spark function and column APIs with the Spark column internally used + in Series or Index. + + .. note:: set `index_col` and keep the column named as so in the output Spark + DataFrame to avoid using the default index to prevent performance penalty. + If you omit `index_col`, it will use default index which is potentially + expensive in general. + + .. note:: it will lose column labels. This is a synonym of + ``func(kdf.to_spark(index_col)).to_koalas(index_col)``. + + Parameters + ---------- + func : function + Function to apply the function against the data by using Spark DataFrame. + + Returns + ------- + DataFrame + + Raises + ------ + ValueError : If the output from the function is not a Spark DataFrame. + + Examples + -------- + >>> kdf = pp.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}, columns=["a", "b"]) + >>> kdf + a b + 0 1 4 + 1 2 5 + 2 3 6 + + >>> kdf.spark.apply( + ... lambda sdf: sdf.selectExpr("a + b as c", "index"), index_col="index") + ... # doctest: +NORMALIZE_WHITESPACE + c + index + 0 5 + 1 7 + 2 9 + + The case below ends up with using the default index, which should be avoided + if possible. + + >>> kdf.spark.apply(lambda sdf: sdf.groupby("a").count().sort("a")) + a count + 0 1 1 + 1 2 1 + 2 3 1 + """ + output = func(self.frame(index_col)) + if not isinstance(output, SparkDataFrame): + raise ValueError( + "The output of the function [%s] should be of a " + "pyspark.sql.DataFrame; however, got [%s]." % (func, type(output)) + ) + return output.to_koalas(index_col) + + def repartition(self, num_partitions: int) -> "pp.DataFrame": + """ + Returns a new DataFrame partitioned by the given partitioning expressions. The + resulting DataFrame is hash partitioned. + + Parameters + ---------- + num_partitions : int + The target number of partitions. + + Returns + ------- + DataFrame + + Examples + -------- + >>> kdf = pp.DataFrame({"age": [5, 5, 2, 2], + ... "name": ["Bob", "Bob", "Alice", "Alice"]}).set_index("age") + >>> kdf.sort_index() # doctest: +NORMALIZE_WHITESPACE + name + age + 2 Alice + 2 Alice + 5 Bob + 5 Bob + >>> new_kdf = kdf.spark.repartition(7) + >>> new_kdf.to_spark().rdd.getNumPartitions() + 7 + >>> new_kdf.sort_index() # doctest: +NORMALIZE_WHITESPACE + name + age + 2 Alice + 2 Alice + 5 Bob + 5 Bob + """ + from pyspark.pandas.frame import DataFrame + + internal = self._kdf._internal.resolved_copy + repartitioned_sdf = internal.spark_frame.repartition(num_partitions) + return DataFrame(internal.with_new_sdf(repartitioned_sdf)) + + def coalesce(self, num_partitions: int) -> "pp.DataFrame": + """ + Returns a new DataFrame that has exactly `num_partitions` partitions. + + .. note:: This operation results in a narrow dependency, e.g. if you go from 1000 + partitions to 100 partitions, there will not be a shuffle, instead each of the 100 new + partitions will claim 10 of the current partitions. If a larger number of partitions is + requested, it will stay at the current number of partitions. However, if you're doing a + drastic coalesce, e.g. to num_partitions = 1, this may result in your computation taking + place on fewer nodes than you like (e.g. one node in the case of num_partitions = 1). To + avoid this, you can call repartition(). This will add a shuffle step, but means the + current upstream partitions will be executed in parallel (per whatever the current + partitioning is). + + Parameters + ---------- + num_partitions : int + The target number of partitions. + + Returns + ------- + DataFrame + + Examples + -------- + >>> kdf = pp.DataFrame({"age": [5, 5, 2, 2], + ... "name": ["Bob", "Bob", "Alice", "Alice"]}).set_index("age") + >>> kdf.sort_index() # doctest: +NORMALIZE_WHITESPACE + name + age + 2 Alice + 2 Alice + 5 Bob + 5 Bob + >>> new_kdf = kdf.spark.coalesce(1) + >>> new_kdf.to_spark().rdd.getNumPartitions() + 1 + >>> new_kdf.sort_index() # doctest: +NORMALIZE_WHITESPACE + name + age + 2 Alice + 2 Alice + 5 Bob + 5 Bob + """ + from pyspark.pandas.frame import DataFrame + + internal = self._kdf._internal.resolved_copy + coalesced_sdf = internal.spark_frame.coalesce(num_partitions) + return DataFrame(internal.with_new_sdf(coalesced_sdf)) + + def checkpoint(self, eager: bool = True) -> "pp.DataFrame": + """Returns a checkpointed version of this DataFrame. + + Checkpointing can be used to truncate the logical plan of this DataFrame, which is + especially useful in iterative algorithms where the plan may grow exponentially. It will be + saved to files inside the checkpoint directory set with `SparkContext.setCheckpointDir`. + + Parameters + ---------- + eager : bool + Whether to checkpoint this DataFrame immediately + + Returns + ------- + DataFrame + + Examples + -------- + >>> kdf = pp.DataFrame({"a": ["a", "b", "c"]}) + >>> kdf + a + 0 a + 1 b + 2 c + >>> new_kdf = kdf.spark.checkpoint() # doctest: +SKIP + >>> new_kdf # doctest: +SKIP + a + 0 a + 1 b + 2 c + """ + from pyspark.pandas.frame import DataFrame + + internal = self._kdf._internal.resolved_copy + checkpointed_sdf = internal.spark_frame.checkpoint(eager) + return DataFrame(internal.with_new_sdf(checkpointed_sdf)) + + def local_checkpoint(self, eager: bool = True) -> "pp.DataFrame": + """Returns a locally checkpointed version of this DataFrame. + + Checkpointing can be used to truncate the logical plan of this DataFrame, which is + especially useful in iterative algorithms where the plan may grow exponentially. Local + checkpoints are stored in the executors using the caching subsystem and therefore they are + not reliable. + + Parameters + ---------- + eager : bool + Whether to locally checkpoint this DataFrame immediately + + Returns + ------- + DataFrame + + Examples + -------- + >>> kdf = pp.DataFrame({"a": ["a", "b", "c"]}) + >>> kdf + a + 0 a + 1 b + 2 c + >>> new_kdf = kdf.spark.local_checkpoint() + >>> new_kdf + a + 0 a + 1 b + 2 c + """ + from pyspark.pandas.frame import DataFrame + + internal = self._kdf._internal.resolved_copy + checkpointed_sdf = internal.spark_frame.localCheckpoint(eager) + return DataFrame(internal.with_new_sdf(checkpointed_sdf)) + + @property + def analyzed(self) -> "pp.DataFrame": + """ + Returns a new DataFrame with the analyzed Spark DataFrame. + + After multiple operations, the underlying Spark plan could grow huge + and make the Spark planner take a long time to finish the planning. + + This function is for the workaround to avoid it. + + .. note:: After analyzed, operations between the analyzed DataFrame and the original one + will **NOT** work without setting a config `compute.ops_on_diff_frames` to `True`. + + Returns + ------- + DataFrame + + Examples + -------- + >>> df = pp.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]}, columns=["a", "b"]) + >>> df + a b + 0 1 4 + 1 2 5 + 2 3 6 + + The analyzed one should return the same value. + + >>> df.spark.analyzed + a b + 0 1 4 + 1 2 5 + 2 3 6 + + However, it won't work with the same anchor Series. + + >>> df + df.spark.analyzed + Traceback (most recent call last): + ... + ValueError: ... enable 'compute.ops_on_diff_frames' option. + + >>> with pp.option_context('compute.ops_on_diff_frames', True): + ... (df + df.spark.analyzed).sort_index() + a b + 0 2 8 + 1 4 10 + 2 6 12 + """ + from pyspark.pandas.frame import DataFrame + + return DataFrame(self._kdf._internal.resolved_copy) + + +class CachedSparkFrameMethods(SparkFrameMethods): + """Spark related features for cached DataFrame. This is usually created via + `df.spark.cache()`.""" + + def __init__(self, frame: "CachedDataFrame"): + super().__init__(frame) + + @property + def storage_level(self) -> StorageLevel: + """ + Return the storage level of this cache. + + Examples + -------- + >>> import pyspark.pandas as pp + >>> import pyspark + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats']) + >>> df + dogs cats + 0 0.2 0.3 + 1 0.0 0.6 + 2 0.6 0.0 + 3 0.2 0.1 + + >>> with df.spark.cache() as cached_df: + ... print(cached_df.spark.storage_level) + ... + Disk Memory Deserialized 1x Replicated + + Set the StorageLevel to `MEMORY_ONLY`. + + >>> with df.spark.persist(pyspark.StorageLevel.MEMORY_ONLY) as cached_df: + ... print(cached_df.spark.storage_level) + ... + Memory Serialized 1x Replicated + """ + return self._kdf._cached.storageLevel + + def unpersist(self) -> None: + """ + The `unpersist` function is used to uncache the Koalas DataFrame when it + is not used with `with` statement. + + Returns + ------- + None + + Examples + -------- + >>> df = pp.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)], + ... columns=['dogs', 'cats']) + >>> df = df.spark.cache() + + To uncache the dataframe, use `unpersist` function + + >>> df.spark.unpersist() + """ + if self._kdf._cached.is_cached: + self._kdf._cached.unpersist() diff --git a/python/pyspark/pandas/spark/functions.py b/python/pyspark/pandas/spark/functions.py new file mode 100644 index 0000000000..42b2a4c9b5 --- /dev/null +++ b/python/pyspark/pandas/spark/functions.py @@ -0,0 +1,98 @@ +# +# 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. +# +""" +Additional Spark functions used in Koalas. +""" + +from pyspark import SparkContext +from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal + + +__all__ = ["percentile_approx"] + + +def percentile_approx(col, percentage, accuracy=10000): + """ + Returns the approximate percentile value of numeric column col at the given percentage. + The value of percentage must be between 0.0 and 1.0. + + The accuracy parameter (default: 10000) + is a positive numeric literal which controls approximation accuracy at the cost of memory. + Higher value of accuracy yields better accuracy, 1.0/accuracy is the relative error + of the approximation. + + When percentage is an array, each value of the percentage array must be between 0.0 and 1.0. + In this case, returns the approximate percentile array of column col + at the given percentage array. + + Ported from Spark 3.1. + """ + sc = SparkContext._active_spark_context + + if isinstance(percentage, (list, tuple)): + # A local list + percentage = sc._jvm.functions.array( + _to_seq(sc, [_create_column_from_literal(x) for x in percentage]) + ) + elif isinstance(percentage, Column): + # Already a Column + percentage = _to_java_column(percentage) + else: + # Probably scalar + percentage = _create_column_from_literal(percentage) + + accuracy = ( + _to_java_column(accuracy) + if isinstance(accuracy, Column) + else _create_column_from_literal(accuracy) + ) + + return _call_udf(sc, "percentile_approx", _to_java_column(col), percentage, accuracy) + + +def array_repeat(col, count): + """ + Collection function: creates an array containing a column repeated count times. + + Ported from Spark 3.0. + """ + sc = SparkContext._active_spark_context + return Column( + sc._jvm.functions.array_repeat( + _to_java_column(col), _to_java_column(count) if isinstance(count, Column) else count + ) + ) + + +def repeat(col, n): + """ + Repeats a string column n times, and returns it as a new string column. + """ + sc = SparkContext._active_spark_context + n = _to_java_column(n) if isinstance(n, Column) else _create_column_from_literal(n) + return _call_udf(sc, "repeat", _to_java_column(col), n) + + +def _call_udf(sc, name, *cols): + return Column(sc._jvm.functions.callUDF(name, _make_arguments(sc, *cols))) + + +def _make_arguments(sc, *cols): + java_arr = sc._gateway.new_array(sc._jvm.Column, len(cols)) + for i, col in enumerate(cols): + java_arr[i] = col + return java_arr diff --git a/python/pyspark/pandas/spark/utils.py b/python/pyspark/pandas/spark/utils.py new file mode 100644 index 0000000000..46505a97c4 --- /dev/null +++ b/python/pyspark/pandas/spark/utils.py @@ -0,0 +1,124 @@ +# +# 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. +# +""" +Helpers and utilities to deal with PySpark instances +""" +from pyspark.sql.types import DecimalType, StructType, MapType, ArrayType, StructField, DataType + + +def as_nullable_spark_type(dt: DataType) -> DataType: + """ + Returns a nullable schema or data types. + + Examples + -------- + >>> from pyspark.sql.types import * + >>> as_nullable_spark_type(StructType([ + ... StructField("A", IntegerType(), True), + ... StructField("B", FloatType(), False)])) # doctest: +NORMALIZE_WHITESPACE + StructType(List(StructField(A,IntegerType,true),StructField(B,FloatType,true))) + + >>> as_nullable_spark_type(StructType([ + ... StructField("A", + ... StructType([ + ... StructField('a', + ... MapType(IntegerType(), + ... ArrayType(IntegerType(), False), False), False), + ... StructField('b', StringType(), True)])), + ... StructField("B", FloatType(), False)])) # doctest: +NORMALIZE_WHITESPACE + StructType(List(StructField(A,StructType(List(StructField(a,MapType(IntegerType,ArrayType\ +(IntegerType,true),true),true),StructField(b,StringType,true))),true),\ +StructField(B,FloatType,true))) + """ + if isinstance(dt, StructType): + new_fields = [] + for field in dt.fields: + new_fields.append( + StructField( + field.name, + as_nullable_spark_type(field.dataType), + nullable=True, + metadata=field.metadata, + ) + ) + return StructType(new_fields) + elif isinstance(dt, ArrayType): + return ArrayType(as_nullable_spark_type(dt.elementType), containsNull=True) + elif isinstance(dt, MapType): + return MapType( + as_nullable_spark_type(dt.keyType), + as_nullable_spark_type(dt.valueType), + valueContainsNull=True, + ) + else: + return dt + + +def force_decimal_precision_scale(dt: DataType, precision: int = 38, scale: int = 18) -> DataType: + """ + Returns a data type with a fixed decimal type. + + The precision and scale of the decimal type are fixed with the given values. + + Examples + -------- + >>> from pyspark.sql.types import * + >>> force_decimal_precision_scale(StructType([ + ... StructField("A", DecimalType(10, 0), True), + ... StructField("B", DecimalType(14, 7), False)])) # doctest: +NORMALIZE_WHITESPACE + StructType(List(StructField(A,DecimalType(38,18),true),StructField(B,DecimalType(38,18),false))) + + >>> force_decimal_precision_scale(StructType([ + ... StructField("A", + ... StructType([ + ... StructField('a', + ... MapType(DecimalType(5, 0), + ... ArrayType(DecimalType(20, 0), False), False), False), + ... StructField('b', StringType(), True)])), + ... StructField("B", DecimalType(30, 15), False)]), + ... precision=30, scale=15) # doctest: +NORMALIZE_WHITESPACE + StructType(List(StructField(A,StructType(List(StructField(a,MapType(DecimalType(30,15),\ +ArrayType(DecimalType(30,15),false),false),false),StructField(b,StringType,true))),true),\ +StructField(B,DecimalType(30,15),false))) + """ + if isinstance(dt, StructType): + new_fields = [] + for field in dt.fields: + new_fields.append( + StructField( + field.name, + force_decimal_precision_scale(field.dataType, precision, scale), + nullable=field.nullable, + metadata=field.metadata, + ) + ) + return StructType(new_fields) + elif isinstance(dt, ArrayType): + return ArrayType( + force_decimal_precision_scale(dt.elementType, precision, scale), + containsNull=dt.containsNull, + ) + elif isinstance(dt, MapType): + return MapType( + force_decimal_precision_scale(dt.keyType, precision, scale), + force_decimal_precision_scale(dt.valueType, precision, scale), + valueContainsNull=dt.valueContainsNull, + ) + elif isinstance(dt, DecimalType): + return DecimalType(precision=precision, scale=scale) + else: + return dt diff --git a/python/pyspark/pandas/sql.py b/python/pyspark/pandas/sql.py new file mode 100644 index 0000000000..aec2ae9213 --- /dev/null +++ b/python/pyspark/pandas/sql.py @@ -0,0 +1,302 @@ +# +# 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 _string +from typing import Dict, Any, Optional # noqa: F401 (SPARK-34943) +import inspect +import pandas as pd + +from pyspark.sql import SparkSession, DataFrame as SDataFrame # noqa: F401 (SPARK-34943) + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas.utils import default_session +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.series import Series + + +__all__ = ["sql"] + +from builtins import globals as builtin_globals +from builtins import locals as builtin_locals + + +def sql(query: str, globals=None, locals=None, **kwargs) -> DataFrame: + """ + Execute a SQL query and return the result as a Koalas DataFrame. + + This function also supports embedding Python variables (locals, globals, and parameters) + in the SQL statement by wrapping them in curly braces. See examples section for details. + + In addition to the locals, globals and parameters, the function will also attempt + to determine if the program currently runs in an IPython (or Jupyter) environment + and to import the variables from this environment. The variables have the same + precedence as globals. + + The following variable types are supported: + + * string + * int + * float + * list, tuple, range of above types + * Koalas DataFrame + * Koalas Series + * pandas DataFrame + + Parameters + ---------- + query : str + the SQL query + globals : dict, optional + the dictionary of global variables, if explicitly set by the user + locals : dict, optional + the dictionary of local variables, if explicitly set by the user + kwargs + other variables that the user may want to set manually that can be referenced in the query + + Returns + ------- + Koalas DataFrame + + Examples + -------- + + Calling a built-in SQL function. + + >>> pp.sql("select * from range(10) where id > 7") + id + 0 8 + 1 9 + + A query can also reference a local variable or parameter by wrapping them in curly braces: + + >>> bound1 = 7 + >>> pp.sql("select * from range(10) where id > {bound1} and id < {bound2}", bound2=9) + id + 0 8 + + You can also wrap a DataFrame with curly braces to query it directly. Note that when you do + that, the indexes, if any, automatically become top level columns. + + >>> mydf = pp.range(10) + >>> x = range(4) + >>> pp.sql("SELECT * from {mydf} WHERE id IN {x}") + id + 0 0 + 1 1 + 2 2 + 3 3 + + Queries can also be arbitrarily nested in functions: + + >>> def statement(): + ... mydf2 = pp.DataFrame({"x": range(2)}) + ... return pp.sql("SELECT * from {mydf2}") + >>> statement() + x + 0 0 + 1 1 + + Mixing Koalas and pandas DataFrames in a join operation. Note that the index is dropped. + + >>> pp.sql(''' + ... SELECT m1.a, m2.b + ... FROM {table1} m1 INNER JOIN {table2} m2 + ... ON m1.key = m2.key + ... ORDER BY m1.a, m2.b''', + ... table1=pp.DataFrame({"a": [1,2], "key": ["a", "b"]}), + ... table2=pd.DataFrame({"b": [3,4,5], "key": ["a", "b", "b"]})) + a b + 0 1 3 + 1 2 4 + 2 2 5 + + Also, it is possible to query using Series. + + >>> myser = pp.Series({'a': [1.0, 2.0, 3.0], 'b': [15.0, 30.0, 45.0]}) + >>> pp.sql("SELECT * from {myser}") + 0 + 0 [1.0, 2.0, 3.0] + 1 [15.0, 30.0, 45.0] + """ + if globals is None: + globals = _get_ipython_scope() + _globals = builtin_globals() if globals is None else dict(globals) + _locals = builtin_locals() if locals is None else dict(locals) + # The default choice is the globals + _dict = dict(_globals) + # The vars: + _scope = _get_local_scope() + _dict.update(_scope) + # Then the locals + _dict.update(_locals) + # Highest order of precedence is the locals + _dict.update(kwargs) + return SQLProcessor(_dict, query, default_session()).execute() + + +_CAPTURE_SCOPES = 2 + + +def _get_local_scope(): + # Get 2 scopes above (_get_local_scope -> sql -> ...) to capture the vars there. + try: + return inspect.stack()[_CAPTURE_SCOPES][0].f_locals + except Exception as e: + # TODO (rxin, thunterdb): use a more narrow scope exception. + # See https://github.com/pyspark.pandas/pull/448 + return {} + + +def _get_ipython_scope(): + """ + Tries to extract the dictionary of variables if the program is running + in an IPython notebook environment. + """ + try: + from IPython import get_ipython + + shell = get_ipython() + return shell.user_ns + except Exception as e: + # TODO (rxin, thunterdb): use a more narrow scope exception. + # See https://github.com/pyspark.pandas/pull/448 + return None + + +# Originally from pymysql package +_escape_table = [chr(x) for x in range(128)] +_escape_table[0] = "\\0" +_escape_table[ord("\\")] = "\\\\" +_escape_table[ord("\n")] = "\\n" +_escape_table[ord("\r")] = "\\r" +_escape_table[ord("\032")] = "\\Z" +_escape_table[ord('"')] = '\\"' +_escape_table[ord("'")] = "\\'" + + +def escape_sql_string(value: str) -> str: + """Escapes value without adding quotes. + + >>> escape_sql_string("foo\\nbar") + 'foo\\\\nbar' + + >>> escape_sql_string("'abc'de") + "\\\\'abc\\\\'de" + + >>> escape_sql_string('"abc"de') + '\\\\"abc\\\\"de' + """ + return value.translate(_escape_table) + + +class SQLProcessor(object): + def __init__(self, scope: Dict[str, Any], statement: str, session: SparkSession): + self._scope = scope + self._statement = statement + # All the temporary views created when executing this statement + # The key is the name of the variable in {} + # The value is the cached Spark Dataframe. + self._temp_views = {} # type: Dict[str, SDataFrame] + # All the other variables, converted to a normalized form. + # The normalized form is typically a string + self._cached_vars = {} # type: Dict[str, Any] + # The SQL statement after: + # - all the dataframes have been have been registered as temporary views + # - all the values have been converted normalized to equivalent SQL representations + self._normalized_statement = None # type: Optional[str] + self._session = session + + def execute(self) -> DataFrame: + """ + Returns a DataFrame for which the SQL statement has been executed by + the underlying SQL engine. + + >>> str0 = 'abc' + >>> pp.sql("select {str0}") + abc + 0 abc + + >>> str1 = 'abc"abc' + >>> str2 = "abc'abc" + >>> pp.sql("select {str0}, {str1}, {str2}") + abc abc"abc abc'abc + 0 abc abc"abc abc'abc + + >>> strs = ['a', 'b'] + >>> pp.sql("select 'a' in {strs} as cond1, 'c' in {strs} as cond2") + cond1 cond2 + 0 True False + """ + blocks = _string.formatter_parser(self._statement) + # TODO: use a string builder + res = "" + try: + for (pre, inner, _, _) in blocks: + var_next = "" if inner is None else self._convert(inner) + res = res + pre + var_next + self._normalized_statement = res + + sdf = self._session.sql(self._normalized_statement) + finally: + for v in self._temp_views: + self._session.catalog.dropTempView(v) + return DataFrame(sdf) + + def _convert(self, key) -> Any: + """ + Given a {} key, returns an equivalent SQL representation. + This conversion performs all the necessary escaping so that the string + returned can be directly injected into the SQL statement. + """ + # Already cached? + if key in self._cached_vars: + return self._cached_vars[key] + # Analyze: + if key not in self._scope: + raise ValueError( + "The key {} in the SQL statement was not found in global," + " local or parameters variables".format(key) + ) + var = self._scope[key] + fillin = self._convert_var(var) + self._cached_vars[key] = fillin + return fillin + + def _convert_var(self, var) -> Any: + """ + Converts a python object into a string that is legal SQL. + """ + if isinstance(var, (int, float)): + return str(var) + if isinstance(var, Series): + return self._convert_var(var.to_dataframe()) + if isinstance(var, pd.DataFrame): + return self._convert_var(pp.DataFrame(var)) + if isinstance(var, DataFrame): + df_id = "koalas_" + str(id(var)) + if df_id not in self._temp_views: + sdf = var.to_spark() + sdf.createOrReplaceTempView(df_id) + self._temp_views[df_id] = sdf + return df_id + if isinstance(var, str): + return '"' + escape_sql_string(var) + '"' + if isinstance(var, list): + return "(" + ", ".join([self._convert_var(v) for v in var]) + ")" + if isinstance(var, (tuple, range)): + return self._convert_var(list(var)) + raise ValueError("Unsupported variable type {}: {}".format(type(var).__name__, str(var))) diff --git a/python/pyspark/pandas/strings.py b/python/pyspark/pandas/strings.py new file mode 100644 index 0000000000..3945184dfb --- /dev/null +++ b/python/pyspark/pandas/strings.py @@ -0,0 +1,2289 @@ +# +# 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. +# + +""" +String functions on Koalas Series +""" +from typing import Union, TYPE_CHECKING, cast, Optional, List + +import numpy as np + +from pyspark.sql.types import StringType, BinaryType, ArrayType, LongType, MapType +from pyspark.sql import functions as F +from pyspark.sql.functions import pandas_udf, PandasUDFType + +from pyspark.pandas.spark import functions as SF + +if TYPE_CHECKING: + import pyspark.pandas as pp # noqa: F401 (SPARK-34943) + + +class StringMethods(object): + """String methods for Koalas Series""" + + def __init__(self, series: "pp.Series"): + if not isinstance(series.spark.data_type, (StringType, BinaryType, ArrayType)): + raise ValueError("Cannot call StringMethods on type {}".format(series.spark.data_type)) + self._data = series + + # Methods + def capitalize(self) -> "pp.Series": + """ + Convert Strings in the series to be capitalized. + + Examples + -------- + >>> s = pp.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe']) + >>> s + 0 lower + 1 CAPITALS + 2 this is a sentence + 3 SwApCaSe + dtype: object + + >>> s.str.capitalize() + 0 Lower + 1 Capitals + 2 This is a sentence + 3 Swapcase + dtype: object + """ + + def pandas_capitalize(s) -> "pp.Series[str]": + return s.str.capitalize() + + return self._data.koalas.transform_batch(pandas_capitalize) + + def title(self) -> "pp.Series": + """ + Convert Strings in the series to be titlecase. + + Examples + -------- + >>> s = pp.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe']) + >>> s + 0 lower + 1 CAPITALS + 2 this is a sentence + 3 SwApCaSe + dtype: object + + >>> s.str.title() + 0 Lower + 1 Capitals + 2 This Is A Sentence + 3 Swapcase + dtype: object + """ + + def pandas_title(s) -> "pp.Series[str]": + return s.str.title() + + return self._data.koalas.transform_batch(pandas_title) + + def lower(self) -> "pp.Series": + """ + Convert strings in the Series/Index to all lowercase. + + Examples + -------- + >>> s = pp.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe']) + >>> s + 0 lower + 1 CAPITALS + 2 this is a sentence + 3 SwApCaSe + dtype: object + + >>> s.str.lower() + 0 lower + 1 capitals + 2 this is a sentence + 3 swapcase + dtype: object + """ + return self._data.spark.transform(F.lower) + + def upper(self) -> "pp.Series": + """ + Convert strings in the Series/Index to all uppercase. + + Examples + -------- + >>> s = pp.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe']) + >>> s + 0 lower + 1 CAPITALS + 2 this is a sentence + 3 SwApCaSe + dtype: object + + >>> s.str.upper() + 0 LOWER + 1 CAPITALS + 2 THIS IS A SENTENCE + 3 SWAPCASE + dtype: object + """ + return self._data.spark.transform(F.upper) + + def swapcase(self) -> "pp.Series": + """ + Convert strings in the Series/Index to be swapcased. + + Examples + -------- + >>> s = pp.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe']) + >>> s + 0 lower + 1 CAPITALS + 2 this is a sentence + 3 SwApCaSe + dtype: object + + >>> s.str.swapcase() + 0 LOWER + 1 capitals + 2 THIS IS A SENTENCE + 3 sWaPcAsE + dtype: object + """ + + def pandas_swapcase(s) -> "pp.Series[str]": + return s.str.swapcase() + + return self._data.koalas.transform_batch(pandas_swapcase) + + def startswith(self, pattern, na=None) -> "pp.Series": + """ + Test if the start of each string element matches a pattern. + + Equivalent to :func:`str.startswith`. + + Parameters + ---------- + pattern : str + Character sequence. Regular expressions are not accepted. + na : object, default None + Object shown if element is not a string. NaN converted to None. + + Returns + ------- + Series of bool or object + Koalas Series of booleans indicating whether the given pattern + matches the start of each string element. + + Examples + -------- + >>> s = pp.Series(['bat', 'Bear', 'cat', np.nan]) + >>> s + 0 bat + 1 Bear + 2 cat + 3 None + dtype: object + + >>> s.str.startswith('b') + 0 True + 1 False + 2 False + 3 None + dtype: object + + Specifying na to be False instead of None. + + >>> s.str.startswith('b', na=False) + 0 True + 1 False + 2 False + 3 False + dtype: bool + """ + + def pandas_startswith(s) -> "pp.Series[bool]": + return s.str.startswith(pattern, na) + + return self._data.koalas.transform_batch(pandas_startswith) + + def endswith(self, pattern, na=None) -> "pp.Series": + """ + Test if the end of each string element matches a pattern. + + Equivalent to :func:`str.endswith`. + + Parameters + ---------- + pattern : str + Character sequence. Regular expressions are not accepted. + na : object, default None + Object shown if element is not a string. NaN converted to None. + + Returns + ------- + Series of bool or object + Koalas Series of booleans indicating whether the given pattern + matches the end of each string element. + + Examples + -------- + >>> s = pp.Series(['bat', 'Bear', 'cat', np.nan]) + >>> s + 0 bat + 1 Bear + 2 cat + 3 None + dtype: object + + >>> s.str.endswith('t') + 0 True + 1 False + 2 True + 3 None + dtype: object + + Specifying na to be False instead of None. + + >>> s.str.endswith('t', na=False) + 0 True + 1 False + 2 True + 3 False + dtype: bool + """ + + def pandas_endswith(s) -> "pp.Series[bool]": + return s.str.endswith(pattern, na) + + return self._data.koalas.transform_batch(pandas_endswith) + + def strip(self, to_strip=None) -> "pp.Series": + """ + Remove leading and trailing characters. + + Strip whitespaces (including newlines) or a set of specified + characters from each string in the Series/Index from left and + right sides. Equivalent to :func:`str.strip`. + + Parameters + ---------- + to_strip : str + Specifying the set of characters to be removed. All combinations + of this set of characters will be stripped. If None then + whitespaces are removed. + + Returns + ------- + Series of objects + + Examples + -------- + >>> s = pp.Series(['1. Ant.', '2. Bee!\\t', None]) + >>> s + 0 1. Ant. + 1 2. Bee!\\t + 2 None + dtype: object + + >>> s.str.strip() + 0 1. Ant. + 1 2. Bee! + 2 None + dtype: object + + >>> s.str.strip('12.') + 0 Ant + 1 Bee!\\t + 2 None + dtype: object + + >>> s.str.strip('.!\\t') + 0 1. Ant + 1 2. Bee + 2 None + dtype: object + """ + + def pandas_strip(s) -> "pp.Series[str]": + return s.str.strip(to_strip) + + return self._data.koalas.transform_batch(pandas_strip) + + def lstrip(self, to_strip=None) -> "pp.Series": + """ + Remove leading characters. + + Strip whitespaces (including newlines) or a set of specified + characters from each string in the Series/Index from left side. + Equivalent to :func:`str.lstrip`. + + Parameters + ---------- + to_strip : str + Specifying the set of characters to be removed. All combinations + of this set of characters will be stripped. If None then + whitespaces are removed. + + Returns + ------- + Series of object + + Examples + -------- + >>> s = pp.Series(['1. Ant.', '2. Bee!\\t', None]) + >>> s + 0 1. Ant. + 1 2. Bee!\\t + 2 None + dtype: object + + >>> s.str.lstrip('12.') + 0 Ant. + 1 Bee!\\t + 2 None + dtype: object + """ + + def pandas_lstrip(s) -> "pp.Series[str]": + return s.str.lstrip(to_strip) + + return self._data.koalas.transform_batch(pandas_lstrip) + + def rstrip(self, to_strip=None) -> "pp.Series": + """ + Remove trailing characters. + + Strip whitespaces (including newlines) or a set of specified + characters from each string in the Series/Index from right side. + Equivalent to :func:`str.rstrip`. + + Parameters + ---------- + to_strip : str + Specifying the set of characters to be removed. All combinations + of this set of characters will be stripped. If None then + whitespaces are removed. + + Returns + ------- + Series of object + + Examples + -------- + >>> s = pp.Series(['1. Ant.', '2. Bee!\\t', None]) + >>> s + 0 1. Ant. + 1 2. Bee!\\t + 2 None + dtype: object + + >>> s.str.rstrip('.!\\t') + 0 1. Ant + 1 2. Bee + 2 None + dtype: object + """ + + def pandas_rstrip(s) -> "pp.Series[str]": + return s.str.rstrip(to_strip) + + return self._data.koalas.transform_batch(pandas_rstrip) + + def get(self, i) -> "pp.Series": + """ + Extract element from each string or string list/tuple in the Series + at the specified position. + + Parameters + ---------- + i : int + Position of element to extract. + + Returns + ------- + Series of objects + + Examples + -------- + >>> s1 = pp.Series(["String", "123"]) + >>> s1 + 0 String + 1 123 + dtype: object + + >>> s1.str.get(1) + 0 t + 1 2 + dtype: object + + >>> s1.str.get(-1) + 0 g + 1 3 + dtype: object + + >>> s2 = pp.Series([["a", "b", "c"], ["x", "y"]]) + >>> s2 + 0 [a, b, c] + 1 [x, y] + dtype: object + + >>> s2.str.get(0) + 0 a + 1 x + dtype: object + + >>> s2.str.get(2) + 0 c + 1 None + dtype: object + """ + + def pandas_get(s) -> "pp.Series[str]": + return s.str.get(i) + + return self._data.koalas.transform_batch(pandas_get) + + def isalnum(self) -> "pp.Series": + """ + Check whether all characters in each string are alphanumeric. + + This is equivalent to running the Python string method + :func:`str.isalnum` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s1 = pp.Series(['one', 'one1', '1', '']) + + >>> s1.str.isalnum() + 0 True + 1 True + 2 True + 3 False + dtype: bool + + Note that checks against characters mixed with any additional + punctuation or whitespace will evaluate to false for an alphanumeric + check. + + >>> s2 = pp.Series(['A B', '1.5', '3,000']) + >>> s2.str.isalnum() + 0 False + 1 False + 2 False + dtype: bool + """ + + def pandas_isalnum(s) -> "pp.Series[bool]": + return s.str.isalnum() + + return self._data.koalas.transform_batch(pandas_isalnum) + + def isalpha(self) -> "pp.Series": + """ + Check whether all characters in each string are alphabetic. + + This is equivalent to running the Python string method + :func:`str.isalpha` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s1 = pp.Series(['one', 'one1', '1', '']) + + >>> s1.str.isalpha() + 0 True + 1 False + 2 False + 3 False + dtype: bool + """ + + def pandas_isalpha(s) -> "pp.Series[bool]": + return s.str.isalpha() + + return self._data.koalas.transform_batch(pandas_isalpha) + + def isdigit(self) -> "pp.Series": + """ + Check whether all characters in each string are digits. + + This is equivalent to running the Python string method + :func:`str.isdigit` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s = pp.Series(['23', '³', '⅕', '']) + + The s.str.isdecimal method checks for characters used to form numbers + in base 10. + + >>> s.str.isdecimal() + 0 True + 1 False + 2 False + 3 False + dtype: bool + + The s.str.isdigit method is the same as s.str.isdecimal but also + includes special digits, like superscripted and subscripted digits in + unicode. + + >>> s.str.isdigit() + 0 True + 1 True + 2 False + 3 False + dtype: bool + + The s.str.isnumeric method is the same as s.str.isdigit but also + includes other characters that can represent quantities such as unicode + fractions. + + >>> s.str.isnumeric() + 0 True + 1 True + 2 True + 3 False + dtype: bool + """ + + def pandas_isdigit(s) -> "pp.Series[bool]": + return s.str.isdigit() + + return self._data.koalas.transform_batch(pandas_isdigit) + + def isspace(self) -> "pp.Series": + """ + Check whether all characters in each string are whitespaces. + + This is equivalent to running the Python string method + :func:`str.isspace` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s = pp.Series([' ', '\\t\\r\\n ', '']) + >>> s.str.isspace() + 0 True + 1 True + 2 False + dtype: bool + """ + + def pandas_isspace(s) -> "pp.Series[bool]": + return s.str.isspace() + + return self._data.koalas.transform_batch(pandas_isspace) + + def islower(self) -> "pp.Series": + """ + Check whether all characters in each string are lowercase. + + This is equivalent to running the Python string method + :func:`str.islower` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s = pp.Series(['leopard', 'Golden Eagle', 'SNAKE', '']) + >>> s.str.islower() + 0 True + 1 False + 2 False + 3 False + dtype: bool + """ + + def pandas_isspace(s) -> "pp.Series[bool]": + return s.str.islower() + + return self._data.koalas.transform_batch(pandas_isspace) + + def isupper(self) -> "pp.Series": + """ + Check whether all characters in each string are uppercase. + + This is equivalent to running the Python string method + :func:`str.isupper` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s = pp.Series(['leopard', 'Golden Eagle', 'SNAKE', '']) + >>> s.str.isupper() + 0 False + 1 False + 2 True + 3 False + dtype: bool + """ + + def pandas_isspace(s) -> "pp.Series[bool]": + return s.str.isupper() + + return self._data.koalas.transform_batch(pandas_isspace) + + def istitle(self) -> "pp.Series": + """ + Check whether all characters in each string are titlecase. + + This is equivalent to running the Python string method + :func:`str.istitle` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s = pp.Series(['leopard', 'Golden Eagle', 'SNAKE', '']) + + The s.str.istitle method checks for whether all words are in title + case (whether only the first letter of each word is capitalized). + Words are assumed to be as any sequence of non-numeric characters + separated by whitespace characters. + + >>> s.str.istitle() + 0 False + 1 True + 2 False + 3 False + dtype: bool + """ + + def pandas_istitle(s) -> "pp.Series[bool]": + return s.str.istitle() + + return self._data.koalas.transform_batch(pandas_istitle) + + def isnumeric(self) -> "pp.Series": + """ + Check whether all characters in each string are numeric. + + This is equivalent to running the Python string method + :func:`str.isnumeric` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s1 = pp.Series(['one', 'one1', '1', '']) + >>> s1.str.isnumeric() + 0 False + 1 False + 2 True + 3 False + dtype: bool + + >>> s2 = pp.Series(['23', '³', '⅕', '']) + + The s2.str.isdecimal method checks for characters used to form numbers + in base 10. + + >>> s2.str.isdecimal() + 0 True + 1 False + 2 False + 3 False + dtype: bool + + The s2.str.isdigit method is the same as s2.str.isdecimal but also + includes special digits, like superscripted and subscripted digits in + unicode. + + >>> s2.str.isdigit() + 0 True + 1 True + 2 False + 3 False + dtype: bool + + The s2.str.isnumeric method is the same as s2.str.isdigit but also + includes other characters that can represent quantities such as unicode + fractions. + + >>> s2.str.isnumeric() + 0 True + 1 True + 2 True + 3 False + dtype: bool + """ + + def pandas_isnumeric(s) -> "pp.Series[bool]": + return s.str.isnumeric() + + return self._data.koalas.transform_batch(pandas_isnumeric) + + def isdecimal(self) -> "pp.Series": + """ + Check whether all characters in each string are decimals. + + This is equivalent to running the Python string method + :func:`str.isdecimal` for each element of the Series/Index. + If a string has zero characters, False is returned for that check. + + Examples + -------- + >>> s = pp.Series(['23', '³', '⅕', '']) + + The s.str.isdecimal method checks for characters used to form numbers + in base 10. + + >>> s.str.isdecimal() + 0 True + 1 False + 2 False + 3 False + dtype: bool + + The s.str.isdigit method is the same as s.str.isdecimal but also + includes special digits, like superscripted and subscripted digits in + unicode. + + >>> s.str.isdigit() + 0 True + 1 True + 2 False + 3 False + dtype: bool + + The s.str.isnumeric method is the same as s.str.isdigit but also + includes other characters that can represent quantities such as unicode + fractions. + + >>> s.str.isnumeric() + 0 True + 1 True + 2 True + 3 False + dtype: bool + """ + + def pandas_isdecimal(s) -> "pp.Series[bool]": + return s.str.isdecimal() + + return self._data.koalas.transform_batch(pandas_isdecimal) + + def cat(self, others=None, sep=None, na_rep=None, join=None) -> "pp.Series": + """ + Not supported. + """ + raise NotImplementedError() + + def center(self, width, fillchar=" ") -> "pp.Series": + """ + Filling left and right side of strings in the Series/Index with an + additional character. Equivalent to :func:`str.center`. + + Parameters + ---------- + width : int + Minimum width of resulting string; additional characters will be + filled with fillchar. + fillchar : str + Additional character for filling, default is whitespace. + + Returns + ------- + Series of objects + + Examples + -------- + >>> s = pp.Series(["caribou", "tiger"]) + >>> s + 0 caribou + 1 tiger + dtype: object + + >>> s.str.center(width=10, fillchar='-') + 0 -caribou-- + 1 --tiger--- + dtype: object + """ + + def pandas_center(s) -> "pp.Series[str]": + return s.str.center(width, fillchar) + + return self._data.koalas.transform_batch(pandas_center) + + def contains(self, pat, case=True, flags=0, na=None, regex=True) -> "pp.Series": + """ + Test if pattern or regex is contained within a string of a Series. + + Return boolean Series based on whether a given pattern or regex is + contained within a string of a Series. + + Analogous to :func:`match`, but less strict, relying on + :func:`re.search` instead of :func:`re.match`. + + Parameters + ---------- + pat : str + Character sequence or regular expression. + case : bool, default True + If True, case sensitive. + flags : int, default 0 (no flags) + Flags to pass through to the re module, e.g. re.IGNORECASE. + na : default None + Fill value for missing values. NaN converted to None. + regex : bool, default True + If True, assumes the pat is a regular expression. + If False, treats the pat as a literal string. + + + Returns + ------- + Series of boolean values or object + A Series of boolean values indicating whether the given pattern is + contained within the string of each element of the Series. + + Examples + -------- + Returning a Series of booleans using only a literal pattern. + + >>> s1 = pp.Series(['Mouse', 'dog', 'house and parrot', '23', np.NaN]) + >>> s1.str.contains('og', regex=False) + 0 False + 1 True + 2 False + 3 False + 4 None + dtype: object + + Specifying case sensitivity using case. + + >>> s1.str.contains('oG', case=True, regex=True) + 0 False + 1 False + 2 False + 3 False + 4 None + dtype: object + + Specifying na to be False instead of NaN replaces NaN values with + False. If Series does not contain NaN values the resultant dtype will + be bool, otherwise, an object dtype. + + >>> s1.str.contains('og', na=False, regex=True) + 0 False + 1 True + 2 False + 3 False + 4 False + dtype: bool + + Returning ‘house’ or ‘dog’ when either expression occurs in a string. + + >>> s1.str.contains('house|dog', regex=True) + 0 False + 1 True + 2 True + 3 False + 4 None + dtype: object + + Ignoring case sensitivity using flags with regex. + + >>> import re + >>> s1.str.contains('PARROT', flags=re.IGNORECASE, regex=True) + 0 False + 1 False + 2 True + 3 False + 4 None + dtype: object + + Returning any digit using regular expression. + + >>> s1.str.contains('[0-9]', regex=True) + 0 False + 1 False + 2 False + 3 True + 4 None + dtype: object + + Ensure pat is a not a literal pattern when regex is set to True. + Note in the following example one might expect only s2[1] and s2[3] + to return True. However, ‘.0’ as a regex matches any character followed + by a 0. + + >>> s2 = pp.Series(['40','40.0','41','41.0','35']) + >>> s2.str.contains('.0', regex=True) + 0 True + 1 True + 2 False + 3 True + 4 False + dtype: bool + """ + + def pandas_contains(s) -> "pp.Series[bool]": + return s.str.contains(pat, case, flags, na, regex) + + return self._data.koalas.transform_batch(pandas_contains) + + def count(self, pat, flags=0) -> "pp.Series": + """ + Count occurrences of pattern in each string of the Series. + + This function is used to count the number of times a particular regex + pattern is repeated in each of the string elements of the Series. + + Parameters + ---------- + pat : str + Valid regular expression. + flags : int, default 0 (no flags) + Flags for the re module. + + Returns + ------- + Series of int + A Series containing the integer counts of pattern matches. + + Examples + -------- + >>> s = pp.Series(['A', 'B', 'Aaba', 'Baca', np.NaN, 'CABA', 'cat']) + >>> s.str.count('a') + 0 0.0 + 1 0.0 + 2 2.0 + 3 2.0 + 4 NaN + 5 0.0 + 6 1.0 + dtype: float64 + + Escape '$' to find the literal dollar sign. + + >>> s = pp.Series(['$', 'B', 'Aab$', '$$ca', 'C$B$', 'cat']) + >>> s.str.count('\\$') + 0 1 + 1 0 + 2 1 + 3 2 + 4 2 + 5 0 + dtype: int64 + """ + + def pandas_count(s) -> "pp.Series[int]": + return s.str.count(pat, flags) + + return self._data.koalas.transform_batch(pandas_count) + + def decode(self, encoding, errors="strict") -> "pp.Series": + """ + Not supported. + """ + raise NotImplementedError() + + def encode(self, encoding, errors="strict") -> "pp.Series": + """ + Not supported. + """ + raise NotImplementedError() + + def extract(self, pat, flags=0, expand=True) -> "pp.Series": + """ + Not supported. + """ + raise NotImplementedError() + + def extractall(self, pat, flags=0) -> "pp.Series": + """ + Not supported. + """ + raise NotImplementedError() + + def find(self, sub, start=0, end=None) -> "pp.Series": + """ + Return lowest indexes in each strings in the Series where the + substring is fully contained between [start:end]. + + Return -1 on failure. Equivalent to standard :func:`str.find`. + + Parameters + ---------- + sub : str + Substring being searched. + start : int + Left edge index. + end : int + Right edge index. + + Returns + ------- + Series of int + Series of lowest matching indexes. + + Examples + -------- + >>> s = pp.Series(['apple', 'oranges', 'bananas']) + + >>> s.str.find('a') + 0 0 + 1 2 + 2 1 + dtype: int64 + + >>> s.str.find('a', start=2) + 0 -1 + 1 2 + 2 3 + dtype: int64 + + >>> s.str.find('a', end=1) + 0 0 + 1 -1 + 2 -1 + dtype: int64 + + >>> s.str.find('a', start=2, end=2) + 0 -1 + 1 -1 + 2 -1 + dtype: int64 + """ + + def pandas_find(s) -> "pp.Series[int]": + return s.str.find(sub, start, end) + + return self._data.koalas.transform_batch(pandas_find) + + def findall(self, pat, flags=0) -> "pp.Series": + """ + Find all occurrences of pattern or regular expression in the Series. + + Equivalent to applying :func:`re.findall` to all the elements in + the Series. + + Parameters + ---------- + pat : str + Pattern or regular expression. + flags : int, default 0 (no flags) + `re` module flags, e.g. `re.IGNORECASE`. + + Returns + ------- + Series of object + All non-overlapping matches of pattern or regular expression in + each string of this Series. + + Examples + -------- + >>> s = pp.Series(['Lion', 'Monkey', 'Rabbit']) + + The search for the pattern ‘Monkey’ returns one match: + + >>> s.str.findall('Monkey') + 0 [] + 1 [Monkey] + 2 [] + dtype: object + + On the other hand, the search for the pattern ‘MONKEY’ doesn’t return + any match: + + >>> s.str.findall('MONKEY') + 0 [] + 1 [] + 2 [] + dtype: object + + Flags can be added to the pattern or regular expression. For instance, + to find the pattern ‘MONKEY’ ignoring the case: + + >>> import re + >>> s.str.findall('MONKEY', flags=re.IGNORECASE) + 0 [] + 1 [Monkey] + 2 [] + dtype: object + + When the pattern matches more than one string in the Series, all + matches are returned: + + >>> s.str.findall('on') + 0 [on] + 1 [on] + 2 [] + dtype: object + + Regular expressions are supported too. For instance, the search for all + the strings ending with the word ‘on’ is shown next: + + >>> s.str.findall('on$') + 0 [on] + 1 [] + 2 [] + dtype: object + + If the pattern is found more than once in the same string, then a list + of multiple strings is returned: + + >>> s.str.findall('b') + 0 [] + 1 [] + 2 [b, b] + dtype: object + """ + # type hint does not support to specify array type yet. + pudf = pandas_udf( + lambda s: s.str.findall(pat, flags), + returnType=ArrayType(StringType(), containsNull=True), + functionType=PandasUDFType.SCALAR, + ) + return self._data._with_new_scol(scol=pudf(self._data.spark.column)) + + def index(self, sub, start=0, end=None) -> "pp.Series": + """ + Return lowest indexes in each strings where the substring is fully + contained between [start:end]. + + This is the same as :func:`str.find` except instead of returning -1, + it raises a ValueError when the substring is not found. Equivalent to + standard :func:`str.index`. + + Parameters + ---------- + sub : str + Substring being searched. + start : int + Left edge index. + end : int + Right edge index. + + Returns + ------- + Series of int + Series of lowest matching indexes. + + Examples + -------- + >>> s = pp.Series(['apple', 'oranges', 'bananas']) + + >>> s.str.index('a') + 0 0 + 1 2 + 2 1 + dtype: int64 + + The following expression throws an exception: + + >>> s.str.index('a', start=2) # doctest: +SKIP + """ + + def pandas_index(s) -> "pp.Series[np.int64]": + return s.str.index(sub, start, end) + + return self._data.koalas.transform_batch(pandas_index) + + def join(self, sep) -> "pp.Series": + """ + Join lists contained as elements in the Series with passed delimiter. + + If the elements of a Series are lists themselves, join the content of + these lists using the delimiter passed to the function. This function + is an equivalent to calling :func:`str.join` on the lists. + + Parameters + ---------- + sep : str + Delimiter to use between list entries. + + Returns + ------- + Series of object + Series with list entries concatenated by intervening occurrences of + the delimiter. + + See Also + -------- + str.split : Split strings around given separator/delimiter. + str.rsplit : Splits string around given separator/delimiter, + starting from the right. + + Examples + -------- + Example with a list that contains a None element. + + >>> s = pp.Series([['lion', 'elephant', 'zebra'], + ... ['cat', None, 'dog']]) + >>> s + 0 [lion, elephant, zebra] + 1 [cat, None, dog] + dtype: object + + Join all lists using a ‘-‘. The list containing None will produce None. + + >>> s.str.join('-') + 0 lion-elephant-zebra + 1 None + dtype: object + """ + + def pandas_join(s) -> "pp.Series[str]": + return s.str.join(sep) + + return self._data.koalas.transform_batch(pandas_join) + + def len(self) -> "pp.Series": + """ + Computes the length of each element in the Series. + + The element may be a sequence (such as a string, tuple or list). + + Returns + ------- + Series of int + A Series of integer values indicating the length of each element in + the Series. + + Examples + -------- + Returns the length (number of characters) in a string. Returns the + number of entries for lists or tuples. + + >>> s1 = pp.Series(['dog', 'monkey']) + >>> s1.str.len() + 0 3 + 1 6 + dtype: int64 + + >>> s2 = pp.Series([["a", "b", "c"], []]) + >>> s2.str.len() + 0 3 + 1 0 + dtype: int64 + """ + if isinstance(self._data.spark.data_type, (ArrayType, MapType)): + return self._data.spark.transform(lambda c: F.size(c).cast(LongType())) + else: + return self._data.spark.transform(lambda c: F.length(c).cast(LongType())) + + def ljust(self, width, fillchar=" ") -> "pp.Series": + """ + Filling right side of strings in the Series with an additional + character. Equivalent to :func:`str.ljust`. + + Parameters + ---------- + width : int + Minimum width of resulting string; additional characters will be + filled with `fillchar`. + fillchar : str + Additional character for filling, default is whitespace. + + Returns + ------- + Series of object + + Examples + -------- + >>> s = pp.Series(["caribou", "tiger"]) + >>> s + 0 caribou + 1 tiger + dtype: object + + >>> s.str.ljust(width=10, fillchar='-') + 0 caribou--- + 1 tiger----- + dtype: object + """ + + def pandas_ljust(s) -> "pp.Series[str]": + return s.str.ljust(width, fillchar) + + return self._data.koalas.transform_batch(pandas_ljust) + + def match(self, pat, case=True, flags=0, na=np.NaN) -> "pp.Series": + """ + Determine if each string matches a regular expression. + + Analogous to :func:`contains`, but more strict, relying on + :func:`re.match` instead of :func:`re.search`. + + Parameters + ---------- + pat : str + Character sequence or regular expression. + case : bool, default True + If True, case sensitive. + flags : int, default 0 (no flags) + Flags to pass through to the re module, e.g. re.IGNORECASE. + na : default NaN + Fill value for missing values. + + Returns + ------- + Series of boolean values or object + A Series of boolean values indicating whether the given pattern can + be matched in the string of each element of the Series. + + Examples + -------- + >>> s = pp.Series(['Mouse', 'dog', 'house and parrot', '23', np.NaN]) + >>> s.str.match('dog') + 0 False + 1 True + 2 False + 3 False + 4 None + dtype: object + + >>> s.str.match('mouse|dog', case=False) + 0 True + 1 True + 2 False + 3 False + 4 None + dtype: object + + >>> s.str.match('.+and.+', na=True) + 0 False + 1 False + 2 True + 3 False + 4 True + dtype: bool + + >>> import re + >>> s.str.match('MOUSE', flags=re.IGNORECASE) + 0 True + 1 False + 2 False + 3 False + 4 None + dtype: object + """ + + def pandas_match(s) -> "pp.Series[bool]": + return s.str.match(pat, case, flags, na) + + return self._data.koalas.transform_batch(pandas_match) + + def normalize(self, form) -> "pp.Series": + """ + Return the Unicode normal form for the strings in the Series. + + For more information on the forms, see the + :func:`unicodedata.normalize`. + + Parameters + ---------- + form : {‘NFC’, ‘NFKC’, ‘NFD’, ‘NFKD’} + Unicode form. + + Returns + ------- + Series of objects + A Series of normalized strings. + """ + + def pandas_normalize(s) -> "pp.Series[str]": + return s.str.normalize(form) + + return self._data.koalas.transform_batch(pandas_normalize) + + def pad(self, width, side="left", fillchar=" ") -> "pp.Series": + """ + Pad strings in the Series up to width. + + Parameters + ---------- + width : int + Minimum width of resulting string; additional characters will be + filled with character defined in `fillchar`. + side : {‘left’, ‘right’, ‘both’}, default ‘left’ + Side from which to fill resulting string. + fillchar : str, default ' ' + Additional character for filling, default is whitespace. + + Returns + ------- + Series of object + Returns Series with minimum number of char in object. + + Examples + -------- + >>> s = pp.Series(["caribou", "tiger"]) + >>> s + 0 caribou + 1 tiger + dtype: object + + >>> s.str.pad(width=10) + 0 caribou + 1 tiger + dtype: object + + >>> s.str.pad(width=10, side='right', fillchar='-') + 0 caribou--- + 1 tiger----- + dtype: object + + >>> s.str.pad(width=10, side='both', fillchar='-') + 0 -caribou-- + 1 --tiger--- + dtype: object + """ + + def pandas_pad(s) -> "pp.Series[str]": + return s.str.pad(width, side, fillchar) + + return self._data.koalas.transform_batch(pandas_pad) + + def partition(self, sep=" ", expand=True) -> "pp.Series": + """ + Not supported. + """ + raise NotImplementedError() + + def repeat(self, repeats) -> "pp.Series": + """ + Duplicate each string in the Series. + + Parameters + ---------- + repeats : int + Repeat the string given number of times (int). Sequence of int + is not supported. + + Returns + ------- + Series of object + Series or Index of repeated string objects specified by input + parameter repeats. + + Examples + -------- + >>> s = pp.Series(['a', 'b', 'c']) + >>> s + 0 a + 1 b + 2 c + dtype: object + + Single int repeats string in Series + + >>> s.str.repeat(repeats=2) + 0 aa + 1 bb + 2 cc + dtype: object + """ + if not isinstance(repeats, int): + raise ValueError("repeats expects an int parameter") + return self._data.spark.transform(lambda c: SF.repeat(col=c, n=repeats)) + + def replace(self, pat, repl, n=-1, case=None, flags=0, regex=True) -> "pp.Series": + """ + Replace occurrences of pattern/regex in the Series with some other + string. Equivalent to :func:`str.replace` or :func:`re.sub`. + + Parameters + ---------- + pat : str or compiled regex + String can be a character sequence or regular expression. + repl : str or callable + Replacement string or a callable. The callable is passed the regex + match object and must return a replacement string to be used. See + :func:`re.sub`. + n : int, default -1 (all) + Number of replacements to make from start. + case : boolean, default None + If True, case sensitive (the default if pat is a string). + Set to False for case insensitive. + Cannot be set if pat is a compiled regex. + flags: int, default 0 (no flags) + re module flags, e.g. re.IGNORECASE. + Cannot be set if pat is a compiled regex. + regex : boolean, default True + If True, assumes the passed-in pattern is a regular expression. + If False, treats the pattern as a literal string. + Cannot be set to False if pat is a compile regex or repl is a + callable. + + Returns + ------- + Series of object + A copy of the string with all matching occurrences of pat replaced + by repl. + + Examples + -------- + When pat is a string and regex is True (the default), the given pat is + compiled as a regex. When repl is a string, it replaces matching regex + patterns as with :func:`re.sub`. NaN value(s) in the Series are changed + to None: + + >>> pp.Series(['foo', 'fuz', np.nan]).str.replace('f.', 'ba', regex=True) + 0 bao + 1 baz + 2 None + dtype: object + + When pat is a string and regex is False, every pat is replaced with + repl as with :func:`str.replace`: + + >>> pp.Series(['f.o', 'fuz', np.nan]).str.replace('f.', 'ba', regex=False) + 0 bao + 1 fuz + 2 None + dtype: object + + When repl is a callable, it is called on every pat using + :func:`re.sub`. The callable should expect one positional argument (a + regex object) and return a string. + + Reverse every lowercase alphabetic word: + + >>> repl = lambda m: m.group(0)[::-1] + >>> pp.Series(['foo 123', 'bar baz', np.nan]).str.replace(r'[a-z]+', repl) + 0 oof 123 + 1 rab zab + 2 None + dtype: object + + Using regex groups (extract second group and swap case): + + >>> pat = r"(?P\\w+) (?P\\w+) (?P\\w+)" + >>> repl = lambda m: m.group('two').swapcase() + >>> pp.Series(['One Two Three', 'Foo Bar Baz']).str.replace(pat, repl) + 0 tWO + 1 bAR + dtype: object + + Using a compiled regex with flags: + + >>> import re + >>> regex_pat = re.compile(r'FUZ', flags=re.IGNORECASE) + >>> pp.Series(['foo', 'fuz', np.nan]).str.replace(regex_pat, 'bar') + 0 foo + 1 bar + 2 None + dtype: object + """ + + def pandas_replace(s) -> "pp.Series[str]": + return s.str.replace(pat, repl, n=n, case=case, flags=flags, regex=regex) + + return self._data.koalas.transform_batch(pandas_replace) + + def rfind(self, sub, start=0, end=None) -> "pp.Series": + """ + Return highest indexes in each strings in the Series where the + substring is fully contained between [start:end]. + + Return -1 on failure. Equivalent to standard :func:`str.rfind`. + + Parameters + ---------- + sub : str + Substring being searched. + start : int + Left edge index. + end : int + Right edge index. + + Returns + ------- + Series of int + Series of highest matching indexes. + + Examples + -------- + >>> s = pp.Series(['apple', 'oranges', 'bananas']) + + >>> s.str.rfind('a') + 0 0 + 1 2 + 2 5 + dtype: int64 + + >>> s.str.rfind('a', start=2) + 0 -1 + 1 2 + 2 5 + dtype: int64 + + >>> s.str.rfind('a', end=1) + 0 0 + 1 -1 + 2 -1 + dtype: int64 + + >>> s.str.rfind('a', start=2, end=2) + 0 -1 + 1 -1 + 2 -1 + dtype: int64 + """ + + def pandas_rfind(s) -> "pp.Series[int]": + return s.str.rfind(sub, start, end) + + return self._data.koalas.transform_batch(pandas_rfind) + + def rindex(self, sub, start=0, end=None) -> "pp.Series": + """ + Return highest indexes in each strings where the substring is fully + contained between [start:end]. + + This is the same as :func:`str.rfind` except instead of returning -1, + it raises a ValueError when the substring is not found. Equivalent to + standard :func:`str.rindex`. + + Parameters + ---------- + sub : str + Substring being searched. + start : int + Left edge index. + end : int + Right edge index. + + Returns + ------- + Series of int + Series of highest matching indexes. + + Examples + -------- + >>> s = pp.Series(['apple', 'oranges', 'bananas']) + + >>> s.str.rindex('a') + 0 0 + 1 2 + 2 5 + dtype: int64 + + The following expression throws an exception: + + >>> s.str.rindex('a', start=2) # doctest: +SKIP + """ + + def pandas_rindex(s) -> "pp.Series[np.int64]": + return s.str.rindex(sub, start, end) + + return self._data.koalas.transform_batch(pandas_rindex) + + def rjust(self, width, fillchar=" ") -> "pp.Series": + """ + Filling left side of strings in the Series with an additional + character. Equivalent to :func:`str.rjust`. + + Parameters + ---------- + width : int + Minimum width of resulting string; additional characters will be + filled with `fillchar`. + fillchar : str + Additional character for filling, default is whitespace. + + Returns + ------- + Series of object + + Examples + -------- + >>> s = pp.Series(["caribou", "tiger"]) + >>> s + 0 caribou + 1 tiger + dtype: object + + >>> s.str.rjust(width=10) + 0 caribou + 1 tiger + dtype: object + + >>> s.str.rjust(width=10, fillchar='-') + 0 ---caribou + 1 -----tiger + dtype: object + """ + + def pandas_rjust(s) -> "pp.Series[str]": + return s.str.rjust(width, fillchar) + + return self._data.koalas.transform_batch(pandas_rjust) + + def rpartition(self, sep=" ", expand=True) -> "pp.Series": + """ + Not supported. + """ + raise NotImplementedError() + + def slice(self, start=None, stop=None, step=None) -> "pp.Series": + """ + Slice substrings from each element in the Series. + + Parameters + ---------- + start : int, optional + Start position for slice operation. + stop : int, optional + Stop position for slice operation. + step : int, optional + Step size for slice operation. + + Returns + ------- + Series of object + Series from sliced substrings from original string objects. + + Examples + -------- + >>> s = pp.Series(["koala", "fox", "chameleon"]) + >>> s + 0 koala + 1 fox + 2 chameleon + dtype: object + + >>> s.str.slice(start=1) + 0 oala + 1 ox + 2 hameleon + dtype: object + + >>> s.str.slice(stop=2) + 0 ko + 1 fo + 2 ch + dtype: object + + >>> s.str.slice(step=2) + 0 kaa + 1 fx + 2 caeen + dtype: object + + >>> s.str.slice(start=0, stop=5, step=3) + 0 kl + 1 f + 2 cm + dtype: object + """ + + def pandas_slice(s) -> "pp.Series[str]": + return s.str.slice(start, stop, step) + + return self._data.koalas.transform_batch(pandas_slice) + + def slice_replace(self, start=None, stop=None, repl=None) -> "pp.Series": + """ + Slice substrings from each element in the Series. + + Parameters + ---------- + start : int, optional + Start position for slice operation. If not specified (None), the + slice is unbounded on the left, i.e. slice from the start of the + string. + stop : int, optional + Stop position for slice operation. If not specified (None), the + slice is unbounded on the right, i.e. slice until the end of the + string. + repl : str, optional + String for replacement. If not specified (None), the sliced region + is replaced with an empty string. + + Returns + ------- + Series of object + Series from sliced substrings from original string objects. + + Examples + -------- + >>> s = pp.Series(['a', 'ab', 'abc', 'abdc', 'abcde']) + >>> s + 0 a + 1 ab + 2 abc + 3 abdc + 4 abcde + dtype: object + + Specify just start, meaning replace start until the end of the string + with repl. + + >>> s.str.slice_replace(1, repl='X') + 0 aX + 1 aX + 2 aX + 3 aX + 4 aX + dtype: object + + Specify just stop, meaning the start of the string to stop is replaced + with repl, and the rest of the string is included. + + >>> s.str.slice_replace(stop=2, repl='X') + 0 X + 1 X + 2 Xc + 3 Xdc + 4 Xcde + dtype: object + + Specify start and stop, meaning the slice from start to stop is + replaced with repl. Everything before or after start and stop is + included as is. + + >>> s.str.slice_replace(start=1, stop=3, repl='X') + 0 aX + 1 aX + 2 aX + 3 aXc + 4 aXde + dtype: object + """ + + def pandas_slice_replace(s) -> "pp.Series[str]": + return s.str.slice_replace(start, stop, repl) + + return self._data.koalas.transform_batch(pandas_slice_replace) + + def split(self, pat=None, n=-1, expand=False) -> Union["pp.Series", "pp.DataFrame"]: + """ + Split strings around given separator/delimiter. + + Splits the string in the Series from the beginning, at the specified + delimiter string. Equivalent to :func:`str.split`. + + Parameters + ---------- + pat : str, optional + String or regular expression to split on. If not specified, split + on whitespace. + n : int, default -1 (all) + Limit number of splits in output. None, 0 and -1 will be + interpreted as return all splits. + expand : bool, default False + Expand the splitted strings into separate columns. + + * If ``True``, `n` must be a positive integer, and return DataFrame expanding + dimensionality. + * If ``False``, return Series, containing lists of strings. + + Returns + ------- + Series, DataFrame + Type matches caller unless `expand=True` (see Notes). + + See Also + -------- + str.rsplit : Splits string around given separator/delimiter, + starting from the right. + str.join : Join lists contained as elements in the Series/Index + with passed delimiter. + + Notes + ----- + The handling of the `n` keyword depends on the number of found splits: + + - If found splits > `n`, make first `n` splits only + - If found splits <= `n`, make all splits + - If for a certain row the number of found splits < `n`, + append `None` for padding up to `n` if ``expand=True`` + + If using ``expand=True``, Series callers return DataFrame objects with `n + 1` columns. + + .. note:: Even if `n` is much larger than found splits, the number of columns does NOT + shrink unlike pandas. + + Examples + -------- + >>> s = pp.Series(["this is a regular sentence", + ... "https://docs.python.org/3/tutorial/index.html", + ... np.nan]) + + In the default setting, the string is split by whitespace. + + >>> s.str.split() + 0 [this, is, a, regular, sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + Without the n parameter, the outputs of rsplit and split are identical. + + >>> s.str.rsplit() + 0 [this, is, a, regular, sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + The n parameter can be used to limit the number of splits on the + delimiter. The outputs of split and rsplit are different. + + >>> s.str.split(n=2) + 0 [this, is, a regular sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + >>> s.str.rsplit(n=2) + 0 [this is a, regular, sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + The pat parameter can be used to split by other characters. + + >>> s.str.split(pat = "/") + 0 [this is a regular sentence] + 1 [https:, , docs.python.org, 3, tutorial, index... + 2 None + dtype: object + + When using ``expand=True``, the split elements will expand out into + separate columns. If NaN is present, it is propagated throughout + the columns during the split. + + >>> s.str.split(n=4, expand=True) + 0 1 2 3 4 + 0 this is a regular sentence + 1 https://docs.python.org/3/tutorial/index.html None None None None + 2 None None None None None + + For slightly more complex use cases like splitting the html document name + from a url, a combination of parameter settings can be used. + + >>> s.str.rsplit("/", n=1, expand=True) + 0 1 + 0 this is a regular sentence None + 1 https://docs.python.org/3/tutorial index.html + 2 None None + + Remember to escape special characters when explicitly using regular + expressions. + + >>> s = pp.Series(["1+1=2"]) + >>> s.str.split(r"\\+|=", n=2, expand=True) + 0 1 2 + 0 1 1 2 + """ + from pyspark.pandas.frame import DataFrame + + if expand and n <= 0: + raise NotImplementedError("expand=True is currently only supported with n > 0.") + + # type hint does not support to specify array type yet. + pudf = pandas_udf( + lambda s: s.str.split(pat, n), + returnType=ArrayType(StringType(), containsNull=True), + functionType=PandasUDFType.SCALAR, + ) + kser = self._data._with_new_scol(pudf(self._data.spark.column), dtype=self._data.dtype) + + if expand: + kdf = kser.to_frame() + scol = kdf._internal.data_spark_columns[0] + spark_columns = [scol[i].alias(str(i)) for i in range(n + 1)] + column_labels = [(i,) for i in range(n + 1)] + internal = kdf._internal.with_new_columns( + spark_columns, + column_labels=cast(Optional[List], column_labels), + data_dtypes=([self._data.dtype] * len(column_labels)), + ) + return DataFrame(internal) + else: + return kser + + def rsplit(self, pat=None, n=-1, expand=False) -> Union["pp.Series", "pp.DataFrame"]: + """ + Split strings around given separator/delimiter. + + Splits the string in the Series from the end, at the specified + delimiter string. Equivalent to :func:`str.rsplit`. + + Parameters + ---------- + pat : str, optional + String or regular expression to split on. If not specified, split + on whitespace. + n : int, default -1 (all) + Limit number of splits in output. None, 0 and -1 will be + interpreted as return all splits. + expand : bool, default False + Expand the splitted strings into separate columns. + + * If ``True``, `n` must be a positive integer, and return DataFrame expanding + dimensionality. + * If ``False``, return Series, containing lists of strings. + + Returns + ------- + Series, DataFrame + Type matches caller unless `expand=True` (see Notes). + + See Also + -------- + str.split : Split strings around given separator/delimiter. + str.join : Join lists contained as elements in the Series/Index + with passed delimiter. + + Notes + ----- + The handling of the `n` keyword depends on the number of found splits: + + - If found splits > `n`, make first `n` splits only + - If found splits <= `n`, make all splits + - If for a certain row the number of found splits < `n`, + append `None` for padding up to `n` if ``expand=True`` + + If using ``expand=True``, Series callers return DataFrame objects with `n + 1` columns. + + .. note:: Even if `n` is much larger than found splits, the number of columns does NOT + shrink unlike pandas. + + Examples + -------- + >>> s = pp.Series(["this is a regular sentence", + ... "https://docs.python.org/3/tutorial/index.html", + ... np.nan]) + + In the default setting, the string is split by whitespace. + + >>> s.str.split() + 0 [this, is, a, regular, sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + Without the n parameter, the outputs of rsplit and split are identical. + + >>> s.str.rsplit() + 0 [this, is, a, regular, sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + The n parameter can be used to limit the number of splits on the + delimiter. The outputs of split and rsplit are different. + + >>> s.str.split(n=2) + 0 [this, is, a regular sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + >>> s.str.rsplit(n=2) + 0 [this is a, regular, sentence] + 1 [https://docs.python.org/3/tutorial/index.html] + 2 None + dtype: object + + When using ``expand=True``, the split elements will expand out into + separate columns. If NaN is present, it is propagated throughout + the columns during the split. + + >>> s.str.split(n=4, expand=True) + 0 1 2 3 4 + 0 this is a regular sentence + 1 https://docs.python.org/3/tutorial/index.html None None None None + 2 None None None None None + + For slightly more complex use cases like splitting the html document name + from a url, a combination of parameter settings can be used. + + >>> s.str.rsplit("/", n=1, expand=True) + 0 1 + 0 this is a regular sentence None + 1 https://docs.python.org/3/tutorial index.html + 2 None None + + Remember to escape special characters when explicitly using regular + expressions. + + >>> s = pp.Series(["1+1=2"]) + >>> s.str.split(r"\\+|=", n=2, expand=True) + 0 1 2 + 0 1 1 2 + """ + from pyspark.pandas.frame import DataFrame + + if expand and n <= 0: + raise NotImplementedError("expand=True is currently only supported with n > 0.") + + # type hint does not support to specify array type yet. + pudf = pandas_udf( + lambda s: s.str.rsplit(pat, n), + returnType=ArrayType(StringType(), containsNull=True), + functionType=PandasUDFType.SCALAR, + ) + kser = self._data._with_new_scol(pudf(self._data.spark.column), dtype=self._data.dtype) + + if expand: + kdf = kser.to_frame() + scol = kdf._internal.data_spark_columns[0] + spark_columns = [scol[i].alias(str(i)) for i in range(n + 1)] + column_labels = [(i,) for i in range(n + 1)] + internal = kdf._internal.with_new_columns( + spark_columns, + column_labels=cast(Optional[List], column_labels), + data_dtypes=([self._data.dtype] * len(column_labels)), + ) + return DataFrame(internal) + else: + return kser + + def translate(self, table) -> "pp.Series": + """ + Map all characters in the string through the given mapping table. + Equivalent to standard :func:`str.translate`. + + Parameters + ---------- + table : dict + Table is a mapping of Unicode ordinals to Unicode ordinals, + strings, or None. Unmapped characters are left untouched. + Characters mapped to None are deleted. :func:`str.maketrans` is a + helper function for making translation tables. + + Returns + ------- + Series of object + Series with translated strings. + + Examples + -------- + >>> s = pp.Series(["dog", "cat", "bird"]) + >>> m = str.maketrans({'a': 'X', 'i': 'Y', 'o': None}) + >>> s.str.translate(m) + 0 dg + 1 cXt + 2 bYrd + dtype: object + """ + + def pandas_translate(s) -> "pp.Series[str]": + return s.str.translate(table) + + return self._data.koalas.transform_batch(pandas_translate) + + def wrap(self, width, **kwargs) -> "pp.Series": + """ + Wrap long strings in the Series to be formatted in paragraphs with + length less than a given width. + + This method has the same keyword parameters and defaults as + :class:`textwrap.TextWrapper`. + + Parameters + ---------- + width : int + Maximum line-width. Lines separated with newline char. + expand_tabs : bool, optional + If true, tab characters will be expanded to spaces (default: True). + replace_whitespace : bool, optional + If true, each whitespace character remaining after tab expansion + will be replaced by a single space (default: True). + drop_whitespace : bool, optional + If true, whitespace that, after wrapping, happens to end up at the + beginning or end of a line is dropped (default: True). + break_long_words : bool, optional + If true, then words longer than width will be broken in order to + ensure that no lines are longer than width. If it is false, long + words will not be broken, and some lines may be longer than width + (default: True). + break_on_hyphens : bool, optional + If true, wrapping will occur preferably on whitespace and right + after hyphens in compound words, as it is customary in English. + If false, only whitespaces will be considered as potentially good + places for line breaks, but you need to set break_long_words to + false if you want truly insecable words (default: True). + + Returns + ------- + Series of object + Series with wrapped strings. + + Examples + -------- + >>> s = pp.Series(['line to be wrapped', 'another line to be wrapped']) + >>> s.str.wrap(12) + 0 line to be\\nwrapped + 1 another line\\nto be\\nwrapped + dtype: object + """ + + def pandas_wrap(s) -> "pp.Series[str]": + return s.str.wrap(width, **kwargs) + + return self._data.koalas.transform_batch(pandas_wrap) + + def zfill(self, width) -> "pp.Series": + """ + Pad strings in the Series by prepending ‘0’ characters. + + Strings in the Series are padded with ‘0’ characters on the left of the + string to reach a total string length width. Strings in the Series with + length greater or equal to width are unchanged. + + Differs from :func:`str.zfill` which has special handling for ‘+’/’-‘ + in the string. + + Parameters + ---------- + width : int + Minimum length of resulting string; strings with length less than + width be prepended with ‘0’ characters. + + Returns + ------- + Series of object + Series with '0' left-padded strings. + + Examples + -------- + >>> s = pp.Series(['-1', '1', '1000', np.nan]) + >>> s + 0 -1 + 1 1 + 2 1000 + 3 None + dtype: object + + Note that NaN is not a string, therefore it is converted to NaN. The + minus sign in '-1' is treated as a regular character and the zero is + added to the left of it (:func:`str.zfill` would have moved it to the + left). 1000 remains unchanged as it is longer than width. + + >>> s.str.zfill(3) + 0 0-1 + 1 001 + 2 1000 + 3 None + dtype: object + """ + + def pandas_zfill(s) -> "pp.Series[str]": + return s.str.zfill(width) + + return self._data.koalas.transform_batch(pandas_zfill) + + def get_dummies(self, sep="|"): + """ + Not supported. + """ + raise NotImplementedError() diff --git a/python/pyspark/pandas/typedef/__init__.py b/python/pyspark/pandas/typedef/__init__.py new file mode 100644 index 0000000000..5f7ea2834a --- /dev/null +++ b/python/pyspark/pandas/typedef/__init__.py @@ -0,0 +1,18 @@ +# +# 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. +# + +from pyspark.pandas.typedef.typehints import * # noqa: F401,F405 diff --git a/python/pyspark/pandas/typedef/string_typehints.py b/python/pyspark/pandas/typedef/string_typehints.py new file mode 100644 index 0000000000..c78cdbba96 --- /dev/null +++ b/python/pyspark/pandas/typedef/string_typehints.py @@ -0,0 +1,37 @@ +# +# 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 numpy as np # noqa: F401 +import pandas # noqa: F401 +import pandas as pd # noqa: F401 +from numpy import * # noqa: F401 +from pandas import * # noqa: F401 +from inspect import getfullargspec # noqa: F401 + + +def resolve_string_type_hint(tpe): + import pyspark.pandas as pp + from pyspark.pandas import DataFrame, Series + + locs = { + "pp": pp, + "koalas": pp, + "DataFrame": DataFrame, + "Series": Series, + } + # This is a hack to resolve the forward reference string. + exec("def func() -> %s: pass\narg_spec = getfullargspec(func)" % tpe, globals(), locs) + return locs["arg_spec"].annotations.get("return", None) diff --git a/python/pyspark/pandas/typedef/typehints.py b/python/pyspark/pandas/typedef/typehints.py new file mode 100644 index 0000000000..657c9a74d3 --- /dev/null +++ b/python/pyspark/pandas/typedef/typehints.py @@ -0,0 +1,521 @@ +# +# 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. +# + +""" +Utilities to deal with types. This is mostly focused on python3. +""" +import datetime +import decimal +from inspect import getfullargspec, isclass +from typing import Generic, List, Optional, Tuple, TypeVar, Union # noqa: F401 + +import numpy as np +import pandas as pd +from pandas.api.types import CategoricalDtype, pandas_dtype +from pandas.api.extensions import ExtensionDtype + +try: + from pandas import Int8Dtype, Int16Dtype, Int32Dtype, Int64Dtype + + extension_dtypes_available = True + extension_dtypes = (Int8Dtype, Int16Dtype, Int32Dtype, Int64Dtype) # type: Tuple + + try: + from pandas import BooleanDtype, StringDtype + + extension_object_dtypes_available = True + extension_dtypes += (BooleanDtype, StringDtype) + except ImportError: + extension_object_dtypes_available = False + + try: + from pandas import Float32Dtype, Float64Dtype + + extension_float_dtypes_available = True + extension_dtypes += (Float32Dtype, Float64Dtype) + except ImportError: + extension_float_dtypes_available = False + +except ImportError: + extension_dtypes_available = False + extension_object_dtypes_available = False + extension_float_dtypes_available = False + extension_dtypes = () + +import pyarrow as pa +import pyspark.sql.types as types + +try: + from pyspark.sql.types import to_arrow_type, from_arrow_type +except ImportError: + from pyspark.sql.pandas.types import to_arrow_type, from_arrow_type + +from pyspark import pandas as pp # For running doctests and reference resolution in PyCharm. +from pyspark.pandas.typedef.string_typehints import resolve_string_type_hint + +T = TypeVar("T") + +Scalar = Union[ + int, float, bool, str, bytes, decimal.Decimal, datetime.date, datetime.datetime, None +] + +Dtype = Union[np.dtype, ExtensionDtype] + + +# A column of data, with the data type. +class SeriesType(Generic[T]): + def __init__(self, dtype: Dtype, spark_type: types.DataType): + self.dtype = dtype + self.spark_type = spark_type + + def __repr__(self): + return "SeriesType[{}]".format(self.spark_type) + + +class DataFrameType(object): + def __init__( + self, dtypes: List[Dtype], spark_types: List[types.DataType], names: List[Optional[str]] + ): + from pyspark.pandas.utils import name_like_string + + self.dtypes = dtypes + self.spark_type = types.StructType( + [ + types.StructField(name_like_string(n) if n is not None else ("c%s" % i), t) + for i, (n, t) in enumerate(zip(names, spark_types)) + ] + ) # type: types.StructType + + def __repr__(self): + return "DataFrameType[{}]".format(self.spark_type) + + +# The type is a scalar type that is furthermore understood by Spark. +class ScalarType(object): + def __init__(self, dtype: Dtype, spark_type: types.DataType): + self.dtype = dtype + self.spark_type = spark_type + + def __repr__(self): + return "ScalarType[{}]".format(self.spark_type) + + +# The type is left unspecified or we do not know about this type. +class UnknownType(object): + def __init__(self, tpe): + self.tpe = tpe + + def __repr__(self): + return "UnknownType[{}]".format(self.tpe) + + +class NameTypeHolder(object): + name = None + tpe = None + + +def as_spark_type(tpe: Union[str, type, Dtype], *, raise_error: bool = True) -> types.DataType: + """ + Given a Python type, returns the equivalent spark type. + Accepts: + - the built-in types in Python + - the built-in types in numpy + - list of pairs of (field_name, type) + - dictionaries of field_name -> type + - Python3's typing system + """ + if isinstance(tpe, np.dtype) and tpe == np.dtype("object"): + pass + # ArrayType + elif tpe in (np.ndarray,): + return types.ArrayType(types.StringType()) + elif hasattr(tpe, "__origin__") and issubclass(tpe.__origin__, list): # type: ignore + element_type = as_spark_type(tpe.__args__[0], raise_error=raise_error) # type: ignore + if element_type is None: + return None + return types.ArrayType(element_type) + # BinaryType + elif tpe in (bytes, np.character, np.bytes_, np.string_): + return types.BinaryType() + # BooleanType + elif tpe in (bool, np.bool, "bool", "?"): + return types.BooleanType() + # DateType + elif tpe in (datetime.date,): + return types.DateType() + # NumericType + elif tpe in (np.int8, np.byte, "int8", "byte", "b"): + return types.ByteType() + elif tpe in (decimal.Decimal,): + # TODO: considering about the precision & scale for decimal type. + return types.DecimalType(38, 18) + elif tpe in (float, np.float, np.float64, "float", "float64", "double"): + return types.DoubleType() + elif tpe in (np.float32, "float32", "f"): + return types.FloatType() + elif tpe in (np.int32, "int32", "i"): + return types.IntegerType() + elif tpe in (int, np.int, np.int64, "int", "int64", "long"): + return types.LongType() + elif tpe in (np.int16, "int16", "short"): + return types.ShortType() + # StringType + elif tpe in (str, np.unicode_, "str", "U"): + return types.StringType() + # TimestampType + elif tpe in (datetime.datetime, np.datetime64, "datetime64[ns]", "M"): + return types.TimestampType() + + # categorical types + elif isinstance(tpe, CategoricalDtype) or (isinstance(tpe, str) and type == "category"): + return types.LongType() + + # extension types + elif extension_dtypes_available: + # IntegralType + if isinstance(tpe, Int8Dtype) or (isinstance(tpe, str) and tpe == "Int8"): + return types.ByteType() + elif isinstance(tpe, Int16Dtype) or (isinstance(tpe, str) and tpe == "Int16"): + return types.ShortType() + elif isinstance(tpe, Int32Dtype) or (isinstance(tpe, str) and tpe == "Int32"): + return types.IntegerType() + elif isinstance(tpe, Int64Dtype) or (isinstance(tpe, str) and tpe == "Int64"): + return types.LongType() + + if extension_object_dtypes_available: + # BooleanType + if isinstance(tpe, BooleanDtype) or (isinstance(tpe, str) and tpe == "boolean"): + return types.BooleanType() + # StringType + elif isinstance(tpe, StringDtype) or (isinstance(tpe, str) and tpe == "string"): + return types.StringType() + + if extension_float_dtypes_available: + # FractionalType + if isinstance(tpe, Float32Dtype) or (isinstance(tpe, str) and tpe == "Float32"): + return types.FloatType() + elif isinstance(tpe, Float64Dtype) or (isinstance(tpe, str) and tpe == "Float64"): + return types.DoubleType() + + if raise_error: + raise TypeError("Type %s was not understood." % tpe) + else: + return None + + +def spark_type_to_pandas_dtype( + spark_type: types.DataType, *, use_extension_dtypes: bool = False +) -> Dtype: + """ Return the given Spark DataType to pandas dtype. """ + + if use_extension_dtypes and extension_dtypes_available: + # IntegralType + if isinstance(spark_type, types.ByteType): + return Int8Dtype() + elif isinstance(spark_type, types.ShortType): + return Int16Dtype() + elif isinstance(spark_type, types.IntegerType): + return Int32Dtype() + elif isinstance(spark_type, types.LongType): + return Int64Dtype() + + if extension_object_dtypes_available: + # BooleanType + if isinstance(spark_type, types.BooleanType): + return BooleanDtype() + # StringType + elif isinstance(spark_type, types.StringType): + return StringDtype() + + # FractionalType + if extension_float_dtypes_available: + if isinstance(spark_type, types.FloatType): + return Float32Dtype() + elif isinstance(spark_type, types.DoubleType): + return Float64Dtype() + + if isinstance( + spark_type, + ( + types.DateType, + types.NullType, + types.ArrayType, + types.MapType, + types.StructType, + types.UserDefinedType, + ), + ): + return np.dtype("object") + elif isinstance(spark_type, types.TimestampType): + return np.dtype("datetime64[ns]") + else: + return np.dtype(to_arrow_type(spark_type).to_pandas_dtype()) + + +def koalas_dtype(tpe) -> Tuple[Dtype, types.DataType]: + """ + Convert input into a pandas only dtype object or a numpy dtype object, + and its corresponding Spark DataType. + + Parameters + ---------- + tpe : object to be converted + + Returns + ------- + tuple of np.dtype or a pandas dtype, and Spark DataType + + Raises + ------ + TypeError if not a dtype + + Examples + -------- + >>> koalas_dtype(int) + (dtype('int64'), LongType) + >>> koalas_dtype(str) + (dtype('>> koalas_dtype(datetime.date) + (dtype('O'), DateType) + >>> koalas_dtype(datetime.datetime) + (dtype('>> koalas_dtype(List[bool]) + (dtype('O'), ArrayType(BooleanType,true)) + """ + try: + dtype = pandas_dtype(tpe) + spark_type = as_spark_type(dtype) + except TypeError: + spark_type = as_spark_type(tpe) + dtype = spark_type_to_pandas_dtype(spark_type) + return dtype, spark_type + + +def infer_pd_series_spark_type(pser: pd.Series, dtype: Dtype) -> types.DataType: + """Infer Spark DataType from pandas Series dtype. + + :param pser: :class:`pandas.Series` to be inferred + :param dtype: the Series' dtype + :return: the inferred Spark data type + """ + if dtype == np.dtype("object"): + if len(pser) == 0 or pser.isnull().all(): + return types.NullType() + elif hasattr(pser.iloc[0], "__UDT__"): + return pser.iloc[0].__UDT__ + else: + return from_arrow_type(pa.Array.from_pandas(pser).type) + elif isinstance(dtype, CategoricalDtype): + # `pser` must already be converted to codes. + return as_spark_type(pser.dtype) + else: + return as_spark_type(dtype) + + +def infer_return_type(f) -> Union[SeriesType, DataFrameType, ScalarType, UnknownType]: + """ + Infer the return type from the return type annotation of the given function. + + The returned type class indicates both dtypes (a pandas only dtype object + or a numpy dtype object) and its corresponding Spark DataType. + + >>> def func() -> int: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtype + dtype('int64') + >>> inferred.spark_type + LongType + + >>> def func() -> pp.Series[int]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtype + dtype('int64') + >>> inferred.spark_type + LongType + + >>> def func() -> pp.DataFrame[np.float, str]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('float64'), dtype('>> inferred.spark_type + StructType(List(StructField(c0,DoubleType,true),StructField(c1,StringType,true))) + + >>> def func() -> pp.DataFrame[np.float]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('float64')] + >>> inferred.spark_type + StructType(List(StructField(c0,DoubleType,true))) + + >>> def func() -> 'int': + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtype + dtype('int64') + >>> inferred.spark_type + LongType + + >>> def func() -> 'pp.Series[int]': + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtype + dtype('int64') + >>> inferred.spark_type + LongType + + >>> def func() -> 'pp.DataFrame[np.float, str]': + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('float64'), dtype('>> inferred.spark_type + StructType(List(StructField(c0,DoubleType,true),StructField(c1,StringType,true))) + + >>> def func() -> 'pp.DataFrame[np.float]': + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('float64')] + >>> inferred.spark_type + StructType(List(StructField(c0,DoubleType,true))) + + >>> def func() -> pp.DataFrame['a': np.float, 'b': int]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('float64'), dtype('int64')] + >>> inferred.spark_type + StructType(List(StructField(a,DoubleType,true),StructField(b,LongType,true))) + + >>> def func() -> "pp.DataFrame['a': np.float, 'b': int]": + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('float64'), dtype('int64')] + >>> inferred.spark_type + StructType(List(StructField(a,DoubleType,true),StructField(b,LongType,true))) + + >>> pdf = pd.DataFrame({"a": [1, 2, 3], "b": [3, 4, 5]}) + >>> def func() -> pp.DataFrame[pdf.dtypes]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('int64'), dtype('int64')] + >>> inferred.spark_type + StructType(List(StructField(c0,LongType,true),StructField(c1,LongType,true))) + + >>> pdf = pd.DataFrame({"a": [1, 2, 3], "b": [3, 4, 5]}) + >>> def func() -> pp.DataFrame[zip(pdf.columns, pdf.dtypes)]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('int64'), dtype('int64')] + >>> inferred.spark_type + StructType(List(StructField(a,LongType,true),StructField(b,LongType,true))) + + >>> pdf = pd.DataFrame({("x", "a"): [1, 2, 3], ("y", "b"): [3, 4, 5]}) + >>> def func() -> pp.DataFrame[zip(pdf.columns, pdf.dtypes)]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('int64'), dtype('int64')] + >>> inferred.spark_type + StructType(List(StructField((x, a),LongType,true),StructField((y, b),LongType,true))) + + >>> pdf = pd.DataFrame({"a": [1, 2, 3], "b": pd.Categorical([3, 4, 5])}) + >>> def func() -> pp.DataFrame[pdf.dtypes]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('int64'), CategoricalDtype(categories=[3, 4, 5], ordered=False)] + >>> inferred.spark_type + StructType(List(StructField(c0,LongType,true),StructField(c1,LongType,true))) + + >>> def func() -> pp.DataFrame[zip(pdf.columns, pdf.dtypes)]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtypes + [dtype('int64'), CategoricalDtype(categories=[3, 4, 5], ordered=False)] + >>> inferred.spark_type + StructType(List(StructField(a,LongType,true),StructField(b,LongType,true))) + + >>> def func() -> pp.Series[pdf.b.dtype]: + ... pass + >>> inferred = infer_return_type(func) + >>> inferred.dtype + CategoricalDtype(categories=[3, 4, 5], ordered=False) + >>> inferred.spark_type + LongType + """ + # We should re-import to make sure the class 'SeriesType' is not treated as a class + # within this module locally. See Series.__class_getitem__ which imports this class + # canonically. + from pyspark.pandas.typedef import SeriesType, NameTypeHolder + + spec = getfullargspec(f) + tpe = spec.annotations.get("return", None) + if isinstance(tpe, str): + # This type hint can happen when given hints are string to avoid forward reference. + tpe = resolve_string_type_hint(tpe) + + if hasattr(tpe, "__origin__") and ( + tpe.__origin__ == pp.DataFrame or tpe.__origin__ == pp.Series + ): + # When Python version is lower then 3.7. Unwrap it to a Tuple/SeriesType type hints. + tpe = tpe.__args__[0] + + if hasattr(tpe, "__origin__") and issubclass(tpe.__origin__, SeriesType): + tpe = tpe.__args__[0] + if issubclass(tpe, NameTypeHolder): + tpe = tpe.tpe + dtype, spark_type = koalas_dtype(tpe) + return SeriesType(dtype, spark_type) + + # Note that, DataFrame type hints will create a Tuple. + # Python 3.6 has `__name__`. Python 3.7 and 3.8 have `_name`. + # Check if the name is Tuple. + name = getattr(tpe, "_name", getattr(tpe, "__name__", None)) + if name == "Tuple": + tuple_type = tpe + if hasattr(tuple_type, "__tuple_params__"): + # Python 3.5.0 to 3.5.2 has '__tuple_params__' instead. + # See https://github.com/python/cpython/blob/v3.5.2/Lib/typing.py + parameters = getattr(tuple_type, "__tuple_params__") + else: + parameters = getattr(tuple_type, "__args__") + dtypes, spark_types = zip( + *( + koalas_dtype(p.tpe) + if isclass(p) and issubclass(p, NameTypeHolder) + else koalas_dtype(p) + for p in parameters + ) + ) + names = [ + p.name if isclass(p) and issubclass(p, NameTypeHolder) else None for p in parameters + ] + return DataFrameType(list(dtypes), list(spark_types), names) + + types = koalas_dtype(tpe) + if types is None: + return UnknownType(tpe) + else: + return ScalarType(*types) diff --git a/python/pyspark/pandas/usage_logging/__init__.py b/python/pyspark/pandas/usage_logging/__init__.py new file mode 100644 index 0000000000..cd04d9930a --- /dev/null +++ b/python/pyspark/pandas/usage_logging/__init__.py @@ -0,0 +1,269 @@ +# +# 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 functools +import importlib +import inspect +import threading +import time +from types import ModuleType +from typing import Union + +import pandas as pd + +from pyspark.pandas import config, namespace, sql +from pyspark.pandas.accessors import KoalasFrameMethods +from pyspark.pandas.frame import DataFrame +from pyspark.pandas.datetimes import DatetimeMethods +from pyspark.pandas.groupby import DataFrameGroupBy, SeriesGroupBy +from pyspark.pandas.indexes.base import Index +from pyspark.pandas.indexes.category import CategoricalIndex +from pyspark.pandas.indexes.datetimes import DatetimeIndex +from pyspark.pandas.indexes.multi import MultiIndex +from pyspark.pandas.indexes.numeric import Float64Index, Int64Index +from pyspark.pandas.missing.frame import _MissingPandasLikeDataFrame +from pyspark.pandas.missing.groupby import ( + MissingPandasLikeDataFrameGroupBy, + MissingPandasLikeSeriesGroupBy, +) +from pyspark.pandas.missing.indexes import ( + MissingPandasLikeCategoricalIndex, + MissingPandasLikeDatetimeIndex, + MissingPandasLikeIndex, + MissingPandasLikeMultiIndex, +) +from pyspark.pandas.missing.series import MissingPandasLikeSeries +from pyspark.pandas.missing.window import ( + MissingPandasLikeExpanding, + MissingPandasLikeRolling, + MissingPandasLikeExpandingGroupby, + MissingPandasLikeRollingGroupby, +) +from pyspark.pandas.series import Series +from pyspark.pandas.spark.accessors import ( + CachedSparkFrameMethods, + SparkFrameMethods, + SparkIndexOpsMethods, +) +from pyspark.pandas.strings import StringMethods +from pyspark.pandas.window import Expanding, ExpandingGroupby, Rolling, RollingGroupby + + +def attach(logger_module: Union[str, ModuleType]) -> None: + """ + Attach the usage logger. + + Parameters + ---------- + logger_module : the module or module name contains the usage logger. + The module needs to provide `get_logger` function as an entry point of the plug-in + returning the usage logger. + + See Also + -------- + usage_logger : the reference implementation of the usage logger. + """ + if isinstance(logger_module, str): + logger_module = importlib.import_module(logger_module) + + logger = getattr(logger_module, "get_logger")() + + modules = [config, namespace] + classes = [ + DataFrame, + Series, + Index, + MultiIndex, + Int64Index, + Float64Index, + CategoricalIndex, + DatetimeIndex, + DataFrameGroupBy, + SeriesGroupBy, + DatetimeMethods, + StringMethods, + Expanding, + ExpandingGroupby, + Rolling, + RollingGroupby, + CachedSparkFrameMethods, + SparkFrameMethods, + SparkIndexOpsMethods, + KoalasFrameMethods, + ] + + try: + from pyspark.pandas import mlflow + + modules.append(mlflow) + classes.append(mlflow.PythonModelWrapper) + except ImportError: + pass + + sql._CAPTURE_SCOPES = 3 # type: ignore + modules.append(sql) # type: ignore + + # Modules + for target_module in modules: + target_name = target_module.__name__.split(".")[-1] + for name in getattr(target_module, "__all__"): + func = getattr(target_module, name) + if not inspect.isfunction(func): + continue + setattr(target_module, name, _wrap_function(target_name, name, func, logger)) + + special_functions = set( + [ + "__init__", + "__repr__", + "__str__", + "_repr_html_", + "__len__", + "__getitem__", + "__setitem__", + "__getattr__", + ] + ) + + # Classes + for target_class in classes: + for name, func in inspect.getmembers(target_class, inspect.isfunction): + if name.startswith("_") and name not in special_functions: + continue + setattr(target_class, name, _wrap_function(target_class.__name__, name, func, logger)) + + for name, prop in inspect.getmembers(target_class, lambda o: isinstance(o, property)): + if name.startswith("_"): + continue + setattr(target_class, name, _wrap_property(target_class.__name__, name, prop, logger)) + + # Missings + for original, missing in [ + (pd.DataFrame, _MissingPandasLikeDataFrame), + (pd.Series, MissingPandasLikeSeries), + (pd.Index, MissingPandasLikeIndex), + (pd.MultiIndex, MissingPandasLikeMultiIndex), + (pd.CategoricalIndex, MissingPandasLikeCategoricalIndex), + (pd.DatetimeIndex, MissingPandasLikeDatetimeIndex), + (pd.core.groupby.DataFrameGroupBy, MissingPandasLikeDataFrameGroupBy), + (pd.core.groupby.SeriesGroupBy, MissingPandasLikeSeriesGroupBy), + (pd.core.window.Expanding, MissingPandasLikeExpanding), + (pd.core.window.Rolling, MissingPandasLikeRolling), + (pd.core.window.ExpandingGroupby, MissingPandasLikeExpandingGroupby), + (pd.core.window.RollingGroupby, MissingPandasLikeRollingGroupby), + ]: + for name, func in inspect.getmembers(missing, inspect.isfunction): + setattr( + missing, + name, + _wrap_missing_function(original.__name__, name, func, original, logger), + ) + + for name, prop in inspect.getmembers(missing, lambda o: isinstance(o, property)): + setattr(missing, name, _wrap_missing_property(original.__name__, name, prop, logger)) + + +_local = threading.local() + + +def _wrap_function(class_name, function_name, func, logger): + + signature = inspect.signature(func) + + @functools.wraps(func) + def wrapper(*args, **kwargs): + if hasattr(_local, "logging") and _local.logging: + # no need to log since this should be internal call. + return func(*args, **kwargs) + _local.logging = True + try: + start = time.perf_counter() + try: + res = func(*args, **kwargs) + logger.log_success( + class_name, function_name, time.perf_counter() - start, signature + ) + return res + except Exception as ex: + logger.log_failure( + class_name, function_name, ex, time.perf_counter() - start, signature + ) + raise + finally: + _local.logging = False + + return wrapper + + +def _wrap_property(class_name, property_name, prop, logger): + @property + def wrapper(self): + if hasattr(_local, "logging") and _local.logging: + # no need to log since this should be internal call. + return prop.fget(self) + _local.logging = True + try: + start = time.perf_counter() + try: + res = prop.fget(self) + logger.log_success(class_name, property_name, time.perf_counter() - start) + return res + except Exception as ex: + logger.log_failure(class_name, property_name, ex, time.perf_counter() - start) + raise + finally: + _local.logging = False + + wrapper.__doc__ = prop.__doc__ + + if prop.fset is not None: + wrapper = wrapper.setter(_wrap_function(class_name, prop.fset.__name__, prop.fset, logger)) + + return wrapper + + +def _wrap_missing_function(class_name, function_name, func, original, logger): + + if not hasattr(original, function_name): + return func + + signature = inspect.signature(getattr(original, function_name)) + + is_deprecated = func.__name__ == "deprecated_function" + + @functools.wraps(func) + def wrapper(*args, **kwargs): + try: + return func(*args, **kwargs) + finally: + logger.log_missing(class_name, function_name, is_deprecated, signature) + + return wrapper + + +def _wrap_missing_property(class_name, property_name, prop, logger): + + is_deprecated = prop.fget.__name__ == "deprecated_property" + + @property + def wrapper(self): + try: + return prop.fget(self) + finally: + logger.log_missing(class_name, property_name, is_deprecated) + + return wrapper diff --git a/python/pyspark/pandas/usage_logging/usage_logger.py b/python/pyspark/pandas/usage_logging/usage_logger.py new file mode 100644 index 0000000000..25ec5470bc --- /dev/null +++ b/python/pyspark/pandas/usage_logging/usage_logger.py @@ -0,0 +1,132 @@ +# +# 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. +# + +""" +The reference implementation of usage logger using the Python standard logging library. +""" + +from inspect import Signature +import logging +from typing import Any, Optional + + +def get_logger() -> Any: + """ An entry point of the plug-in and return the usage logger. """ + return KoalasUsageLogger() + + +def _format_signature(signature): + return ( + "({})".format(", ".join([p.name for p in signature.parameters.values()])) + if signature is not None + else "" + ) + + +class KoalasUsageLogger(object): + """ + The reference implementation of usage logger. + + The usage logger needs to provide the following methods: + + - log_success(self, class_name, name, duration, signature=None) + - log_failure(self, class_name, name, ex, duration, signature=None) + - log_missing(self, class_name, name, is_deprecated=False, signature=None) + """ + + def __init__(self): + self.logger = logging.getLogger("pyspark.pandas.usage_logger") + + def log_success( + self, class_name: str, name: str, duration: float, signature: Optional[Signature] = None + ) -> None: + """ + Log the function or property call is successfully finished. + + :param class_name: the target class name + :param name: the target function or property name + :param duration: the duration to finish the function or property call + :param signature: the signature if the target is a function, else None + """ + if self.logger.isEnabledFor(logging.INFO): + msg = ( + "A {function} `{class_name}.{name}{signature}` was successfully finished " + "after {duration:.3f} ms." + ).format( + class_name=class_name, + name=name, + signature=_format_signature(signature), + duration=duration * 1000, + function="function" if signature is not None else "property", + ) + self.logger.info(msg) + + def log_failure( + self, + class_name: str, + name: str, + ex: Exception, + duration: float, + signature: Optional[Signature] = None, + ) -> None: + """ + Log the function or property call failed. + + :param class_name: the target class name + :param name: the target function or property name + :param ex: the exception causing the failure + :param duration: the duration until the function or property call fails + :param signature: the signature if the target is a function, else None + """ + if self.logger.isEnabledFor(logging.WARNING): + msg = ( + "A {function} `{class_name}.{name}{signature}` was failed " + "after {duration:.3f} ms: {msg}" + ).format( + class_name=class_name, + name=name, + signature=_format_signature(signature), + msg=str(ex), + duration=duration * 1000, + function="function" if signature is not None else "property", + ) + self.logger.warning(msg) + + def log_missing( + self, + class_name: str, + name: str, + is_deprecated: bool = False, + signature: Optional[Signature] = None, + ) -> None: + """ + Log the missing or deprecated function or property is called. + + :param class_name: the target class name + :param name: the target function or property name + :param is_deprecated: True if the function or property is marked as deprecated + :param signature: the original function signature if the target is a function, else None + """ + if self.logger.isEnabledFor(logging.INFO): + msg = "A {deprecated} {function} `{class_name}.{name}{signature}` was called.".format( + class_name=class_name, + name=name, + signature=_format_signature(signature), + function="function" if signature is not None else "property", + deprecated="deprecated" if is_deprecated else "missing", + ) + self.logger.info(msg) diff --git a/python/pyspark/pandas/utils.py b/python/pyspark/pandas/utils.py new file mode 100644 index 0000000000..002bac9404 --- /dev/null +++ b/python/pyspark/pandas/utils.py @@ -0,0 +1,878 @@ +# +# 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. +# +""" +Commonly used utils in Koalas. +""" + +import functools +from collections import OrderedDict +from contextlib import contextmanager +from distutils.version import LooseVersion +import os +from typing import Any, Callable, Dict, List, Optional, Tuple, Union, TYPE_CHECKING +import warnings + +import pyarrow +import pyspark +from pyspark import sql as spark +from pyspark.sql import functions as F +from pyspark.sql.types import DoubleType +import pandas as pd +from pandas.api.types import is_list_like + +# For running doctests and reference resolution in PyCharm. +from pyspark import pandas as pp # noqa: F401 +from pyspark.pandas.typedef.typehints import ( + as_spark_type, + extension_dtypes, + spark_type_to_pandas_dtype, +) + +if TYPE_CHECKING: + # This is required in old Python 3.5 to prevent circular reference. + from pyspark.pandas.base import IndexOpsMixin # noqa: F401 (SPARK-34943) + from pyspark.pandas.frame import DataFrame # noqa: F401 (SPARK-34943) + from pyspark.pandas.internal import InternalFrame # noqa: F401 (SPARK-34943) + + +ERROR_MESSAGE_CANNOT_COMBINE = ( + "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." +) + + +if LooseVersion(pyspark.__version__) < LooseVersion("3.0"): + SPARK_CONF_ARROW_ENABLED = "spark.sql.execution.arrow.enabled" +else: + SPARK_CONF_ARROW_ENABLED = "spark.sql.execution.arrow.pyspark.enabled" + + +def same_anchor( + this: Union["DataFrame", "IndexOpsMixin", "InternalFrame"], + that: Union["DataFrame", "IndexOpsMixin", "InternalFrame"], +) -> bool: + """ + Check if the anchors of the given DataFrame or Series are the same or not. + """ + from pyspark.pandas.base import IndexOpsMixin + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.internal import InternalFrame + + if isinstance(this, InternalFrame): + this_internal = this + else: + assert isinstance(this, (DataFrame, IndexOpsMixin)), type(this) + this_internal = this._internal + + if isinstance(that, InternalFrame): + that_internal = that + else: + assert isinstance(that, (DataFrame, IndexOpsMixin)), type(that) + that_internal = that._internal + + return ( + this_internal.spark_frame is that_internal.spark_frame + and this_internal.index_level == that_internal.index_level + and all( + this_scol._jc.equals(that_scol._jc) + for this_scol, that_scol in zip( + this_internal.index_spark_columns, that_internal.index_spark_columns + ) + ) + ) + + +def combine_frames(this, *args, how="full", preserve_order_column=False): + """ + This method combines `this` DataFrame with a different `that` DataFrame or + Series from a different DataFrame. + + It returns a DataFrame that has prefix `this_` and `that_` to distinct + the columns names from both DataFrames + + It internally performs a join operation which can be expensive in general. + So, if `compute.ops_on_diff_frames` option is False, + this method throws an exception. + """ + from pyspark.pandas.config import get_option + from pyspark.pandas.frame import DataFrame + from pyspark.pandas.internal import ( + InternalFrame, + HIDDEN_COLUMNS, + NATURAL_ORDER_COLUMN_NAME, + SPARK_INDEX_NAME_FORMAT, + ) + from pyspark.pandas.series import Series + + if all(isinstance(arg, Series) for arg in args): + assert all( + same_anchor(arg, args[0]) for arg in args + ), "Currently only one different DataFrame (from given Series) is supported" + assert not same_anchor(this, args[0]), "We don't need to combine. All series is in this." + that = args[0]._kdf[list(args)] + elif len(args) == 1 and isinstance(args[0], DataFrame): + assert isinstance(args[0], DataFrame) + assert not same_anchor( + this, args[0] + ), "We don't need to combine. `this` and `that` are same." + that = args[0] + else: + raise AssertionError("args should be single DataFrame or " "single/multiple Series") + + if get_option("compute.ops_on_diff_frames"): + + def resolve(internal, side): + rename = lambda col: "__{}_{}".format(side, col) + internal = internal.resolved_copy + sdf = internal.spark_frame + sdf = internal.spark_frame.select( + [ + scol_for(sdf, col).alias(rename(col)) + for col in sdf.columns + if col not in HIDDEN_COLUMNS + ] + + list(HIDDEN_COLUMNS) + ) + return internal.copy( + spark_frame=sdf, + index_spark_columns=[ + scol_for(sdf, rename(col)) for col in internal.index_spark_column_names + ], + data_spark_columns=[ + scol_for(sdf, rename(col)) for col in internal.data_spark_column_names + ], + ) + + this_internal = resolve(this._internal, "this") + that_internal = resolve(that._internal, "that") + + this_index_map = list( + zip( + this_internal.index_spark_column_names, + this_internal.index_names, + this_internal.index_dtypes, + ) + ) + that_index_map = list( + zip( + that_internal.index_spark_column_names, + that_internal.index_names, + that_internal.index_dtypes, + ) + ) + assert len(this_index_map) == len(that_index_map) + + join_scols = [] + merged_index_scols = [] + + # Note that the order of each element in index_map is guaranteed according to the index + # level. + this_and_that_index_map = list(zip(this_index_map, that_index_map)) + + this_sdf = this_internal.spark_frame.alias("this") + that_sdf = that_internal.spark_frame.alias("that") + + # If the same named index is found, that's used. + index_column_names = [] + index_use_extension_dtypes = [] + for ( + i, + ((this_column, this_name, this_dtype), (that_column, that_name, that_dtype)), + ) in enumerate(this_and_that_index_map): + if this_name == that_name: + # We should merge the Spark columns into one + # to mimic pandas' behavior. + this_scol = scol_for(this_sdf, this_column) + that_scol = scol_for(that_sdf, that_column) + join_scol = this_scol == that_scol + join_scols.append(join_scol) + + column_name = SPARK_INDEX_NAME_FORMAT(i) + index_column_names.append(column_name) + index_use_extension_dtypes.append( + any(isinstance(dtype, extension_dtypes) for dtype in [this_dtype, that_dtype]) + ) + merged_index_scols.append( + F.when(this_scol.isNotNull(), this_scol).otherwise(that_scol).alias(column_name) + ) + else: + raise ValueError("Index names must be exactly matched currently.") + + assert len(join_scols) > 0, "cannot join with no overlapping index names" + + joined_df = this_sdf.join(that_sdf, on=join_scols, how=how) + + if preserve_order_column: + order_column = [scol_for(this_sdf, NATURAL_ORDER_COLUMN_NAME)] + else: + order_column = [] + + joined_df = joined_df.select( + merged_index_scols + + [ + scol_for(this_sdf, this_internal.spark_column_name_for(label)) + for label in this_internal.column_labels + ] + + [ + scol_for(that_sdf, that_internal.spark_column_name_for(label)) + for label in that_internal.column_labels + ] + + order_column + ) + + index_spark_columns = [scol_for(joined_df, col) for col in index_column_names] + index_dtypes = [ + spark_type_to_pandas_dtype(field.dataType, use_extension_dtypes=use_extension_dtypes) + for field, use_extension_dtypes in zip( + joined_df.select(index_spark_columns).schema, index_use_extension_dtypes + ) + ] + + index_columns = set(index_column_names) + new_data_columns = [ + col + for col in joined_df.columns + if col not in index_columns and col != NATURAL_ORDER_COLUMN_NAME + ] + data_dtypes = this_internal.data_dtypes + that_internal.data_dtypes + + level = max(this_internal.column_labels_level, that_internal.column_labels_level) + + def fill_label(label): + if label is None: + return ([""] * (level - 1)) + [None] + else: + return ([""] * (level - len(label))) + list(label) + + column_labels = [ + tuple(["this"] + fill_label(label)) for label in this_internal.column_labels + ] + [tuple(["that"] + fill_label(label)) for label in that_internal.column_labels] + column_label_names = ( + [None] * (1 + level - this_internal.column_labels_level) + ) + this_internal.column_label_names + return DataFrame( + InternalFrame( + spark_frame=joined_df, + index_spark_columns=index_spark_columns, + index_names=this_internal.index_names, + index_dtypes=index_dtypes, + column_labels=column_labels, + data_spark_columns=[scol_for(joined_df, col) for col in new_data_columns], + data_dtypes=data_dtypes, + column_label_names=column_label_names, + ) + ) + else: + raise ValueError(ERROR_MESSAGE_CANNOT_COMBINE) + + +def align_diff_frames( + resolve_func, + this: "DataFrame", + that: "DataFrame", + fillna: bool = True, + how: str = "full", + preserve_order_column: bool = False, +) -> "DataFrame": + """ + This method aligns two different DataFrames with a given `func`. Columns are resolved and + handled within the given `func`. + To use this, `compute.ops_on_diff_frames` should be True, for now. + + :param resolve_func: Takes aligned (joined) DataFrame, the column of the current DataFrame, and + the column of another DataFrame. It returns an iterable that produces Series. + + >>> from pyspark.pandas.config import set_option, reset_option + >>> + >>> set_option("compute.ops_on_diff_frames", True) + >>> + >>> kdf1 = pp.DataFrame({'a': [9, 8, 7, 6, 5, 4, 3, 2, 1]}) + >>> kdf2 = pp.DataFrame({'a': [9, 8, 7, 6, 5, 4, 3, 2, 1]}) + >>> + >>> def func(kdf, this_column_labels, that_column_labels): + ... kdf # conceptually this is A + B. + ... + ... # Within this function, Series from A or B can be performed against `kdf`. + ... this_label = this_column_labels[0] # this is ('a',) from kdf1. + ... that_label = that_column_labels[0] # this is ('a',) from kdf2. + ... new_series = (kdf[this_label] - kdf[that_label]).rename(str(this_label)) + ... + ... # This new series will be placed in new DataFrame. + ... yield (new_series, this_label) + >>> + >>> + >>> align_diff_frames(func, kdf1, kdf2).sort_index() + a + 0 0 + 1 0 + 2 0 + 3 0 + 4 0 + 5 0 + 6 0 + 7 0 + 8 0 + >>> reset_option("compute.ops_on_diff_frames") + + :param this: a DataFrame to align + :param that: another DataFrame to align + :param fillna: If True, it fills missing values in non-common columns in both `this` and `that`. + Otherwise, it returns as are. + :param how: join way. In addition, it affects how `resolve_func` resolves the column conflict. + - full: `resolve_func` should resolve only common columns from 'this' and 'that' DataFrames. + For instance, if 'this' has columns A, B, C and that has B, C, D, `this_columns` and + 'that_columns' in this function are B, C and B, C. + - left: `resolve_func` should resolve columns including that columns. + For instance, if 'this' has columns A, B, C and that has B, C, D, `this_columns` is + B, C but `that_columns` are B, C, D. + - inner: Same as 'full' mode; however, internally performs inner join instead. + :return: Aligned DataFrame + """ + from pyspark.pandas.frame import DataFrame + + assert how == "full" or how == "left" or how == "inner" + + this_column_labels = this._internal.column_labels + that_column_labels = that._internal.column_labels + common_column_labels = set(this_column_labels).intersection(that_column_labels) + + # 1. Perform the join given two dataframes. + combined = combine_frames(this, that, how=how, preserve_order_column=preserve_order_column) + + # 2. Apply the given function to transform the columns in a batch and keep the new columns. + combined_column_labels = combined._internal.column_labels + + that_columns_to_apply = [] + this_columns_to_apply = [] + additional_that_columns = [] + columns_to_keep = [] + column_labels_to_keep = [] + + for combined_label in combined_column_labels: + for common_label in common_column_labels: + if combined_label == tuple(["this", *common_label]): + this_columns_to_apply.append(combined_label) + break + elif combined_label == tuple(["that", *common_label]): + that_columns_to_apply.append(combined_label) + break + else: + if how == "left" and combined_label in [ + tuple(["that", *label]) for label in that_column_labels + ]: + # In this case, we will drop `that_columns` in `columns_to_keep` but passes + # it later to `func`. `func` should resolve it. + # Note that adding this into a separate list (`additional_that_columns`) + # is intentional so that `this_columns` and `that_columns` can be paired. + additional_that_columns.append(combined_label) + elif fillna: + columns_to_keep.append(F.lit(None).cast(DoubleType()).alias(str(combined_label))) + column_labels_to_keep.append(combined_label) + else: + columns_to_keep.append(combined._kser_for(combined_label)) + column_labels_to_keep.append(combined_label) + + that_columns_to_apply += additional_that_columns + + # Should extract columns to apply and do it in a batch in case + # it adds new columns for example. + if len(this_columns_to_apply) > 0 or len(that_columns_to_apply) > 0: + kser_set, column_labels_applied = zip( + *resolve_func(combined, this_columns_to_apply, that_columns_to_apply) + ) + columns_applied = list(kser_set) + column_labels_applied = list(column_labels_applied) + else: + columns_applied = [] + column_labels_applied = [] + + applied = DataFrame( + combined._internal.with_new_columns( + columns_applied + columns_to_keep, + column_labels=column_labels_applied + column_labels_to_keep, + ) + ) # type: DataFrame + + # 3. Restore the names back and deduplicate columns. + this_labels = OrderedDict() + # Add columns in an order of its original frame. + for this_label in this_column_labels: + for new_label in applied._internal.column_labels: + if new_label[1:] not in this_labels and this_label == new_label[1:]: + this_labels[new_label[1:]] = new_label + + # After that, we will add the rest columns. + other_labels = OrderedDict() + for new_label in applied._internal.column_labels: + if new_label[1:] not in this_labels: + other_labels[new_label[1:]] = new_label + + kdf = applied[list(this_labels.values()) + list(other_labels.values())] + kdf.columns = kdf.columns.droplevel() + return kdf + + +def is_testing(): + """ Indicates whether Koalas is currently running tests. """ + return "KOALAS_TESTING" in os.environ + + +def default_session(conf=None): + if conf is None: + conf = dict() + should_use_legacy_ipc = False + if LooseVersion(pyarrow.__version__) >= LooseVersion("0.15") and LooseVersion( + pyspark.__version__ + ) < LooseVersion("3.0"): + conf["spark.executorEnv.ARROW_PRE_0_15_IPC_FORMAT"] = "1" + conf["spark.yarn.appMasterEnv.ARROW_PRE_0_15_IPC_FORMAT"] = "1" + conf["spark.mesos.driverEnv.ARROW_PRE_0_15_IPC_FORMAT"] = "1" + conf["spark.kubernetes.driverEnv.ARROW_PRE_0_15_IPC_FORMAT"] = "1" + should_use_legacy_ipc = True + + builder = spark.SparkSession.builder.appName("Koalas") + for key, value in conf.items(): + builder = builder.config(key, value) + # Currently, Koalas is dependent on such join due to 'compute.ops_on_diff_frames' + # configuration. This is needed with Spark 3.0+. + builder.config("spark.sql.analyzer.failAmbiguousSelfJoin", False) + + if LooseVersion(pyspark.__version__) >= LooseVersion("3.0.1") and is_testing(): + builder.config("spark.executor.allowSparkContext", False) + + session = builder.getOrCreate() + + if not should_use_legacy_ipc: + is_legacy_ipc_set = any( + v == "1" + for v in [ + session.conf.get("spark.executorEnv.ARROW_PRE_0_15_IPC_FORMAT", None), + session.conf.get("spark.yarn.appMasterEnv.ARROW_PRE_0_15_IPC_FORMAT", None), + session.conf.get("spark.mesos.driverEnv.ARROW_PRE_0_15_IPC_FORMAT", None), + session.conf.get("spark.kubernetes.driverEnv.ARROW_PRE_0_15_IPC_FORMAT", None), + ] + ) + if is_legacy_ipc_set: + raise RuntimeError( + "Please explicitly unset 'ARROW_PRE_0_15_IPC_FORMAT' environment variable in " + "both driver and executor sides. Check your spark.executorEnv.*, " + "spark.yarn.appMasterEnv.*, spark.mesos.driverEnv.* and " + "spark.kubernetes.driverEnv.* configurations. It is required to set this " + "environment variable only when you use pyarrow>=0.15 and pyspark<3.0." + ) + return session + + +@contextmanager +def sql_conf(pairs, *, spark=None): + """ + A convenient context manager to set `value` to the Spark SQL configuration `key` and + then restores it back when it exits. + """ + assert isinstance(pairs, dict), "pairs should be a dictionary." + + if spark is None: + spark = default_session() + + keys = pairs.keys() + new_values = pairs.values() + old_values = [spark.conf.get(key, None) for key in keys] + for key, new_value in zip(keys, new_values): + spark.conf.set(key, new_value) + try: + yield + finally: + for key, old_value in zip(keys, old_values): + if old_value is None: + spark.conf.unset(key) + else: + spark.conf.set(key, old_value) + + +def validate_arguments_and_invoke_function( + pobj: Union[pd.DataFrame, pd.Series], + koalas_func: Callable, + pandas_func: Callable, + input_args: Dict, +): + """ + Invokes a pandas function. + + This is created because different versions of pandas support different parameters, and as a + result when we code against the latest version, our users might get a confusing + "got an unexpected keyword argument" error if they are using an older version of pandas. + + This function validates all the arguments, removes the ones that are not supported if they + are simply the default value (i.e. most likely the user didn't explicitly specify it). It + throws a TypeError if the user explicitly specify an argument that is not supported by the + pandas version available. + + For example usage, look at DataFrame.to_html(). + + :param pobj: the pandas DataFrame or Series to operate on + :param koalas_func: Koalas function, used to get default parameter values + :param pandas_func: pandas function, used to check whether pandas supports all the arguments + :param input_args: arguments to pass to the pandas function, often created by using locals(). + Make sure locals() call is at the top of the function so it captures only + input parameters, rather than local variables. + :return: whatever pandas_func returns + """ + import inspect + + # Makes a copy since whatever passed in is likely created by locals(), and we can't delete + # 'self' key from that. + args = input_args.copy() + del args["self"] + + if "kwargs" in args: + # explode kwargs + kwargs = args["kwargs"] + del args["kwargs"] + args = {**args, **kwargs} + + koalas_params = inspect.signature(koalas_func).parameters + pandas_params = inspect.signature(pandas_func).parameters + + for param in koalas_params.values(): + if param.name not in pandas_params: + if args[param.name] == param.default: + del args[param.name] + else: + raise TypeError( + ( + "The pandas version [%s] available does not support parameter '%s' " + + "for function '%s'." + ) + % (pd.__version__, param.name, pandas_func.__name__) + ) + + args["self"] = pobj + return pandas_func(**args) + + +def lazy_property(fn): + """ + Decorator that makes a property lazy-evaluated. + + Copied from https://stevenloria.com/lazy-properties/ + """ + attr_name = "_lazy_" + fn.__name__ + + @property + @functools.wraps(fn) + def wrapped_lazy_property(self): + if not hasattr(self, attr_name): + setattr(self, attr_name, fn(self)) + return getattr(self, attr_name) + + def deleter(self): + if hasattr(self, attr_name): + delattr(self, attr_name) + + return wrapped_lazy_property.deleter(deleter) + + +def scol_for(sdf: spark.DataFrame, column_name: str) -> spark.Column: + """ Return Spark Column for the given column name. """ + return sdf["`{}`".format(column_name)] + + +def column_labels_level(column_labels: List[Tuple]) -> int: + """ Return the level of the column index. """ + if len(column_labels) == 0: + return 1 + else: + levels = set(1 if label is None else len(label) for label in column_labels) + assert len(levels) == 1, levels + return list(levels)[0] + + +def name_like_string(name: Optional[Union[str, Tuple]]) -> str: + """ + Return the name-like strings from str or tuple of str + + Examples + -------- + >>> name = 'abc' + >>> name_like_string(name) + 'abc' + + >>> name = ('abc',) + >>> name_like_string(name) + 'abc' + + >>> name = ('a', 'b', 'c') + >>> name_like_string(name) + '(a, b, c)' + """ + if name is None: + name = ("__none__",) + elif is_list_like(name): + name = tuple([str(n) for n in name]) + else: + name = (str(name),) + return ("(%s)" % ", ".join(name)) if len(name) > 1 else name[0] + + +def is_name_like_tuple(value: Any, allow_none: bool = True, check_type: bool = False) -> bool: + """ + Check the given tuple is be able to be used as a name. + + Examples + -------- + >>> is_name_like_tuple(('abc',)) + True + >>> is_name_like_tuple((1,)) + True + >>> is_name_like_tuple(('abc', 1, None)) + True + >>> is_name_like_tuple(('abc', 1, None), check_type=True) + True + >>> is_name_like_tuple((1.0j,)) + True + >>> is_name_like_tuple(tuple()) + False + >>> is_name_like_tuple((list('abc'),)) + False + >>> is_name_like_tuple(('abc', 1, None), allow_none=False) + False + >>> is_name_like_tuple((1.0j,), check_type=True) + False + """ + if value is None: + return allow_none + elif not isinstance(value, tuple): + return False + elif len(value) == 0: + return False + elif not allow_none and any(v is None for v in value): + return False + elif any(is_list_like(v) or isinstance(v, slice) for v in value): + return False + elif check_type: + return all( + v is None or as_spark_type(type(v), raise_error=False) is not None for v in value + ) + else: + return True + + +def is_name_like_value( + value: Any, allow_none: bool = True, allow_tuple: bool = True, check_type: bool = False +) -> bool: + """ + Check the given value is like a name. + + Examples + -------- + >>> is_name_like_value('abc') + True + >>> is_name_like_value(1) + True + >>> is_name_like_value(None) + True + >>> is_name_like_value(('abc',)) + True + >>> is_name_like_value(1.0j) + True + >>> is_name_like_value(list('abc')) + False + >>> is_name_like_value(None, allow_none=False) + False + >>> is_name_like_value(('abc',), allow_tuple=False) + False + >>> is_name_like_value(1.0j, check_type=True) + False + """ + if value is None: + return allow_none + elif isinstance(value, tuple): + return allow_tuple and is_name_like_tuple( + value, allow_none=allow_none, check_type=check_type + ) + elif is_list_like(value) or isinstance(value, slice): + return False + elif check_type: + return as_spark_type(type(value), raise_error=False) is not None + else: + return True + + +def validate_axis(axis=0, none_axis=0): + """ Check the given axis is valid. """ + # convert to numeric axis + axis = {None: none_axis, "index": 0, "columns": 1}.get(axis, axis) + if axis not in (none_axis, 0, 1): + raise ValueError("No axis named {0}".format(axis)) + return axis + + +def validate_bool_kwarg(value, arg_name): + """ Ensures that argument passed in arg_name is of type bool. """ + if not (isinstance(value, bool) or value is None): + raise ValueError( + 'For argument "{}" expected type bool, received ' + "type {}.".format(arg_name, type(value).__name__) + ) + return value + + +def validate_how(how: str) -> str: + """ Check the given how for join is valid. """ + if how == "full": + warnings.warn( + "Warning: While Koalas will accept 'full', you should use 'outer' " + + "instead to be compatible with the pandas merge API", + UserWarning, + ) + if how == "outer": + # 'outer' in pandas equals 'full' in Spark + how = "full" + if how not in ("inner", "left", "right", "full"): + raise ValueError( + "The 'how' parameter has to be amongst the following values: ", + "['inner', 'left', 'right', 'outer']", + ) + return how + + +def verify_temp_column_name( + df: Union["DataFrame", spark.DataFrame], column_name_or_label: Union[Any, Tuple] +) -> Union[Any, Tuple]: + """ + Verify that the given column name does not exist in the given Koalas or Spark DataFrame. + + The temporary column names should start and end with `__`. In addition, `column_name_or_label` + expects a single string, or column labels when `df` is a Koalas DataFrame. + + >>> kdf = pp.DataFrame({("x", "a"): ['a', 'b', 'c']}) + >>> kdf["__dummy__"] = 0 + >>> kdf[("", "__dummy__")] = 1 + >>> kdf # doctest: +NORMALIZE_WHITESPACE + x __dummy__ + a __dummy__ + 0 a 0 1 + 1 b 0 1 + 2 c 0 1 + + >>> verify_temp_column_name(kdf, '__tmp__') + ('__tmp__', '') + >>> verify_temp_column_name(kdf, ('', '__tmp__')) + ('', '__tmp__') + >>> verify_temp_column_name(kdf, '__dummy__') + Traceback (most recent call last): + ... + AssertionError: ... `(__dummy__, )` ... + >>> verify_temp_column_name(kdf, ('', '__dummy__')) + Traceback (most recent call last): + ... + AssertionError: ... `(, __dummy__)` ... + >>> verify_temp_column_name(kdf, 'dummy') + Traceback (most recent call last): + ... + AssertionError: ... should be empty or start and end with `__`: ('dummy', '') + >>> verify_temp_column_name(kdf, ('', 'dummy')) + Traceback (most recent call last): + ... + AssertionError: ... should be empty or start and end with `__`: ('', 'dummy') + + >>> internal = kdf._internal.resolved_copy + >>> sdf = internal.spark_frame + >>> sdf.select(internal.data_spark_columns).show() # doctest: +NORMALIZE_WHITESPACE + +------+---------+-------------+ + |(x, a)|__dummy__|(, __dummy__)| + +------+---------+-------------+ + | a| 0| 1| + | b| 0| 1| + | c| 0| 1| + +------+---------+-------------+ + + >>> verify_temp_column_name(sdf, '__tmp__') + '__tmp__' + >>> verify_temp_column_name(sdf, '__dummy__') + Traceback (most recent call last): + ... + AssertionError: ... `__dummy__` ... '(x, a)', '__dummy__', '(, __dummy__)', ... + >>> verify_temp_column_name(sdf, ('', '__dummy__')) + Traceback (most recent call last): + ... + AssertionError: + >>> verify_temp_column_name(sdf, 'dummy') + Traceback (most recent call last): + ... + AssertionError: ... should start and end with `__`: dummy + """ + from pyspark.pandas.frame import DataFrame + + if isinstance(df, DataFrame): + if isinstance(column_name_or_label, str): + column_name = column_name_or_label + + level = df._internal.column_labels_level + column_name_or_label = tuple([column_name_or_label] + ([""] * (level - 1))) + else: + column_name = name_like_string(column_name_or_label) + + assert any(len(label) > 0 for label in column_name_or_label) and all( + label == "" or (label.startswith("__") and label.endswith("__")) + for label in column_name_or_label + ), "The temporary column name should be empty or start and end with `__`: {}".format( + column_name_or_label + ) + assert all( + column_name_or_label != label for label in df._internal.column_labels + ), "The given column name `{}` already exists in the Koalas DataFrame: {}".format( + name_like_string(column_name_or_label), df.columns + ) + df = df._internal.resolved_copy.spark_frame + else: + assert isinstance(column_name_or_label, str), type(column_name_or_label) + assert column_name_or_label.startswith("__") and column_name_or_label.endswith( + "__" + ), "The temporary column name should start and end with `__`: {}".format( + column_name_or_label + ) + column_name = column_name_or_label + + assert isinstance(df, spark.DataFrame), type(df) + assert ( + column_name not in df.columns + ), "The given column name `{}` already exists in the Spark DataFrame: {}".format( + column_name, df.columns + ) + + return column_name_or_label + + +def compare_null_first(left, right, comp): + return (left.isNotNull() & right.isNotNull() & comp(left, right)) | ( + left.isNull() & right.isNotNull() + ) + + +def compare_null_last(left, right, comp): + return (left.isNotNull() & right.isNotNull() & comp(left, right)) | ( + left.isNotNull() & right.isNull() + ) + + +def compare_disallow_null(left, right, comp): + return left.isNotNull() & right.isNotNull() & comp(left, right) + + +def compare_allow_null(left, right, comp): + return left.isNull() | right.isNull() | comp(left, right) diff --git a/python/pyspark/pandas/version.py b/python/pyspark/pandas/version.py new file mode 100644 index 0000000000..449df1d633 --- /dev/null +++ b/python/pyspark/pandas/version.py @@ -0,0 +1,18 @@ +# +# 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. +# + +__version__ = "1.7.0" diff --git a/python/pyspark/pandas/window.py b/python/pyspark/pandas/window.py new file mode 100644 index 0000000000..c42f07447b --- /dev/null +++ b/python/pyspark/pandas/window.py @@ -0,0 +1,1739 @@ +# +# 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. +# +from functools import partial +from typing import Any, Union, TYPE_CHECKING + +from pyspark.sql import Window +from pyspark.sql import functions as F +from pyspark.pandas.missing.window import ( + MissingPandasLikeRolling, + MissingPandasLikeRollingGroupby, + MissingPandasLikeExpanding, + MissingPandasLikeExpandingGroupby, +) + +# For running doctests and reference resolution in PyCharm. +from pyspark import pandas as pp # noqa: F401 + +from pyspark.pandas.internal import NATURAL_ORDER_COLUMN_NAME, SPARK_INDEX_NAME_FORMAT +from pyspark.pandas.utils import scol_for + +if TYPE_CHECKING: + from pyspark.pandas.frame import DataFrame # noqa: F401 (SPARK-34943) + from pyspark.pandas.series import Series # noqa: F401 (SPARK-34943) + + +class RollingAndExpanding(object): + def __init__(self, kdf_or_kser, window, min_periods): + self._kdf_or_kser = kdf_or_kser + self._window = window + # This unbounded Window is later used to handle 'min_periods' for now. + self._unbounded_window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween( + Window.unboundedPreceding, Window.currentRow + ) + self._min_periods = min_periods + + def _apply_as_series_or_frame(self, func): + """ + Wraps a function that handles Spark column in order + to support it in both Koalas Series and DataFrame. + Note that the given `func` name should be same as the API's method name. + """ + raise NotImplementedError( + "A class that inherits this class should implement this method " + "to handle the index and columns of output." + ) + + def count(self) -> Union["Series", "DataFrame"]: + def count(scol): + return F.count(scol).over(self._window) + + return self._apply_as_series_or_frame(count).astype("float64") + + def sum(self) -> Union["Series", "DataFrame"]: + def sum(scol): + return F.when( + F.row_number().over(self._unbounded_window) >= self._min_periods, + F.sum(scol).over(self._window), + ).otherwise(F.lit(None)) + + return self._apply_as_series_or_frame(sum) + + def min(self) -> Union["Series", "DataFrame"]: + def min(scol): + return F.when( + F.row_number().over(self._unbounded_window) >= self._min_periods, + F.min(scol).over(self._window), + ).otherwise(F.lit(None)) + + return self._apply_as_series_or_frame(min) + + def max(self) -> Union["Series", "DataFrame"]: + def max(scol): + return F.when( + F.row_number().over(self._unbounded_window) >= self._min_periods, + F.max(scol).over(self._window), + ).otherwise(F.lit(None)) + + return self._apply_as_series_or_frame(max) + + def mean(self) -> Union["Series", "DataFrame"]: + def mean(scol): + return F.when( + F.row_number().over(self._unbounded_window) >= self._min_periods, + F.mean(scol).over(self._window), + ).otherwise(F.lit(None)) + + return self._apply_as_series_or_frame(mean) + + def std(self) -> Union["Series", "DataFrame"]: + def std(scol): + return F.when( + F.row_number().over(self._unbounded_window) >= self._min_periods, + F.stddev(scol).over(self._window), + ).otherwise(F.lit(None)) + + return self._apply_as_series_or_frame(std) + + def var(self) -> Union["Series", "DataFrame"]: + def var(scol): + return F.when( + F.row_number().over(self._unbounded_window) >= self._min_periods, + F.variance(scol).over(self._window), + ).otherwise(F.lit(None)) + + return self._apply_as_series_or_frame(var) + + +class Rolling(RollingAndExpanding): + def __init__(self, kdf_or_kser, window, min_periods=None): + from pyspark.pandas import DataFrame, Series + + if window < 0: + raise ValueError("window must be >= 0") + if (min_periods is not None) and (min_periods < 0): + raise ValueError("min_periods must be >= 0") + if min_periods is None: + # TODO: 'min_periods' is not equivalent in pandas because it does not count NA as + # a value. + min_periods = window + + if not isinstance(kdf_or_kser, (DataFrame, Series)): + raise TypeError( + "kdf_or_kser must be a series or dataframe; however, got: %s" % type(kdf_or_kser) + ) + + window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween( + Window.currentRow - (window - 1), Window.currentRow + ) + + super().__init__(kdf_or_kser, window, min_periods) + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeRolling, item): + property_or_func = getattr(MissingPandasLikeRolling, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError(item) + + def _apply_as_series_or_frame(self, func): + return self._kdf_or_kser._apply_series_op( + lambda kser: kser._with_new_scol(func(kser.spark.column)), # TODO: dtype? + should_resolve=True, + ) + + def count(self) -> Union["Series", "DataFrame"]: + """ + The rolling count of any non-NaN observations inside the window. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.count : Count of the full Series. + DataFrame.count : Count of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 3, float("nan"), 10]) + >>> s.rolling(1).count() + 0 1.0 + 1 1.0 + 2 0.0 + 3 1.0 + dtype: float64 + + >>> s.rolling(3).count() + 0 1.0 + 1 2.0 + 2 2.0 + 3 2.0 + dtype: float64 + + >>> s.to_frame().rolling(1).count() + 0 + 0 1.0 + 1 1.0 + 2 0.0 + 3 1.0 + + >>> s.to_frame().rolling(3).count() + 0 + 0 1.0 + 1 2.0 + 2 2.0 + 3 2.0 + """ + return super().count() + + def sum(self) -> Union["Series", "DataFrame"]: + """ + Calculate rolling summation of given DataFrame or Series. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Same type as the input, with the same index, containing the + rolling summation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.sum : Reducing sum for Series. + DataFrame.sum : Reducing sum for DataFrame. + + Examples + -------- + >>> s = pp.Series([4, 3, 5, 2, 6]) + >>> s + 0 4 + 1 3 + 2 5 + 3 2 + 4 6 + dtype: int64 + + >>> s.rolling(2).sum() + 0 NaN + 1 7.0 + 2 8.0 + 3 7.0 + 4 8.0 + dtype: float64 + + >>> s.rolling(3).sum() + 0 NaN + 1 NaN + 2 12.0 + 3 10.0 + 4 13.0 + dtype: float64 + + For DataFrame, each rolling summation is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df + A B + 0 4 16 + 1 3 9 + 2 5 25 + 3 2 4 + 4 6 36 + + >>> df.rolling(2).sum() + A B + 0 NaN NaN + 1 7.0 25.0 + 2 8.0 34.0 + 3 7.0 29.0 + 4 8.0 40.0 + + >>> df.rolling(3).sum() + A B + 0 NaN NaN + 1 NaN NaN + 2 12.0 50.0 + 3 10.0 38.0 + 4 13.0 65.0 + """ + return super().sum() + + def min(self) -> Union["Series", "DataFrame"]: + """ + Calculate the rolling minimum. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the rolling + calculation. + + See Also + -------- + Series.rolling : Calling object with a Series. + DataFrame.rolling : Calling object with a DataFrame. + Series.min : Similar method for Series. + DataFrame.min : Similar method for DataFrame. + + Examples + -------- + >>> s = pp.Series([4, 3, 5, 2, 6]) + >>> s + 0 4 + 1 3 + 2 5 + 3 2 + 4 6 + dtype: int64 + + >>> s.rolling(2).min() + 0 NaN + 1 3.0 + 2 3.0 + 3 2.0 + 4 2.0 + dtype: float64 + + >>> s.rolling(3).min() + 0 NaN + 1 NaN + 2 3.0 + 3 2.0 + 4 2.0 + dtype: float64 + + For DataFrame, each rolling minimum is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df + A B + 0 4 16 + 1 3 9 + 2 5 25 + 3 2 4 + 4 6 36 + + >>> df.rolling(2).min() + A B + 0 NaN NaN + 1 3.0 9.0 + 2 3.0 9.0 + 3 2.0 4.0 + 4 2.0 4.0 + + >>> df.rolling(3).min() + A B + 0 NaN NaN + 1 NaN NaN + 2 3.0 9.0 + 3 2.0 4.0 + 4 2.0 4.0 + """ + return super().min() + + def max(self) -> Union["Series", "DataFrame"]: + """ + Calculate the rolling maximum. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Return type is determined by the caller. + + See Also + -------- + Series.rolling : Series rolling. + DataFrame.rolling : DataFrame rolling. + Series.max : Similar method for Series. + DataFrame.max : Similar method for DataFrame. + + Examples + -------- + >>> s = pp.Series([4, 3, 5, 2, 6]) + >>> s + 0 4 + 1 3 + 2 5 + 3 2 + 4 6 + dtype: int64 + + >>> s.rolling(2).max() + 0 NaN + 1 4.0 + 2 5.0 + 3 5.0 + 4 6.0 + dtype: float64 + + >>> s.rolling(3).max() + 0 NaN + 1 NaN + 2 5.0 + 3 5.0 + 4 6.0 + dtype: float64 + + For DataFrame, each rolling maximum is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df + A B + 0 4 16 + 1 3 9 + 2 5 25 + 3 2 4 + 4 6 36 + + >>> df.rolling(2).max() + A B + 0 NaN NaN + 1 4.0 16.0 + 2 5.0 25.0 + 3 5.0 25.0 + 4 6.0 36.0 + + >>> df.rolling(3).max() + A B + 0 NaN NaN + 1 NaN NaN + 2 5.0 25.0 + 3 5.0 25.0 + 4 6.0 36.0 + """ + return super().max() + + def mean(self) -> Union["Series", "DataFrame"]: + """ + Calculate the rolling mean of the values. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the rolling + calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.mean : Equivalent method for Series. + DataFrame.mean : Equivalent method for DataFrame. + + Examples + -------- + >>> s = pp.Series([4, 3, 5, 2, 6]) + >>> s + 0 4 + 1 3 + 2 5 + 3 2 + 4 6 + dtype: int64 + + >>> s.rolling(2).mean() + 0 NaN + 1 3.5 + 2 4.0 + 3 3.5 + 4 4.0 + dtype: float64 + + >>> s.rolling(3).mean() + 0 NaN + 1 NaN + 2 4.000000 + 3 3.333333 + 4 4.333333 + dtype: float64 + + For DataFrame, each rolling mean is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df + A B + 0 4 16 + 1 3 9 + 2 5 25 + 3 2 4 + 4 6 36 + + >>> df.rolling(2).mean() + A B + 0 NaN NaN + 1 3.5 12.5 + 2 4.0 17.0 + 3 3.5 14.5 + 4 4.0 20.0 + + >>> df.rolling(3).mean() + A B + 0 NaN NaN + 1 NaN NaN + 2 4.000000 16.666667 + 3 3.333333 12.666667 + 4 4.333333 21.666667 + """ + return super().mean() + + def std(self) -> Union["Series", "DataFrame"]: + """ + Calculate rolling standard deviation. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the rolling calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.std : Equivalent method for Series. + DataFrame.std : Equivalent method for DataFrame. + numpy.std : Equivalent method for Numpy array. + + Examples + -------- + >>> s = pp.Series([5, 5, 6, 7, 5, 5, 5]) + >>> s.rolling(3).std() + 0 NaN + 1 NaN + 2 0.577350 + 3 1.000000 + 4 1.000000 + 5 1.154701 + 6 0.000000 + dtype: float64 + + For DataFrame, each rolling standard deviation is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.rolling(2).std() + A B + 0 NaN NaN + 1 0.000000 0.000000 + 2 0.707107 7.778175 + 3 0.707107 9.192388 + 4 1.414214 16.970563 + 5 0.000000 0.000000 + 6 0.000000 0.000000 + """ + return super().std() + + def var(self) -> Union["Series", "DataFrame"]: + """ + Calculate unbiased rolling variance. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the rolling calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.var : Equivalent method for Series. + DataFrame.var : Equivalent method for DataFrame. + numpy.var : Equivalent method for Numpy array. + + Examples + -------- + >>> s = pp.Series([5, 5, 6, 7, 5, 5, 5]) + >>> s.rolling(3).var() + 0 NaN + 1 NaN + 2 0.333333 + 3 1.000000 + 4 1.000000 + 5 1.333333 + 6 0.000000 + dtype: float64 + + For DataFrame, each unbiased rolling variance is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.rolling(2).var() + A B + 0 NaN NaN + 1 0.0 0.0 + 2 0.5 60.5 + 3 0.5 84.5 + 4 2.0 288.0 + 5 0.0 0.0 + 6 0.0 0.0 + """ + return super().var() + + +class RollingGroupby(Rolling): + def __init__(self, groupby, window, min_periods=None): + from pyspark.pandas.groupby import SeriesGroupBy + from pyspark.pandas.groupby import DataFrameGroupBy + + if isinstance(groupby, SeriesGroupBy): + kdf_or_kser = groupby._kser + elif isinstance(groupby, DataFrameGroupBy): + kdf_or_kser = groupby._kdf + else: + raise TypeError( + "groupby must be a SeriesGroupBy or DataFrameGroupBy; " + "however, got: %s" % type(groupby) + ) + + super().__init__(kdf_or_kser, window, min_periods) + + self._groupby = groupby + self._window = self._window.partitionBy(*[ser.spark.column for ser in groupby._groupkeys]) + self._unbounded_window = self._unbounded_window.partitionBy( + *[ser.spark.column for ser in groupby._groupkeys] + ) + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeRollingGroupby, item): + property_or_func = getattr(MissingPandasLikeRollingGroupby, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError(item) + + def _apply_as_series_or_frame(self, func): + """ + Wraps a function that handles Spark column in order + to support it in both Koalas Series and DataFrame. + Note that the given `func` name should be same as the API's method name. + """ + from pyspark.pandas import DataFrame + from pyspark.pandas.series import first_series + from pyspark.pandas.groupby import SeriesGroupBy + + groupby = self._groupby + kdf = groupby._kdf + + # Here we need to include grouped key as an index, and shift previous index. + # [index_column0, index_column1] -> [grouped key, index_column0, index_column1] + new_index_scols = [] + new_index_spark_column_names = [] + new_index_names = [] + new_index_dtypes = [] + for groupkey in groupby._groupkeys: + index_column_name = SPARK_INDEX_NAME_FORMAT(len(new_index_scols)) + new_index_scols.append(groupkey.spark.column.alias(index_column_name)) + new_index_spark_column_names.append(index_column_name) + new_index_names.append(groupkey._column_label) + new_index_dtypes.append(groupkey.dtype) + + for new_index_scol, index_name, index_dtype in zip( + kdf._internal.index_spark_columns, kdf._internal.index_names, kdf._internal.index_dtypes + ): + index_column_name = SPARK_INDEX_NAME_FORMAT(len(new_index_scols)) + new_index_scols.append(new_index_scol.alias(index_column_name)) + new_index_spark_column_names.append(index_column_name) + new_index_names.append(index_name) + new_index_dtypes.append(index_dtype) + + if groupby._agg_columns_selected: + agg_columns = groupby._agg_columns + else: + agg_columns = [ + kdf._kser_for(label) + for label in kdf._internal.column_labels + if label not in groupby._column_labels_to_exlcude + ] + + applied = [] + for agg_column in agg_columns: + applied.append(agg_column._with_new_scol(func(agg_column.spark.column))) # TODO: dtype? + + # Seems like pandas filters out when grouped key is NA. + cond = groupby._groupkeys[0].spark.column.isNotNull() + for c in groupby._groupkeys[1:]: + cond = cond | c.spark.column.isNotNull() + + sdf = kdf._internal.spark_frame.filter(cond).select( + new_index_scols + [c.spark.column for c in applied] + ) + + internal = kdf._internal.copy( + spark_frame=sdf, + index_spark_columns=[scol_for(sdf, col) for col in new_index_spark_column_names], + index_names=new_index_names, + index_dtypes=new_index_dtypes, + column_labels=[c._column_label for c in applied], + data_spark_columns=[ + scol_for(sdf, c._internal.data_spark_column_names[0]) for c in applied + ], + data_dtypes=[c.dtype for c in applied], + ) + + ret = DataFrame(internal) + if isinstance(groupby, SeriesGroupBy): + return first_series(ret) + else: + return ret + + def count(self) -> Union["Series", "DataFrame"]: + """ + The rolling count of any non-NaN observations inside the window. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the expanding + calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.count : Count of the full Series. + DataFrame.count : Count of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).rolling(3).count().sort_index() + 2 0 1.0 + 1 2.0 + 3 2 1.0 + 3 2.0 + 4 3.0 + 4 5 1.0 + 6 2.0 + 7 3.0 + 8 3.0 + 5 9 1.0 + 10 2.0 + dtype: float64 + + For DataFrame, each rolling count is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).rolling(2).count().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 1.0 1.0 + 1 2.0 2.0 + 3 2 1.0 1.0 + 3 2.0 2.0 + 4 2.0 2.0 + 4 5 1.0 1.0 + 6 2.0 2.0 + 7 2.0 2.0 + 8 2.0 2.0 + 5 9 1.0 1.0 + 10 2.0 2.0 + """ + return super().count() + + def sum(self) -> Union["Series", "DataFrame"]: + """ + The rolling summation of any non-NaN observations inside the window. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the rolling + calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.sum : Sum of the full Series. + DataFrame.sum : Sum of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).rolling(3).sum().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 9.0 + 4 5 NaN + 6 NaN + 7 12.0 + 8 12.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each rolling summation is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).rolling(2).sum().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 4.0 8.0 + 3 2 NaN NaN + 3 6.0 18.0 + 4 6.0 18.0 + 4 5 NaN NaN + 6 8.0 32.0 + 7 8.0 32.0 + 8 8.0 32.0 + 5 9 NaN NaN + 10 10.0 50.0 + """ + return super().sum() + + def min(self) -> Union["Series", "DataFrame"]: + """ + The rolling minimum of any non-NaN observations inside the window. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the rolling + calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.min : Min of the full Series. + DataFrame.min : Min of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).rolling(3).min().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 3.0 + 4 5 NaN + 6 NaN + 7 4.0 + 8 4.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each rolling minimum is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).rolling(2).min().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 2.0 4.0 + 3 2 NaN NaN + 3 3.0 9.0 + 4 3.0 9.0 + 4 5 NaN NaN + 6 4.0 16.0 + 7 4.0 16.0 + 8 4.0 16.0 + 5 9 NaN NaN + 10 5.0 25.0 + """ + return super().min() + + def max(self) -> Union["Series", "DataFrame"]: + """ + The rolling maximum of any non-NaN observations inside the window. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the rolling + calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.max : Max of the full Series. + DataFrame.max : Max of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).rolling(3).max().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 3.0 + 4 5 NaN + 6 NaN + 7 4.0 + 8 4.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each rolling maximum is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).rolling(2).max().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 2.0 4.0 + 3 2 NaN NaN + 3 3.0 9.0 + 4 3.0 9.0 + 4 5 NaN NaN + 6 4.0 16.0 + 7 4.0 16.0 + 8 4.0 16.0 + 5 9 NaN NaN + 10 5.0 25.0 + """ + return super().max() + + def mean(self) -> Union["Series", "DataFrame"]: + """ + The rolling mean of any non-NaN observations inside the window. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the rolling + calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.mean : Mean of the full Series. + DataFrame.mean : Mean of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).rolling(3).mean().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 3.0 + 4 5 NaN + 6 NaN + 7 4.0 + 8 4.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each rolling mean is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).rolling(2).mean().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 2.0 4.0 + 3 2 NaN NaN + 3 3.0 9.0 + 4 3.0 9.0 + 4 5 NaN NaN + 6 4.0 16.0 + 7 4.0 16.0 + 8 4.0 16.0 + 5 9 NaN NaN + 10 5.0 25.0 + """ + return super().mean() + + def std(self) -> Union["Series", "DataFrame"]: + """ + Calculate rolling standard deviation. + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the rolling calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.std : Equivalent method for Series. + DataFrame.std : Equivalent method for DataFrame. + numpy.std : Equivalent method for Numpy array. + """ + return super().std() + + def var(self) -> Union["Series", "DataFrame"]: + """ + Calculate unbiased rolling variance. + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the rolling calculation. + + See Also + -------- + Series.rolling : Calling object with Series data. + DataFrame.rolling : Calling object with DataFrames. + Series.var : Equivalent method for Series. + DataFrame.var : Equivalent method for DataFrame. + numpy.var : Equivalent method for Numpy array. + """ + return super().var() + + +class Expanding(RollingAndExpanding): + def __init__(self, kdf_or_kser, min_periods=1): + from pyspark.pandas import DataFrame, Series + + if min_periods < 0: + raise ValueError("min_periods must be >= 0") + + if not isinstance(kdf_or_kser, (DataFrame, Series)): + raise TypeError( + "kdf_or_kser must be a series or dataframe; however, got: %s" % type(kdf_or_kser) + ) + + window = Window.orderBy(NATURAL_ORDER_COLUMN_NAME).rowsBetween( + Window.unboundedPreceding, Window.currentRow + ) + + super().__init__(kdf_or_kser, window, min_periods) + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeExpanding, item): + property_or_func = getattr(MissingPandasLikeExpanding, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError(item) + + # TODO: when add 'center' and 'axis' parameter, should add to here too. + def __repr__(self): + return "Expanding [min_periods={}]".format(self._min_periods) + + _apply_as_series_or_frame = Rolling._apply_as_series_or_frame + + def count(self) -> Union["Series", "DataFrame"]: + """ + The expanding count of any non-NaN observations inside the window. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the expanding + calculation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.count : Count of the full Series. + DataFrame.count : Count of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 3, float("nan"), 10]) + >>> s.expanding().count() + 0 1.0 + 1 2.0 + 2 2.0 + 3 3.0 + dtype: float64 + + >>> s.to_frame().expanding().count() + 0 + 0 1.0 + 1 2.0 + 2 2.0 + 3 3.0 + """ + + def count(scol): + return F.when( + F.row_number().over(self._unbounded_window) >= self._min_periods, + F.count(scol).over(self._window), + ).otherwise(F.lit(None)) + + return self._apply_as_series_or_frame(count).astype("float64") # type: ignore + + def sum(self) -> Union["Series", "DataFrame"]: + """ + Calculate expanding summation of given DataFrame or Series. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Same type as the input, with the same index, containing the + expanding summation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.sum : Reducing sum for Series. + DataFrame.sum : Reducing sum for DataFrame. + + Examples + -------- + >>> s = pp.Series([1, 2, 3, 4, 5]) + >>> s + 0 1 + 1 2 + 2 3 + 3 4 + 4 5 + dtype: int64 + + >>> s.expanding(3).sum() + 0 NaN + 1 NaN + 2 6.0 + 3 10.0 + 4 15.0 + dtype: float64 + + For DataFrame, each expanding summation is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df + A B + 0 1 1 + 1 2 4 + 2 3 9 + 3 4 16 + 4 5 25 + + >>> df.expanding(3).sum() + A B + 0 NaN NaN + 1 NaN NaN + 2 6.0 14.0 + 3 10.0 30.0 + 4 15.0 55.0 + """ + return super().sum() + + def min(self) -> Union["Series", "DataFrame"]: + """ + Calculate the expanding minimum. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the expanding + calculation. + + See Also + -------- + Series.expanding : Calling object with a Series. + DataFrame.expanding : Calling object with a DataFrame. + Series.min : Similar method for Series. + DataFrame.min : Similar method for DataFrame. + + Examples + -------- + Performing a expanding minimum with a window size of 3. + + >>> s = pp.Series([4, 3, 5, 2, 6]) + >>> s.expanding(3).min() + 0 NaN + 1 NaN + 2 3.0 + 3 2.0 + 4 2.0 + dtype: float64 + """ + return super().min() + + def max(self) -> Union["Series", "DataFrame"]: + """ + Calculate the expanding maximum. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Return type is determined by the caller. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.max : Similar method for Series. + DataFrame.max : Similar method for DataFrame. + + Examples + -------- + Performing a expanding minimum with a window size of 3. + + >>> s = pp.Series([4, 3, 5, 2, 6]) + >>> s.expanding(3).max() + 0 NaN + 1 NaN + 2 5.0 + 3 5.0 + 4 6.0 + dtype: float64 + """ + return super().max() + + def mean(self) -> Union["Series", "DataFrame"]: + """ + Calculate the expanding mean of the values. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the expanding + calculation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.mean : Equivalent method for Series. + DataFrame.mean : Equivalent method for DataFrame. + + Examples + -------- + The below examples will show expanding mean calculations with window sizes of + two and three, respectively. + + >>> s = pp.Series([1, 2, 3, 4]) + >>> s.expanding(2).mean() + 0 NaN + 1 1.5 + 2 2.0 + 3 2.5 + dtype: float64 + + >>> s.expanding(3).mean() + 0 NaN + 1 NaN + 2 2.0 + 3 2.5 + dtype: float64 + """ + return super().mean() + + def std(self) -> Union["Series", "DataFrame"]: + """ + Calculate expanding standard deviation. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the expanding calculation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.std : Equivalent method for Series. + DataFrame.std : Equivalent method for DataFrame. + numpy.std : Equivalent method for Numpy array. + + Examples + -------- + >>> s = pp.Series([5, 5, 6, 7, 5, 5, 5]) + >>> s.expanding(3).std() + 0 NaN + 1 NaN + 2 0.577350 + 3 0.957427 + 4 0.894427 + 5 0.836660 + 6 0.786796 + dtype: float64 + + For DataFrame, each expanding standard deviation variance is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.expanding(2).std() + A B + 0 NaN NaN + 1 0.000000 0.000000 + 2 0.577350 6.350853 + 3 0.957427 11.412712 + 4 0.894427 10.630146 + 5 0.836660 9.928075 + 6 0.786796 9.327379 + """ + return super().std() + + def var(self) -> Union["Series", "DataFrame"]: + """ + Calculate unbiased expanding variance. + + .. note:: the current implementation of this API uses Spark's Window without + specifying partition specification. This leads to move all data into + single partition in single machine and could cause serious + performance degradation. Avoid this method against very large dataset. + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the expanding calculation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.var : Equivalent method for Series. + DataFrame.var : Equivalent method for DataFrame. + numpy.var : Equivalent method for Numpy array. + + Examples + -------- + >>> s = pp.Series([5, 5, 6, 7, 5, 5, 5]) + >>> s.expanding(3).var() + 0 NaN + 1 NaN + 2 0.333333 + 3 0.916667 + 4 0.800000 + 5 0.700000 + 6 0.619048 + dtype: float64 + + For DataFrame, each unbiased expanding variance is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.expanding(2).var() + A B + 0 NaN NaN + 1 0.000000 0.000000 + 2 0.333333 40.333333 + 3 0.916667 130.250000 + 4 0.800000 113.000000 + 5 0.700000 98.566667 + 6 0.619048 87.000000 + """ + return super().var() + + +class ExpandingGroupby(Expanding): + def __init__(self, groupby, min_periods=1): + from pyspark.pandas.groupby import SeriesGroupBy + from pyspark.pandas.groupby import DataFrameGroupBy + + if isinstance(groupby, SeriesGroupBy): + kdf_or_kser = groupby._kser + elif isinstance(groupby, DataFrameGroupBy): + kdf_or_kser = groupby._kdf + else: + raise TypeError( + "groupby must be a SeriesGroupBy or DataFrameGroupBy; " + "however, got: %s" % type(groupby) + ) + + super().__init__(kdf_or_kser, min_periods) + + self._groupby = groupby + self._window = self._window.partitionBy(*[ser.spark.column for ser in groupby._groupkeys]) + self._unbounded_window = self._window.partitionBy( + *[ser.spark.column for ser in groupby._groupkeys] + ) + + def __getattr__(self, item: str) -> Any: + if hasattr(MissingPandasLikeExpandingGroupby, item): + property_or_func = getattr(MissingPandasLikeExpandingGroupby, item) + if isinstance(property_or_func, property): + return property_or_func.fget(self) # type: ignore + else: + return partial(property_or_func, self) + raise AttributeError(item) + + _apply_as_series_or_frame = RollingGroupby._apply_as_series_or_frame # type: ignore + + def count(self) -> Union["Series", "DataFrame"]: + """ + The expanding count of any non-NaN observations inside the window. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the expanding + calculation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.count : Count of the full Series. + DataFrame.count : Count of the full DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).expanding(3).count().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 3.0 + 4 5 NaN + 6 NaN + 7 3.0 + 8 4.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each expanding count is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).expanding(2).count().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 2.0 2.0 + 3 2 NaN NaN + 3 2.0 2.0 + 4 3.0 3.0 + 4 5 NaN NaN + 6 2.0 2.0 + 7 3.0 3.0 + 8 4.0 4.0 + 5 9 NaN NaN + 10 2.0 2.0 + """ + return super().count() + + def sum(self) -> Union["Series", "DataFrame"]: + """ + Calculate expanding summation of given DataFrame or Series. + + Returns + ------- + Series or DataFrame + Same type as the input, with the same index, containing the + expanding summation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.sum : Reducing sum for Series. + DataFrame.sum : Reducing sum for DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).expanding(3).sum().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 9.0 + 4 5 NaN + 6 NaN + 7 12.0 + 8 16.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each expanding summation is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).expanding(2).sum().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 4.0 8.0 + 3 2 NaN NaN + 3 6.0 18.0 + 4 9.0 27.0 + 4 5 NaN NaN + 6 8.0 32.0 + 7 12.0 48.0 + 8 16.0 64.0 + 5 9 NaN NaN + 10 10.0 50.0 + """ + return super().sum() + + def min(self) -> Union["Series", "DataFrame"]: + """ + Calculate the expanding minimum. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the expanding + calculation. + + See Also + -------- + Series.expanding : Calling object with a Series. + DataFrame.expanding : Calling object with a DataFrame. + Series.min : Similar method for Series. + DataFrame.min : Similar method for DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).expanding(3).min().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 3.0 + 4 5 NaN + 6 NaN + 7 4.0 + 8 4.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each expanding minimum is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).expanding(2).min().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 2.0 4.0 + 3 2 NaN NaN + 3 3.0 9.0 + 4 3.0 9.0 + 4 5 NaN NaN + 6 4.0 16.0 + 7 4.0 16.0 + 8 4.0 16.0 + 5 9 NaN NaN + 10 5.0 25.0 + """ + return super().min() + + def max(self) -> Union["Series", "DataFrame"]: + """ + Calculate the expanding maximum. + + Returns + ------- + Series or DataFrame + Return type is determined by the caller. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.max : Similar method for Series. + DataFrame.max : Similar method for DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).expanding(3).max().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 3.0 + 4 5 NaN + 6 NaN + 7 4.0 + 8 4.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each expanding maximum is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).expanding(2).max().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 2.0 4.0 + 3 2 NaN NaN + 3 3.0 9.0 + 4 3.0 9.0 + 4 5 NaN NaN + 6 4.0 16.0 + 7 4.0 16.0 + 8 4.0 16.0 + 5 9 NaN NaN + 10 5.0 25.0 + """ + return super().max() + + def mean(self) -> Union["Series", "DataFrame"]: + """ + Calculate the expanding mean of the values. + + Returns + ------- + Series or DataFrame + Returned object type is determined by the caller of the expanding + calculation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.mean : Equivalent method for Series. + DataFrame.mean : Equivalent method for DataFrame. + + Examples + -------- + >>> s = pp.Series([2, 2, 3, 3, 3, 4, 4, 4, 4, 5, 5]) + >>> s.groupby(s).expanding(3).mean().sort_index() + 2 0 NaN + 1 NaN + 3 2 NaN + 3 NaN + 4 3.0 + 4 5 NaN + 6 NaN + 7 4.0 + 8 4.0 + 5 9 NaN + 10 NaN + dtype: float64 + + For DataFrame, each expanding mean is computed column-wise. + + >>> df = pp.DataFrame({"A": s.to_numpy(), "B": s.to_numpy() ** 2}) + >>> df.groupby(df.A).expanding(2).mean().sort_index() # doctest: +NORMALIZE_WHITESPACE + A B + A + 2 0 NaN NaN + 1 2.0 4.0 + 3 2 NaN NaN + 3 3.0 9.0 + 4 3.0 9.0 + 4 5 NaN NaN + 6 4.0 16.0 + 7 4.0 16.0 + 8 4.0 16.0 + 5 9 NaN NaN + 10 5.0 25.0 + """ + return super().mean() + + def std(self) -> Union["Series", "DataFrame"]: + """ + Calculate expanding standard deviation. + + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the expanding calculation. + + See Also + -------- + Series.expanding: Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.std : Equivalent method for Series. + DataFrame.std : Equivalent method for DataFrame. + numpy.std : Equivalent method for Numpy array. + """ + return super().std() + + def var(self) -> Union["Series", "DataFrame"]: + """ + Calculate unbiased expanding variance. + + Returns + ------- + Series or DataFrame + Returns the same object type as the caller of the expanding calculation. + + See Also + -------- + Series.expanding : Calling object with Series data. + DataFrame.expanding : Calling object with DataFrames. + Series.var : Equivalent method for Series. + DataFrame.var : Equivalent method for DataFrame. + numpy.var : Equivalent method for Numpy array. + """ + return super().var()