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()