You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by gu...@apache.org on 2021/04/06 03:43:18 UTC

[spark] branch master updated: [SPARK-34890][PYTHON] Port/integrate Koalas main codes into PySpark

This is an automated email from the ASF dual-hosted git repository.

gurwls223 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new caf04f9  [SPARK-34890][PYTHON] Port/integrate Koalas main codes into PySpark
caf04f9 is described below

commit caf04f9b77b3b215963936231fb11027dee57d6c
Author: itholic <ha...@databricks.com>
AuthorDate: Tue Apr 6 12:42:39 2021 +0900

    [SPARK-34890][PYTHON] Port/integrate Koalas main codes into PySpark
    
    ### What changes were proposed in this pull request?
    
    As a first step of [SPARK-34849](https://issues.apache.org/jira/browse/SPARK-34849), this PR proposes porting the Koalas main code into PySpark.
    
    This PR contains minimal changes to the existing Koalas code as follows:
    1. `databricks.koalas` -> `pyspark.pandas`
    2. `from databricks import koalas as ks` -> `from pyspark import pandas as pp`
    3. `ks.xxx -> pp.xxx`
    
    Other than them:
    1. Added a line to `python/mypy.ini` in order to ignore the mypy test. See related issue at [SPARK-34941](https://issues.apache.org/jira/browse/SPARK-34941).
    2. Added a comment to several lines in several files to ignore the flake8 F401. See related issue at [SPARK-34943](https://issues.apache.org/jira/browse/SPARK-34943).
    
    When this PR is merged, all the features that were previously used in [Koalas](https://github.com/databricks/koalas) will be available in PySpark as well.
    
    Users can access to the pandas API in PySpark as below:
    
    ```python
    >>> from pyspark import pandas as pp
    >>> ppdf = pp.DataFrame({"A": [1, 2, 3], "B": [15, 20, 25]})
    >>> ppdf
       A   B
    0  1  15
    1  2  20
    2  3  25
    ```
    
    The existing "options and settings" in Koalas are also available in the same way:
    
    ```python
    >>> from pyspark.pandas.config import set_option, reset_option, get_option
    >>> ppser1 = pp.Series([1, 2, 3])
    >>> ppser2 = pp.Series([3, 4, 5])
    >>> ppser1 + ppser2
    Traceback (most recent call last):
    ...
    ValueError: Cannot combine the series or dataframe because it comes from a different dataframe. In order to allow this operation, enable 'compute.ops_on_diff_frames' option.
    
    >>> set_option("compute.ops_on_diff_frames", True)
    >>> ppser1 + ppser2
    0    4
    1    6
    2    8
    dtype: int64
    ```
    
    Please also refer to the [API Reference](https://koalas.readthedocs.io/en/latest/reference/index.html) and [Options and Settings](https://koalas.readthedocs.io/en/latest/user_guide/options.html) for more detail.
    
    **NOTE** that this PR intentionally ports the main codes of Koalas first almost as are with minimal changes because:
    - Koalas project is fairly large. Making some changes together for PySpark will make it difficult to review the individual change.
        Koalas dev includes multiple Spark committers who will review. By doing this, the committers will be able to more easily and effectively review and drive the development.
    - Koalas tests and documentation require major changes to make it look great together with PySpark whereas main codes do not require.
    - We lately froze the Koalas codebase, and plan to work together on the initial porting. By porting the main codes first as are, it unblocks the Koalas dev to work on other items in parallel.
    
    I promise and will make sure on:
    - Rename Koalas to PySpark pandas APIs and/or pandas-on-Spark accordingly in documentation, and the docstrings and comments in the main codes.
    - Triage APIs to remove that don’t make sense when Koalas is in PySpark
    
    The documentation changes will be tracked in [SPARK-34885](https://issues.apache.org/jira/browse/SPARK-34885), the test code changes will be tracked in [SPARK-34886](https://issues.apache.org/jira/browse/SPARK-34886).
    
    ### Why are the changes needed?
    
    Please refer to:
    - [[DISCUSS] Support pandas API layer on PySpark](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Support-pandas-API-layer-on-PySpark-td30945.html)
    - [[VOTE] SPIP: Support pandas API layer on PySpark](http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-SPIP-Support-pandas-API-layer-on-PySpark-td30996.html)
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, now users can use the pandas APIs on Spark
    
    ### How was this patch tested?
    
    Manually tested for exposed major APIs and options as described above.
    
    ### Koalas contributors
    
    Koalas would not have been possible without the following contributors:
    
    ueshin
    HyukjinKwon
    rxin
    xinrong-databricks
    RainFung
    charlesdong1991
    harupy
    floscha
    beobest2
    thunterdb
    garawalid
    LucasG0
    shril
    deepyaman
    gioa
    fwani
    90jam
    thoo
    AbdealiJK
    abishekganesh72
    gliptak
    DumbMachine
    dvgodoy
    stbof
    nitlev
    hjoo
    gatorsmile
    tomspur
    icexelloss
    awdavidson
    guyao
    akhilputhiry
    scook12
    patryk-oleniuk
    tracek
    dennyglee
    athena15
    gstaubli
    WeichenXu123
    hsubbaraj
    lfdversluis
    ktksq
    shengjh
    margaret-databricks
    LSturtew
    sllynn
    manuzhang
    jijosg
    sadikovi
    
    Closes #32036 from itholic/SPARK-34890.
    
    Authored-by: itholic <ha...@databricks.com>
    Signed-off-by: HyukjinKwon <gu...@apache.org>
---
 dev/lint-python                                    |     1 +
 python/mypy.ini                                    |     4 +
 python/pyspark/pandas/__init__.py                  |   209 +
 python/pyspark/pandas/accessors.py                 |   930 ++
 python/pyspark/pandas/base.py                      |  1995 +++
 python/pyspark/pandas/categorical.py               |   164 +
 python/pyspark/pandas/config.py                    |   442 +
 python/pyspark/pandas/datetimes.py                 |   850 ++
 python/pyspark/pandas/exceptions.py                |   106 +
 python/pyspark/pandas/extensions.py                |   342 +
 python/pyspark/pandas/frame.py                     | 11976 +++++++++++++++++++
 python/pyspark/pandas/generic.py                   |  3102 +++++
 python/pyspark/pandas/groupby.py                   |  3186 +++++
 python/pyspark/pandas/indexes/__init__.py          |    20 +
 python/pyspark/pandas/indexes/base.py              |  2471 ++++
 python/pyspark/pandas/indexes/category.py          |   188 +
 python/pyspark/pandas/indexes/datetimes.py         |   742 ++
 python/pyspark/pandas/indexes/multi.py             |  1170 ++
 python/pyspark/pandas/indexes/numeric.py           |   147 +
 python/pyspark/pandas/indexing.py                  |  1708 +++
 python/pyspark/pandas/internal.py                  |  1438 +++
 python/pyspark/pandas/missing/__init__.py          |    48 +
 python/pyspark/pandas/missing/common.py            |    59 +
 python/pyspark/pandas/missing/frame.py             |    98 +
 python/pyspark/pandas/missing/groupby.py           |   103 +
 python/pyspark/pandas/missing/indexes.py           |   218 +
 python/pyspark/pandas/missing/series.py            |   125 +
 python/pyspark/pandas/missing/window.py            |   126 +
 python/pyspark/pandas/ml.py                        |    91 +
 python/pyspark/pandas/mlflow.py                    |   192 +
 python/pyspark/pandas/namespace.py                 |  2877 +++++
 python/pyspark/pandas/numpy_compat.py              |   210 +
 python/pyspark/pandas/plot/__init__.py             |    17 +
 python/pyspark/pandas/plot/core.py                 |  1134 ++
 python/pyspark/pandas/plot/matplotlib.py           |   897 ++
 python/pyspark/pandas/plot/plotly.py               |   212 +
 python/pyspark/pandas/series.py                    |  6235 ++++++++++
 python/pyspark/pandas/spark/__init__.py            |    16 +
 python/pyspark/pandas/spark/accessors.py           |  1249 ++
 python/pyspark/pandas/spark/functions.py           |    98 +
 python/pyspark/pandas/spark/utils.py               |   124 +
 python/pyspark/pandas/sql.py                       |   302 +
 python/pyspark/pandas/strings.py                   |  2289 ++++
 python/pyspark/pandas/typedef/__init__.py          |    18 +
 python/pyspark/pandas/typedef/string_typehints.py  |    37 +
 python/pyspark/pandas/typedef/typehints.py         |   521 +
 python/pyspark/pandas/usage_logging/__init__.py    |   269 +
 .../pyspark/pandas/usage_logging/usage_logger.py   |   132 +
 python/pyspark/pandas/utils.py                     |   878 ++
 python/pyspark/pandas/version.py                   |    18 +
 python/pyspark/pandas/window.py                    |  1739 +++
 51 files changed, 51523 insertions(+)

diff --git a/dev/lint-python b/dev/lint-python
index 24923c1..320e24f 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 ad4fcf7..5bb4bb8 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 0000000..a605954
--- /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 "<unknown version>"
+                )
+            )
+
+
+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 0000000..39a647c0
--- /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
+        <https://koalas.readthedocs.io/en/latest/user_guide/options.html#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
+        <https://koalas.readthedocs.io/en/latest/user_guide/transform_apply.html>`_.
+
+        .. 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
+            <https://koalas.readthedocs.io/en/latest/user_guide/options.html#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
+        <https://koalas.readthedocs.io/en/latest/user_guide/transform_apply.html>`_.
+
+        .. 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
+            <https://koalas.readthedocs.io/en/latest/user_guide/options.html#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
+        <https://koalas.readthedocs.io/en/latest/user_guide/transform_apply.html>`_.
+
+        .. 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 0000000..6ae27a8
--- /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('<M8[ns]')
+
+        >>> s.rename("a").to_frame().set_index("a").index.dtype
+        dtype('<M8[ns]')
+        """
+        return self._internal.data_dtypes[0]
+
+    @property
+    def empty(self) -> 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 0000000..87d56d0
--- /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 0000000..7f011a2
--- /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 <class 'str'>;
+    however, expected types are [(<class 'float'>, <class 'int'>)].
+
+    >>> option.validate(-1.1)
+    Traceback (most recent call last):
+      ...
+    ValueError: should be a positive float
+
+    >>> option.validate(1.1)
+    """
+
+    def __init__(
+        self,
+        *,
+        key: str,
+        doc: str,
+        default: Any,
+        types: Union[Tuple[type, ...], type] = str,
+        check_func: Tuple[Callable[[Any], bool], str] = (lambda v: True, "")
+    ):
+        self.key = key
+        self.doc = doc
+        self.default = default
+        self.types = types
+        self.check_func = check_func
+
+    def validate(self, v: Any) -> None:
+        """
+        Validate the given value and throw an exception with related information such as key.
+        """
+        if not isinstance(v, self.types):
+            raise ValueError(
+                "The value for option '%s' was %s; however, expected types are "
+                "[%s]." % (self.key, type(v), str(self.types))
+            )
+        if not self.check_func[0](v):
+            raise ValueError(self.check_func[1])
+
+
+# Available options.
+#
+# NOTE: if you are fixing or adding an option here, make sure you execute `show_options()` and
+#     copy & paste the results into show_options 'docs/source/user_guide/options.rst' as well.
+#     See the examples below:
+#     >>> from pyspark.pandas.config import show_options
+#     >>> show_options()
+_options = [
+    Option(
+        key="display.max_rows",
+        doc=(
+            "This sets the maximum number of rows Koalas should output when printing out "
+            "various output. For example, this value determines the number of rows to be "
+            "shown at the repr() in a dataframe. Set `None` to unlimit the input length. "
+            "Default is 1000."
+        ),
+        default=1000,
+        types=(int, type(None)),
+        check_func=(
+            lambda v: v is None or v >= 0,
+            "'display.max_rows' should be greater than or equal to 0.",
+        ),
+    ),
+    Option(
+        key="compute.max_rows",
+        doc=(
+            "'compute.max_rows' sets the limit of the current Koalas DataFrame. Set `None` to "
+            "unlimit the input length. When the limit is set, it is executed by the shortcut by "
+            "collecting the data into the driver, and then using the pandas API. If the limit is "
+            "unset, the operation is executed by PySpark. Default is 1000."
+        ),
+        default=1000,
+        types=(int, type(None)),
+        check_func=(
+            lambda v: v is None or v >= 0,
+            "'compute.max_rows' should be greater than or equal to 0.",
+        ),
+    ),
+    Option(
+        key="compute.shortcut_limit",
+        doc=(
+            "'compute.shortcut_limit' sets the limit for a shortcut. "
+            "It computes specified number of rows and use its schema. When the dataframe "
+            "length is larger than this limit, Koalas uses PySpark to compute."
+        ),
+        default=1000,
+        types=int,
+        check_func=(
+            lambda v: v >= 0,
+            "'compute.shortcut_limit' should be greater than or equal to 0.",
+        ),
+    ),
+    Option(
+        key="compute.ops_on_diff_frames",
+        doc=(
+            "This determines whether or not to operate between two different dataframes. "
+            "For example, 'combine_frames' function internally performs a join operation which "
+            "can be expensive in general. So, if `compute.ops_on_diff_frames` variable is not "
+            "True, that method throws an exception."
+        ),
+        default=False,
+        types=bool,
+    ),
+    Option(
+        key="compute.default_index_type",
+        doc=("This sets the default index type: sequence, distributed and distributed-sequence."),
+        default="sequence",
+        types=str,
+        check_func=(
+            lambda v: v in ("sequence", "distributed", "distributed-sequence"),
+            "Index type should be one of 'sequence', 'distributed', 'distributed-sequence'.",
+        ),
+    ),
+    Option(
+        key="compute.ordered_head",
+        doc=(
+            "'compute.ordered_head' sets whether or not to operate head with natural ordering. "
+            "Koalas does not guarantee the row ordering so `head` could return some rows from "
+            "distributed partitions. If 'compute.ordered_head' is set to True, Koalas performs "
+            "natural ordering beforehand, but it will cause a performance overhead."
+        ),
+        default=False,
+        types=bool,
+    ),
+    Option(
+        key="plotting.max_rows",
+        doc=(
+            "'plotting.max_rows' sets the visual limit on top-n-based plots such as `plot.bar` "
+            "and `plot.pie`. If it is set to 1000, the first 1000 data points will be used "
+            "for plotting. Default is 1000."
+        ),
+        default=1000,
+        types=int,
+        check_func=(
+            lambda v: v is v >= 0,
+            "'plotting.max_rows' should be greater than or equal to 0.",
+        ),
+    ),
+    Option(
+        key="plotting.sample_ratio",
+        doc=(
+            "'plotting.sample_ratio' sets the proportion of data that will be plotted for sample-"
+            "based plots such as `plot.line` and `plot.area`. "
+            "This option defaults to 'plotting.max_rows' option."
+        ),
+        default=None,
+        types=(float, type(None)),
+        check_func=(
+            lambda v: v is None or 1 >= v >= 0,
+            "'plotting.sample_ratio' should be 1.0 >= value >= 0.0.",
+        ),
+    ),
+    Option(
+        key="plotting.backend",
+        doc=(
+            "Backend to use for plotting. Default is plotly. "
+            "Supports any package that has a top-level `.plot` method. "
+            "Known options are: [matplotlib, plotly]."
+        ),
+        default="plotly",
+        types=str,
+    ),
+]  # type: List[Option]
+
+_options_dict = dict(zip((option.key for option in _options), _options))  # type: Dict[str, Option]
+
+_key_format = "koalas.{}".format
+
+
+class OptionError(AttributeError, KeyError):
+    pass
+
+
+def show_options():
+    """
+    Make a pretty table that can be copied and pasted into public documentation.
+    This is currently for an internal purpose.
+
+    Examples
+    --------
+    >>> show_options()  # doctest: +ELLIPSIS, +NORMALIZE_WHITESPACE
+    ================... =======... =====================...
+    Option              Default    Description
+    ================... =======... =====================...
+    display.max_rows    1000       This sets the maximum...
+    ...
+    ================... =======... =====================...
+    """
+
+    import textwrap
+
+    header = ["Option", "Default", "Description"]
+    row_format = "{:<31} {:<14} {:<53}"
+
+    print(row_format.format("=" * 31, "=" * 14, "=" * 53))
+    print(row_format.format(*header))
+    print(row_format.format("=" * 31, "=" * 14, "=" * 53))
+
+    for option in _options:
+        doc = textwrap.fill(option.doc, 53)
+        formatted = "".join([line + "\n" + (" " * 47) for line in doc.split("\n")]).rstrip()
+        print(row_format.format(option.key, repr(option.default), formatted))
+
+    print(row_format.format("=" * 31, "=" * 14, "=" * 53))
+
+
+def get_option(key: str, default: Union[Any, _NoValueType] = _NoValue) -> Any:
+    """
+    Retrieves the value of the specified option.
+
+    Parameters
+    ----------
+    key : str
+        The key which should match a single option.
+    default : object
+        The default value if the option is not set yet. The value should be JSON serializable.
+
+    Returns
+    -------
+    result : the value of the option
+
+    Raises
+    ------
+    OptionError : if no such option exists and the default is not provided
+    """
+    _check_option(key)
+    if default is _NoValue:
+        default = _options_dict[key].default
+    _options_dict[key].validate(default)
+
+    return json.loads(default_session().conf.get(_key_format(key), default=json.dumps(default)))
+
+
+def set_option(key: str, value: Any) -> None:
+    """
+    Sets the value of the specified option.
+
+    Parameters
+    ----------
+    key : str
+        The key which should match a single option.
+    value : object
+        New value of option. The value should be JSON serializable.
+
+    Returns
+    -------
+    None
+    """
+    _check_option(key)
+    _options_dict[key].validate(value)
+
+    default_session().conf.set(_key_format(key), json.dumps(value))
+
+
+def reset_option(key: str) -> None:
+    """
+    Reset one option to their default value.
+
+    Pass "all" as argument to reset all options.
+
+    Parameters
+    ----------
+    key : str
+        If specified only option will be reset.
+
+    Returns
+    -------
+    None
+    """
+    _check_option(key)
+    default_session().conf.unset(_key_format(key))
+
+
+@contextmanager
+def option_context(*args):
+    """
+    Context manager to temporarily set options in the `with` statement context.
+
+    You need to invoke as ``option_context(pat, val, [(pat, val), ...])``.
+
+    Examples
+    --------
+    >>> with option_context('display.max_rows', 10, 'compute.max_rows', 5):
+    ...     print(get_option('display.max_rows'), get_option('compute.max_rows'))
+    10 5
+    >>> print(get_option('display.max_rows'), get_option('compute.max_rows'))
+    1000 1000
+    """
+    if len(args) == 0 or len(args) % 2 != 0:
+        raise ValueError("Need to invoke as option_context(pat, val, [(pat, val), ...]).")
+    opts = dict(zip(args[::2], args[1::2]))
+    orig_opts = {key: get_option(key) for key in opts}
+    try:
+        for key, value in opts.items():
+            set_option(key, value)
+        yield
+    finally:
+        for key, value in orig_opts.items():
+            set_option(key, value)
+
+
+def _check_option(key: str) -> None:
+    if key not in _options_dict:
+        raise OptionError(
+            "No such option: '{}'. Available options are [{}]".format(
+                key, ", ".join(list(_options_dict.keys()))
+            )
+        )
+
+
+class DictWrapper:
+    """ provide attribute-style access to a nested dict"""
+
+    def __init__(self, d, prefix=""):
+        object.__setattr__(self, "d", d)
+        object.__setattr__(self, "prefix", prefix)
+
+    def __setattr__(self, key, val):
+        prefix = object.__getattribute__(self, "prefix")
+        d = object.__getattribute__(self, "d")
+        if prefix:
+            prefix += "."
+        canonical_key = prefix + key
+
+        candidates = [
+            k for k in d.keys() if all(x in k.split(".") for x in canonical_key.split("."))
+        ]
+        if len(candidates) == 1 and candidates[0] == canonical_key:
+            return set_option(canonical_key, val)
+        else:
+            raise OptionError(
+                "No such option: '{}'. Available options are [{}]".format(
+                    key, ", ".join(list(_options_dict.keys()))
+                )
+            )
+
+    def __getattr__(self, key):
+        prefix = object.__getattribute__(self, "prefix")
+        d = object.__getattribute__(self, "d")
+        if prefix:
+            prefix += "."
+        canonical_key = prefix + key
+
+        candidates = [
+            k for k in d.keys() if all(x in k.split(".") for x in canonical_key.split("."))
+        ]
+        if len(candidates) == 1 and candidates[0] == canonical_key:
+            return get_option(canonical_key)
+        elif len(candidates) == 0:
+            raise OptionError(
+                "No such option: '{}'. Available options are [{}]".format(
+                    key, ", ".join(list(_options_dict.keys()))
+                )
+            )
+        else:
+            return DictWrapper(d, canonical_key)
+
+    def __dir__(self):
+        prefix = object.__getattribute__(self, "prefix")
+        d = object.__getattribute__(self, "d")
+
+        if prefix == "":
+            candidates = d.keys()
+            offset = 0
+        else:
+            candidates = [k for k in d.keys() if all(x in k.split(".") for x in prefix.split("."))]
+            offset = len(prefix) + 1  # prefix (e.g. "compute.") to trim.
+        return [c[offset:] for c in candidates]
+
+
+options = DictWrapper(_options_dict)
diff --git a/python/pyspark/pandas/datetimes.py b/python/pyspark/pandas/datetimes.py
new file mode 100644
index 0000000..a5d3f38
--- /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 0000000..7ec874b
--- /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 0000000..9b67f2a
--- /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 0000000..4e7b9fa
--- /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<rows>[0-9]+) rows x (?P<columns>[0-9]+) columns\]$")
+REPR_HTML_PATTERN = re.compile(
+    r"\n\<p\>(?P<rows>[0-9]+) rows × (?P<columns>[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
+            `<table>` tag. Default ``pd.options.html.border``.
+        table_id : str, optional
+            A css id is included in the opening `<table>` 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(<class 'list'>, {'col..., 'col...}), \
+defaultdict(<class 'list'>, {'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}
+        <BLANKLINE>
+        """
+
+        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
+        <https://koalas.readthedocs.io/en/latest/user_guide/transform_apply.html>`_.
+
+        .. 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
+            <https://koalas.readthedocs.io/en/latest/user_guide/options.html#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
+        <https://koalas.readthedocs.io/en/latest/user_guide/transform_apply.html>`_.
+
+        .. 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
+        <pandas.io.formats.style.Styler object at ...>
+        """
+        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'])
... 42032 lines suppressed ...

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org