spark-instrumented-optimizer/python/pyspark/pandas/__init__.py

210 lines
6.9 KiB
Python
Raw Normal View History

[SPARK-34890][PYTHON] Port/integrate Koalas main codes into PySpark ### What changes were proposed in this pull request? As a first step of [SPARK-34849](https://issues.apache.org/jira/browse/SPARK-34849), this PR proposes porting the Koalas main code into PySpark. This PR contains minimal changes to the existing Koalas code as follows: 1. `databricks.koalas` -> `pyspark.pandas` 2. `from databricks import koalas as ks` -> `from pyspark import pandas as pp` 3. `ks.xxx -> pp.xxx` Other than them: 1. Added a line to `python/mypy.ini` in order to ignore the mypy test. See related issue at [SPARK-34941](https://issues.apache.org/jira/browse/SPARK-34941). 2. Added a comment to several lines in several files to ignore the flake8 F401. See related issue at [SPARK-34943](https://issues.apache.org/jira/browse/SPARK-34943). When this PR is merged, all the features that were previously used in [Koalas](https://github.com/databricks/koalas) will be available in PySpark as well. Users can access to the pandas API in PySpark as below: ```python >>> from pyspark import pandas as pp >>> ppdf = pp.DataFrame({"A": [1, 2, 3], "B": [15, 20, 25]}) >>> ppdf A B 0 1 15 1 2 20 2 3 25 ``` The existing "options and settings" in Koalas are also available in the same way: ```python >>> from pyspark.pandas.config import set_option, reset_option, get_option >>> ppser1 = pp.Series([1, 2, 3]) >>> ppser2 = pp.Series([3, 4, 5]) >>> ppser1 + ppser2 Traceback (most recent call last): ... ValueError: Cannot combine the series or dataframe because it comes from a different dataframe. In order to allow this operation, enable 'compute.ops_on_diff_frames' option. >>> set_option("compute.ops_on_diff_frames", True) >>> ppser1 + ppser2 0 4 1 6 2 8 dtype: int64 ``` Please also refer to the [API Reference](https://koalas.readthedocs.io/en/latest/reference/index.html) and [Options and Settings](https://koalas.readthedocs.io/en/latest/user_guide/options.html) for more detail. **NOTE** that this PR intentionally ports the main codes of Koalas first almost as are with minimal changes because: - Koalas project is fairly large. Making some changes together for PySpark will make it difficult to review the individual change. Koalas dev includes multiple Spark committers who will review. By doing this, the committers will be able to more easily and effectively review and drive the development. - Koalas tests and documentation require major changes to make it look great together with PySpark whereas main codes do not require. - We lately froze the Koalas codebase, and plan to work together on the initial porting. By porting the main codes first as are, it unblocks the Koalas dev to work on other items in parallel. I promise and will make sure on: - Rename Koalas to PySpark pandas APIs and/or pandas-on-Spark accordingly in documentation, and the docstrings and comments in the main codes. - Triage APIs to remove that don’t make sense when Koalas is in PySpark The documentation changes will be tracked in [SPARK-34885](https://issues.apache.org/jira/browse/SPARK-34885), the test code changes will be tracked in [SPARK-34886](https://issues.apache.org/jira/browse/SPARK-34886). ### Why are the changes needed? Please refer to: - [[DISCUSS] Support pandas API layer on PySpark](http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-Support-pandas-API-layer-on-PySpark-td30945.html) - [[VOTE] SPIP: Support pandas API layer on PySpark](http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-SPIP-Support-pandas-API-layer-on-PySpark-td30996.html) ### Does this PR introduce _any_ user-facing change? Yes, now users can use the pandas APIs on Spark ### How was this patch tested? Manually tested for exposed major APIs and options as described above. ### Koalas contributors Koalas would not have been possible without the following contributors: ueshin HyukjinKwon rxin xinrong-databricks RainFung charlesdong1991 harupy floscha beobest2 thunterdb garawalid LucasG0 shril deepyaman gioa fwani 90jam thoo AbdealiJK abishekganesh72 gliptak DumbMachine dvgodoy stbof nitlev hjoo gatorsmile tomspur icexelloss awdavidson guyao akhilputhiry scook12 patryk-oleniuk tracek dennyglee athena15 gstaubli WeichenXu123 hsubbaraj lfdversluis ktksq shengjh margaret-databricks LSturtew sllynn manuzhang jijosg sadikovi Closes #32036 from itholic/SPARK-34890. Authored-by: itholic <haejoon.lee@databricks.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-05 23:42:39 -04:00
#
# 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