[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.
|
|
|
|
|
#
|
|
|
|
|
|
|
|
|
|
"""
|
2021-04-14 23:48:59 -04:00
|
|
|
|
String functions on pandas-on-Spark Series
|
[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
|
|
|
|
"""
|
2021-05-21 14:03:35 -04:00
|
|
|
|
from typing import (
|
|
|
|
|
Any,
|
|
|
|
|
Callable,
|
|
|
|
|
Dict,
|
|
|
|
|
List,
|
|
|
|
|
Optional,
|
|
|
|
|
Union,
|
|
|
|
|
TYPE_CHECKING,
|
|
|
|
|
cast,
|
|
|
|
|
no_type_check,
|
|
|
|
|
)
|
[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
|
|
|
|
|
|
|
|
|
import numpy as np
|
|
|
|
|
|
2021-06-14 22:17:56 -04:00
|
|
|
|
import pandas as pd
|
[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
|
|
|
|
from pyspark.sql.types import StringType, BinaryType, ArrayType, LongType, MapType
|
|
|
|
|
from pyspark.sql import functions as F
|
2021-06-14 22:17:56 -04:00
|
|
|
|
from pyspark.sql.functions import pandas_udf
|
[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
|
|
|
|
|
|
|
|
|
from pyspark.pandas.spark import functions as SF
|
|
|
|
|
|
|
|
|
|
if TYPE_CHECKING:
|
2021-04-11 22:18:08 -04:00
|
|
|
|
import pyspark.pandas as ps # noqa: F401 (SPARK-34943)
|
[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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
class StringMethods(object):
|
2021-04-14 23:48:59 -04:00
|
|
|
|
"""String methods for pandas-on-Spark Series"""
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def __init__(self, series: "ps.Series"):
|
[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
|
|
|
|
if not isinstance(series.spark.data_type, (StringType, BinaryType, ArrayType)):
|
|
|
|
|
raise ValueError("Cannot call StringMethods on type {}".format(series.spark.data_type))
|
|
|
|
|
self._data = series
|
|
|
|
|
|
|
|
|
|
# Methods
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def capitalize(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Convert Strings in the series to be capitalized.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe'])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 lower
|
|
|
|
|
1 CAPITALS
|
|
|
|
|
2 this is a sentence
|
|
|
|
|
3 SwApCaSe
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.capitalize()
|
|
|
|
|
0 Lower
|
|
|
|
|
1 Capitals
|
|
|
|
|
2 This is a sentence
|
|
|
|
|
3 Swapcase
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_capitalize(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.capitalize()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_capitalize)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def title(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Convert Strings in the series to be titlecase.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe'])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 lower
|
|
|
|
|
1 CAPITALS
|
|
|
|
|
2 this is a sentence
|
|
|
|
|
3 SwApCaSe
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.title()
|
|
|
|
|
0 Lower
|
|
|
|
|
1 Capitals
|
|
|
|
|
2 This Is A Sentence
|
|
|
|
|
3 Swapcase
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_title(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.title()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_title)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def lower(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Convert strings in the Series/Index to all lowercase.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe'])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 lower
|
|
|
|
|
1 CAPITALS
|
|
|
|
|
2 this is a sentence
|
|
|
|
|
3 SwApCaSe
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.lower()
|
|
|
|
|
0 lower
|
|
|
|
|
1 capitals
|
|
|
|
|
2 this is a sentence
|
|
|
|
|
3 swapcase
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
return self._data.spark.transform(F.lower)
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def upper(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Convert strings in the Series/Index to all uppercase.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe'])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 lower
|
|
|
|
|
1 CAPITALS
|
|
|
|
|
2 this is a sentence
|
|
|
|
|
3 SwApCaSe
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.upper()
|
|
|
|
|
0 LOWER
|
|
|
|
|
1 CAPITALS
|
|
|
|
|
2 THIS IS A SENTENCE
|
|
|
|
|
3 SWAPCASE
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
return self._data.spark.transform(F.upper)
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def swapcase(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Convert strings in the Series/Index to be swapcased.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['lower', 'CAPITALS', 'this is a sentence', 'SwApCaSe'])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 lower
|
|
|
|
|
1 CAPITALS
|
|
|
|
|
2 this is a sentence
|
|
|
|
|
3 SwApCaSe
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.swapcase()
|
|
|
|
|
0 LOWER
|
|
|
|
|
1 capitals
|
|
|
|
|
2 THIS IS A SENTENCE
|
|
|
|
|
3 sWaPcAsE
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_swapcase(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.swapcase()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_swapcase)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def startswith(self, pattern: str, na: Optional[Any] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Test if the start of each string element matches a pattern.
|
|
|
|
|
|
|
|
|
|
Equivalent to :func:`str.startswith`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pattern : str
|
|
|
|
|
Character sequence. Regular expressions are not accepted.
|
|
|
|
|
na : object, default None
|
|
|
|
|
Object shown if element is not a string. NaN converted to None.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of bool or object
|
2021-04-14 23:48:59 -04:00
|
|
|
|
pandas-on-Spark Series of booleans indicating whether the given pattern
|
[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
|
|
|
|
matches the start of each string element.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['bat', 'Bear', 'cat', np.nan])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 bat
|
|
|
|
|
1 Bear
|
|
|
|
|
2 cat
|
|
|
|
|
3 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.startswith('b')
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Specifying na to be False instead of None.
|
|
|
|
|
|
|
|
|
|
>>> s.str.startswith('b', na=False)
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_startswith(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.startswith(pattern, na)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_startswith)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def endswith(self, pattern: str, na: Optional[Any] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Test if the end of each string element matches a pattern.
|
|
|
|
|
|
|
|
|
|
Equivalent to :func:`str.endswith`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pattern : str
|
|
|
|
|
Character sequence. Regular expressions are not accepted.
|
|
|
|
|
na : object, default None
|
|
|
|
|
Object shown if element is not a string. NaN converted to None.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of bool or object
|
2021-04-14 23:48:59 -04:00
|
|
|
|
pandas-on-Spark Series of booleans indicating whether the given pattern
|
[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
|
|
|
|
matches the end of each string element.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['bat', 'Bear', 'cat', np.nan])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 bat
|
|
|
|
|
1 Bear
|
|
|
|
|
2 cat
|
|
|
|
|
3 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.endswith('t')
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 True
|
|
|
|
|
3 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Specifying na to be False instead of None.
|
|
|
|
|
|
|
|
|
|
>>> s.str.endswith('t', na=False)
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_endswith(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.endswith(pattern, na)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_endswith)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def strip(self, to_strip: Optional[str] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Remove leading and trailing characters.
|
|
|
|
|
|
|
|
|
|
Strip whitespaces (including newlines) or a set of specified
|
|
|
|
|
characters from each string in the Series/Index from left and
|
|
|
|
|
right sides. Equivalent to :func:`str.strip`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
to_strip : str
|
|
|
|
|
Specifying the set of characters to be removed. All combinations
|
|
|
|
|
of this set of characters will be stripped. If None then
|
|
|
|
|
whitespaces are removed.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of objects
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['1. Ant.', '2. Bee!\\t', None])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 1. Ant.
|
|
|
|
|
1 2. Bee!\\t
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.strip()
|
|
|
|
|
0 1. Ant.
|
|
|
|
|
1 2. Bee!
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.strip('12.')
|
|
|
|
|
0 Ant
|
|
|
|
|
1 Bee!\\t
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.strip('.!\\t')
|
|
|
|
|
0 1. Ant
|
|
|
|
|
1 2. Bee
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_strip(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.strip(to_strip)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_strip)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def lstrip(self, to_strip: Optional[str] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Remove leading characters.
|
|
|
|
|
|
|
|
|
|
Strip whitespaces (including newlines) or a set of specified
|
|
|
|
|
characters from each string in the Series/Index from left side.
|
|
|
|
|
Equivalent to :func:`str.lstrip`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
to_strip : str
|
|
|
|
|
Specifying the set of characters to be removed. All combinations
|
|
|
|
|
of this set of characters will be stripped. If None then
|
|
|
|
|
whitespaces are removed.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['1. Ant.', '2. Bee!\\t', None])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 1. Ant.
|
|
|
|
|
1 2. Bee!\\t
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.lstrip('12.')
|
|
|
|
|
0 Ant.
|
|
|
|
|
1 Bee!\\t
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_lstrip(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.lstrip(to_strip)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_lstrip)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def rstrip(self, to_strip: Optional[str] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Remove trailing characters.
|
|
|
|
|
|
|
|
|
|
Strip whitespaces (including newlines) or a set of specified
|
|
|
|
|
characters from each string in the Series/Index from right side.
|
|
|
|
|
Equivalent to :func:`str.rstrip`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
to_strip : str
|
|
|
|
|
Specifying the set of characters to be removed. All combinations
|
|
|
|
|
of this set of characters will be stripped. If None then
|
|
|
|
|
whitespaces are removed.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['1. Ant.', '2. Bee!\\t', None])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 1. Ant.
|
|
|
|
|
1 2. Bee!\\t
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.rstrip('.!\\t')
|
|
|
|
|
0 1. Ant
|
|
|
|
|
1 2. Bee
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_rstrip(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.rstrip(to_strip)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_rstrip)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def get(self, i: int) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Extract element from each string or string list/tuple in the Series
|
|
|
|
|
at the specified position.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
i : int
|
|
|
|
|
Position of element to extract.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of objects
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s1 = ps.Series(["String", "123"])
|
[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
|
|
|
|
>>> s1
|
|
|
|
|
0 String
|
|
|
|
|
1 123
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s1.str.get(1)
|
|
|
|
|
0 t
|
|
|
|
|
1 2
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s1.str.get(-1)
|
|
|
|
|
0 g
|
|
|
|
|
1 3
|
|
|
|
|
dtype: object
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s2 = ps.Series([["a", "b", "c"], ["x", "y"]])
|
[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
|
|
|
|
>>> s2
|
|
|
|
|
0 [a, b, c]
|
|
|
|
|
1 [x, y]
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s2.str.get(0)
|
|
|
|
|
0 a
|
|
|
|
|
1 x
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s2.str.get(2)
|
|
|
|
|
0 c
|
|
|
|
|
1 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_get(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.get(i)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_get)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def isalnum(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are alphanumeric.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.isalnum` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s1 = ps.Series(['one', 'one1', '1', ''])
|
[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
|
|
|
|
|
|
|
|
|
>>> s1.str.isalnum()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
Note that checks against characters mixed with any additional
|
|
|
|
|
punctuation or whitespace will evaluate to false for an alphanumeric
|
|
|
|
|
check.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s2 = ps.Series(['A B', '1.5', '3,000'])
|
[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
|
|
|
|
>>> s2.str.isalnum()
|
|
|
|
|
0 False
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isalnum(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.isalnum()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isalnum)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def isalpha(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are alphabetic.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.isalpha` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s1 = ps.Series(['one', 'one1', '1', ''])
|
[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
|
|
|
|
|
|
|
|
|
>>> s1.str.isalpha()
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isalpha(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.isalpha()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isalpha)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def isdigit(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are digits.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.isdigit` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['23', '³', '⅕', ''])
|
[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
|
|
|
|
|
|
|
|
|
The s.str.isdecimal method checks for characters used to form numbers
|
|
|
|
|
in base 10.
|
|
|
|
|
|
|
|
|
|
>>> s.str.isdecimal()
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
The s.str.isdigit method is the same as s.str.isdecimal but also
|
|
|
|
|
includes special digits, like superscripted and subscripted digits in
|
|
|
|
|
unicode.
|
|
|
|
|
|
|
|
|
|
>>> s.str.isdigit()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
The s.str.isnumeric method is the same as s.str.isdigit but also
|
|
|
|
|
includes other characters that can represent quantities such as unicode
|
|
|
|
|
fractions.
|
|
|
|
|
|
|
|
|
|
>>> s.str.isnumeric()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isdigit(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.isdigit()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isdigit)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def isspace(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are whitespaces.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.isspace` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series([' ', '\\t\\r\\n ', ''])
|
[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
|
|
|
|
>>> s.str.isspace()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isspace(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.isspace()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isspace)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def islower(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are lowercase.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.islower` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['leopard', 'Golden Eagle', 'SNAKE', ''])
|
[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
|
|
|
|
>>> s.str.islower()
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isspace(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.islower()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isspace)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def isupper(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are uppercase.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.isupper` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['leopard', 'Golden Eagle', 'SNAKE', ''])
|
[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
|
|
|
|
>>> s.str.isupper()
|
|
|
|
|
0 False
|
|
|
|
|
1 False
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isspace(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.isupper()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isspace)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def istitle(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are titlecase.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.istitle` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['leopard', 'Golden Eagle', 'SNAKE', ''])
|
[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
|
|
|
|
|
|
|
|
|
The s.str.istitle method checks for whether all words are in title
|
|
|
|
|
case (whether only the first letter of each word is capitalized).
|
|
|
|
|
Words are assumed to be as any sequence of non-numeric characters
|
|
|
|
|
separated by whitespace characters.
|
|
|
|
|
|
|
|
|
|
>>> s.str.istitle()
|
|
|
|
|
0 False
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_istitle(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.istitle()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_istitle)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def isnumeric(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are numeric.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.isnumeric` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s1 = ps.Series(['one', 'one1', '1', ''])
|
[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
|
|
|
|
>>> s1.str.isnumeric()
|
|
|
|
|
0 False
|
|
|
|
|
1 False
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s2 = ps.Series(['23', '³', '⅕', ''])
|
[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
|
|
|
|
|
|
|
|
|
The s2.str.isdecimal method checks for characters used to form numbers
|
|
|
|
|
in base 10.
|
|
|
|
|
|
|
|
|
|
>>> s2.str.isdecimal()
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
The s2.str.isdigit method is the same as s2.str.isdecimal but also
|
|
|
|
|
includes special digits, like superscripted and subscripted digits in
|
|
|
|
|
unicode.
|
|
|
|
|
|
|
|
|
|
>>> s2.str.isdigit()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
The s2.str.isnumeric method is the same as s2.str.isdigit but also
|
|
|
|
|
includes other characters that can represent quantities such as unicode
|
|
|
|
|
fractions.
|
|
|
|
|
|
|
|
|
|
>>> s2.str.isnumeric()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isnumeric(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.isnumeric()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isnumeric)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def isdecimal(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Check whether all characters in each string are decimals.
|
|
|
|
|
|
|
|
|
|
This is equivalent to running the Python string method
|
|
|
|
|
:func:`str.isdecimal` for each element of the Series/Index.
|
|
|
|
|
If a string has zero characters, False is returned for that check.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['23', '³', '⅕', ''])
|
[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
|
|
|
|
|
|
|
|
|
The s.str.isdecimal method checks for characters used to form numbers
|
|
|
|
|
in base 10.
|
|
|
|
|
|
|
|
|
|
>>> s.str.isdecimal()
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
The s.str.isdigit method is the same as s.str.isdecimal but also
|
|
|
|
|
includes special digits, like superscripted and subscripted digits in
|
|
|
|
|
unicode.
|
|
|
|
|
|
|
|
|
|
>>> s.str.isdigit()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
The s.str.isnumeric method is the same as s.str.isdigit but also
|
|
|
|
|
includes other characters that can represent quantities such as unicode
|
|
|
|
|
fractions.
|
|
|
|
|
|
|
|
|
|
>>> s.str.isnumeric()
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_isdecimal(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.isdecimal()
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_isdecimal)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def cat(self, others=None, sep=None, na_rep=None, join=None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def center(self, width: int, fillchar: str = " ") -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Filling left and right side of strings in the Series/Index with an
|
|
|
|
|
additional character. Equivalent to :func:`str.center`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
width : int
|
|
|
|
|
Minimum width of resulting string; additional characters will be
|
|
|
|
|
filled with fillchar.
|
|
|
|
|
fillchar : str
|
|
|
|
|
Additional character for filling, default is whitespace.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of objects
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["caribou", "tiger"])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 caribou
|
|
|
|
|
1 tiger
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.center(width=10, fillchar='-')
|
|
|
|
|
0 -caribou--
|
|
|
|
|
1 --tiger---
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_center(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.center(width, fillchar)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_center)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def contains(
|
|
|
|
|
self, pat: str, case: bool = True, flags: int = 0, na: Any = None, regex: bool = True
|
|
|
|
|
) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Test if pattern or regex is contained within a string of a Series.
|
|
|
|
|
|
|
|
|
|
Return boolean Series based on whether a given pattern or regex is
|
|
|
|
|
contained within a string of a Series.
|
|
|
|
|
|
|
|
|
|
Analogous to :func:`match`, but less strict, relying on
|
|
|
|
|
:func:`re.search` instead of :func:`re.match`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pat : str
|
|
|
|
|
Character sequence or regular expression.
|
|
|
|
|
case : bool, default True
|
|
|
|
|
If True, case sensitive.
|
|
|
|
|
flags : int, default 0 (no flags)
|
|
|
|
|
Flags to pass through to the re module, e.g. re.IGNORECASE.
|
|
|
|
|
na : default None
|
|
|
|
|
Fill value for missing values. NaN converted to None.
|
|
|
|
|
regex : bool, default True
|
|
|
|
|
If True, assumes the pat is a regular expression.
|
|
|
|
|
If False, treats the pat as a literal string.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of boolean values or object
|
|
|
|
|
A Series of boolean values indicating whether the given pattern is
|
|
|
|
|
contained within the string of each element of the Series.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
|
|
|
|
Returning a Series of booleans using only a literal pattern.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s1 = ps.Series(['Mouse', 'dog', 'house and parrot', '23', np.NaN])
|
[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
|
|
|
|
>>> s1.str.contains('og', regex=False)
|
|
|
|
|
0 False
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Specifying case sensitivity using case.
|
|
|
|
|
|
|
|
|
|
>>> s1.str.contains('oG', case=True, regex=True)
|
|
|
|
|
0 False
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Specifying na to be False instead of NaN replaces NaN values with
|
|
|
|
|
False. If Series does not contain NaN values the resultant dtype will
|
|
|
|
|
be bool, otherwise, an object dtype.
|
|
|
|
|
|
|
|
|
|
>>> s1.str.contains('og', na=False, regex=True)
|
|
|
|
|
0 False
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
4 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
Returning ‘house’ or ‘dog’ when either expression occurs in a string.
|
|
|
|
|
|
|
|
|
|
>>> s1.str.contains('house|dog', regex=True)
|
|
|
|
|
0 False
|
|
|
|
|
1 True
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Ignoring case sensitivity using flags with regex.
|
|
|
|
|
|
|
|
|
|
>>> import re
|
|
|
|
|
>>> s1.str.contains('PARROT', flags=re.IGNORECASE, regex=True)
|
|
|
|
|
0 False
|
|
|
|
|
1 False
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Returning any digit using regular expression.
|
|
|
|
|
|
|
|
|
|
>>> s1.str.contains('[0-9]', regex=True)
|
|
|
|
|
0 False
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 True
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Ensure pat is a not a literal pattern when regex is set to True.
|
|
|
|
|
Note in the following example one might expect only s2[1] and s2[3]
|
|
|
|
|
to return True. However, ‘.0’ as a regex matches any character followed
|
|
|
|
|
by a 0.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s2 = ps.Series(['40','40.0','41','41.0','35'])
|
[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
|
|
|
|
>>> s2.str.contains('.0', regex=True)
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 True
|
|
|
|
|
4 False
|
|
|
|
|
dtype: bool
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_contains(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.contains(pat, case, flags, na, regex)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_contains)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def count(self, pat: str, flags: int = 0) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Count occurrences of pattern in each string of the Series.
|
|
|
|
|
|
|
|
|
|
This function is used to count the number of times a particular regex
|
|
|
|
|
pattern is repeated in each of the string elements of the Series.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pat : str
|
|
|
|
|
Valid regular expression.
|
|
|
|
|
flags : int, default 0 (no flags)
|
|
|
|
|
Flags for the re module.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of int
|
|
|
|
|
A Series containing the integer counts of pattern matches.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['A', 'B', 'Aaba', 'Baca', np.NaN, 'CABA', 'cat'])
|
[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
|
|
|
|
>>> s.str.count('a')
|
|
|
|
|
0 0.0
|
|
|
|
|
1 0.0
|
|
|
|
|
2 2.0
|
|
|
|
|
3 2.0
|
|
|
|
|
4 NaN
|
|
|
|
|
5 0.0
|
|
|
|
|
6 1.0
|
|
|
|
|
dtype: float64
|
|
|
|
|
|
|
|
|
|
Escape '$' to find the literal dollar sign.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['$', 'B', 'Aab$', '$$ca', 'C$B$', 'cat'])
|
[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
|
|
|
|
>>> s.str.count('\\$')
|
|
|
|
|
0 1
|
|
|
|
|
1 0
|
|
|
|
|
2 1
|
|
|
|
|
3 2
|
|
|
|
|
4 2
|
|
|
|
|
5 0
|
|
|
|
|
dtype: int64
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_count(s) -> "ps.Series[int]":
|
[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
|
|
|
|
return s.str.count(pat, flags)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_count)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def decode(self, encoding, errors="strict") -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def encode(self, encoding, errors="strict") -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def extract(self, pat, flags=0, expand=True) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def extractall(self, pat, flags=0) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def find(self, sub: str, start: int = 0, end: Optional[int] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Return lowest indexes in each strings in the Series where the
|
|
|
|
|
substring is fully contained between [start:end].
|
|
|
|
|
|
|
|
|
|
Return -1 on failure. Equivalent to standard :func:`str.find`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
sub : str
|
|
|
|
|
Substring being searched.
|
|
|
|
|
start : int
|
|
|
|
|
Left edge index.
|
|
|
|
|
end : int
|
|
|
|
|
Right edge index.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of int
|
|
|
|
|
Series of lowest matching indexes.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['apple', 'oranges', 'bananas'])
|
[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
|
|
|
|
|
|
|
|
|
>>> s.str.find('a')
|
|
|
|
|
0 0
|
|
|
|
|
1 2
|
|
|
|
|
2 1
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
>>> s.str.find('a', start=2)
|
|
|
|
|
0 -1
|
|
|
|
|
1 2
|
|
|
|
|
2 3
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
>>> s.str.find('a', end=1)
|
|
|
|
|
0 0
|
|
|
|
|
1 -1
|
|
|
|
|
2 -1
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
>>> s.str.find('a', start=2, end=2)
|
|
|
|
|
0 -1
|
|
|
|
|
1 -1
|
|
|
|
|
2 -1
|
|
|
|
|
dtype: int64
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_find(s) -> "ps.Series[int]":
|
[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
|
|
|
|
return s.str.find(sub, start, end)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_find)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def findall(self, pat: str, flags: int = 0) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Find all occurrences of pattern or regular expression in the Series.
|
|
|
|
|
|
|
|
|
|
Equivalent to applying :func:`re.findall` to all the elements in
|
|
|
|
|
the Series.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pat : str
|
|
|
|
|
Pattern or regular expression.
|
|
|
|
|
flags : int, default 0 (no flags)
|
|
|
|
|
`re` module flags, e.g. `re.IGNORECASE`.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
All non-overlapping matches of pattern or regular expression in
|
|
|
|
|
each string of this Series.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['Lion', 'Monkey', 'Rabbit'])
|
[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
|
|
|
|
|
|
|
|
|
The search for the pattern ‘Monkey’ returns one match:
|
|
|
|
|
|
|
|
|
|
>>> s.str.findall('Monkey')
|
|
|
|
|
0 []
|
|
|
|
|
1 [Monkey]
|
|
|
|
|
2 []
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
On the other hand, the search for the pattern ‘MONKEY’ doesn’t return
|
|
|
|
|
any match:
|
|
|
|
|
|
|
|
|
|
>>> s.str.findall('MONKEY')
|
|
|
|
|
0 []
|
|
|
|
|
1 []
|
|
|
|
|
2 []
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Flags can be added to the pattern or regular expression. For instance,
|
|
|
|
|
to find the pattern ‘MONKEY’ ignoring the case:
|
|
|
|
|
|
|
|
|
|
>>> import re
|
|
|
|
|
>>> s.str.findall('MONKEY', flags=re.IGNORECASE)
|
|
|
|
|
0 []
|
|
|
|
|
1 [Monkey]
|
|
|
|
|
2 []
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
When the pattern matches more than one string in the Series, all
|
|
|
|
|
matches are returned:
|
|
|
|
|
|
|
|
|
|
>>> s.str.findall('on')
|
|
|
|
|
0 [on]
|
|
|
|
|
1 [on]
|
|
|
|
|
2 []
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Regular expressions are supported too. For instance, the search for all
|
|
|
|
|
the strings ending with the word ‘on’ is shown next:
|
|
|
|
|
|
|
|
|
|
>>> s.str.findall('on$')
|
|
|
|
|
0 [on]
|
|
|
|
|
1 []
|
|
|
|
|
2 []
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
If the pattern is found more than once in the same string, then a list
|
|
|
|
|
of multiple strings is returned:
|
|
|
|
|
|
|
|
|
|
>>> s.str.findall('b')
|
|
|
|
|
0 []
|
|
|
|
|
1 []
|
|
|
|
|
2 [b, b]
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
# type hint does not support to specify array type yet.
|
2021-06-14 22:17:56 -04:00
|
|
|
|
@pandas_udf(returnType=ArrayType(StringType(), containsNull=True)) # type: ignore
|
|
|
|
|
def pudf(s: pd.Series) -> pd.Series:
|
|
|
|
|
return s.str.findall(pat, flags)
|
|
|
|
|
|
[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
|
|
|
|
return self._data._with_new_scol(scol=pudf(self._data.spark.column))
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def index(self, sub: str, start: int = 0, end: Optional[int] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Return lowest indexes in each strings where the substring is fully
|
|
|
|
|
contained between [start:end].
|
|
|
|
|
|
|
|
|
|
This is the same as :func:`str.find` except instead of returning -1,
|
|
|
|
|
it raises a ValueError when the substring is not found. Equivalent to
|
|
|
|
|
standard :func:`str.index`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
sub : str
|
|
|
|
|
Substring being searched.
|
|
|
|
|
start : int
|
|
|
|
|
Left edge index.
|
|
|
|
|
end : int
|
|
|
|
|
Right edge index.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of int
|
|
|
|
|
Series of lowest matching indexes.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['apple', 'oranges', 'bananas'])
|
[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
|
|
|
|
|
|
|
|
|
>>> s.str.index('a')
|
|
|
|
|
0 0
|
|
|
|
|
1 2
|
|
|
|
|
2 1
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
The following expression throws an exception:
|
|
|
|
|
|
|
|
|
|
>>> s.str.index('a', start=2) # doctest: +SKIP
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_index(s) -> "ps.Series[np.int64]":
|
[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
|
|
|
|
return s.str.index(sub, start, end)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_index)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def join(self, sep: str) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Join lists contained as elements in the Series with passed delimiter.
|
|
|
|
|
|
|
|
|
|
If the elements of a Series are lists themselves, join the content of
|
|
|
|
|
these lists using the delimiter passed to the function. This function
|
|
|
|
|
is an equivalent to calling :func:`str.join` on the lists.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
sep : str
|
|
|
|
|
Delimiter to use between list entries.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Series with list entries concatenated by intervening occurrences of
|
|
|
|
|
the delimiter.
|
|
|
|
|
|
|
|
|
|
See Also
|
|
|
|
|
--------
|
|
|
|
|
str.split : Split strings around given separator/delimiter.
|
|
|
|
|
str.rsplit : Splits string around given separator/delimiter,
|
|
|
|
|
starting from the right.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
|
|
|
|
Example with a list that contains a None element.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series([['lion', 'elephant', 'zebra'],
|
[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
|
|
|
|
... ['cat', None, 'dog']])
|
|
|
|
|
>>> s
|
|
|
|
|
0 [lion, elephant, zebra]
|
|
|
|
|
1 [cat, None, dog]
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Join all lists using a ‘-‘. The list containing None will produce None.
|
|
|
|
|
|
|
|
|
|
>>> s.str.join('-')
|
|
|
|
|
0 lion-elephant-zebra
|
|
|
|
|
1 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_join(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.join(sep)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_join)
|
[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
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def len(self) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Computes the length of each element in the Series.
|
|
|
|
|
|
|
|
|
|
The element may be a sequence (such as a string, tuple or list).
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of int
|
|
|
|
|
A Series of integer values indicating the length of each element in
|
|
|
|
|
the Series.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
|
|
|
|
Returns the length (number of characters) in a string. Returns the
|
|
|
|
|
number of entries for lists or tuples.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s1 = ps.Series(['dog', 'monkey'])
|
[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
|
|
|
|
>>> s1.str.len()
|
|
|
|
|
0 3
|
|
|
|
|
1 6
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s2 = ps.Series([["a", "b", "c"], []])
|
[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
|
|
|
|
>>> s2.str.len()
|
|
|
|
|
0 3
|
|
|
|
|
1 0
|
|
|
|
|
dtype: int64
|
|
|
|
|
"""
|
|
|
|
|
if isinstance(self._data.spark.data_type, (ArrayType, MapType)):
|
|
|
|
|
return self._data.spark.transform(lambda c: F.size(c).cast(LongType()))
|
|
|
|
|
else:
|
|
|
|
|
return self._data.spark.transform(lambda c: F.length(c).cast(LongType()))
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def ljust(self, width: int, fillchar: str = " ") -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Filling right side of strings in the Series with an additional
|
|
|
|
|
character. Equivalent to :func:`str.ljust`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
width : int
|
|
|
|
|
Minimum width of resulting string; additional characters will be
|
|
|
|
|
filled with `fillchar`.
|
|
|
|
|
fillchar : str
|
|
|
|
|
Additional character for filling, default is whitespace.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["caribou", "tiger"])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 caribou
|
|
|
|
|
1 tiger
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.ljust(width=10, fillchar='-')
|
|
|
|
|
0 caribou---
|
|
|
|
|
1 tiger-----
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_ljust(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.ljust(width, fillchar)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_ljust)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def match(self, pat: str, case: bool = True, flags: int = 0, na: Any = np.NaN) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Determine if each string matches a regular expression.
|
|
|
|
|
|
|
|
|
|
Analogous to :func:`contains`, but more strict, relying on
|
|
|
|
|
:func:`re.match` instead of :func:`re.search`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pat : str
|
|
|
|
|
Character sequence or regular expression.
|
|
|
|
|
case : bool, default True
|
|
|
|
|
If True, case sensitive.
|
|
|
|
|
flags : int, default 0 (no flags)
|
|
|
|
|
Flags to pass through to the re module, e.g. re.IGNORECASE.
|
|
|
|
|
na : default NaN
|
|
|
|
|
Fill value for missing values.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of boolean values or object
|
|
|
|
|
A Series of boolean values indicating whether the given pattern can
|
|
|
|
|
be matched in the string of each element of the Series.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['Mouse', 'dog', 'house and parrot', '23', np.NaN])
|
[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
|
|
|
|
>>> s.str.match('dog')
|
|
|
|
|
0 False
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.match('mouse|dog', case=False)
|
|
|
|
|
0 True
|
|
|
|
|
1 True
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.match('.+and.+', na=True)
|
|
|
|
|
0 False
|
|
|
|
|
1 False
|
|
|
|
|
2 True
|
|
|
|
|
3 False
|
|
|
|
|
4 True
|
|
|
|
|
dtype: bool
|
|
|
|
|
|
|
|
|
|
>>> import re
|
|
|
|
|
>>> s.str.match('MOUSE', flags=re.IGNORECASE)
|
|
|
|
|
0 True
|
|
|
|
|
1 False
|
|
|
|
|
2 False
|
|
|
|
|
3 False
|
|
|
|
|
4 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_match(s) -> "ps.Series[bool]":
|
[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
|
|
|
|
return s.str.match(pat, case, flags, na)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_match)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def normalize(self, form: str) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Return the Unicode normal form for the strings in the Series.
|
|
|
|
|
|
|
|
|
|
For more information on the forms, see the
|
|
|
|
|
:func:`unicodedata.normalize`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
form : {‘NFC’, ‘NFKC’, ‘NFD’, ‘NFKD’}
|
|
|
|
|
Unicode form.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of objects
|
|
|
|
|
A Series of normalized strings.
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_normalize(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.normalize(form)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_normalize)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def pad(self, width: int, side: str = "left", fillchar: str = " ") -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Pad strings in the Series up to width.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
width : int
|
|
|
|
|
Minimum width of resulting string; additional characters will be
|
|
|
|
|
filled with character defined in `fillchar`.
|
|
|
|
|
side : {‘left’, ‘right’, ‘both’}, default ‘left’
|
|
|
|
|
Side from which to fill resulting string.
|
|
|
|
|
fillchar : str, default ' '
|
|
|
|
|
Additional character for filling, default is whitespace.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Returns Series with minimum number of char in object.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["caribou", "tiger"])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 caribou
|
|
|
|
|
1 tiger
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.pad(width=10)
|
|
|
|
|
0 caribou
|
|
|
|
|
1 tiger
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.pad(width=10, side='right', fillchar='-')
|
|
|
|
|
0 caribou---
|
|
|
|
|
1 tiger-----
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.pad(width=10, side='both', fillchar='-')
|
|
|
|
|
0 -caribou--
|
|
|
|
|
1 --tiger---
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_pad(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.pad(width, side, fillchar)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_pad)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def partition(self, sep: str = " ", expand: bool = True) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def repeat(self, repeats: int) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Duplicate each string in the Series.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
repeats : int
|
|
|
|
|
Repeat the string given number of times (int). Sequence of int
|
|
|
|
|
is not supported.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Series or Index of repeated string objects specified by input
|
|
|
|
|
parameter repeats.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['a', 'b', 'c'])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 a
|
|
|
|
|
1 b
|
|
|
|
|
2 c
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Single int repeats string in Series
|
|
|
|
|
|
|
|
|
|
>>> s.str.repeat(repeats=2)
|
|
|
|
|
0 aa
|
|
|
|
|
1 bb
|
|
|
|
|
2 cc
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
if not isinstance(repeats, int):
|
2021-05-03 02:34:24 -04:00
|
|
|
|
raise TypeError("repeats expects an int parameter")
|
[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
|
|
|
|
return self._data.spark.transform(lambda c: SF.repeat(col=c, n=repeats))
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def replace(
|
|
|
|
|
self,
|
|
|
|
|
pat: str,
|
|
|
|
|
repl: Union[str, Callable[[str], str]],
|
|
|
|
|
n: int = -1,
|
|
|
|
|
case: Optional[bool] = None,
|
|
|
|
|
flags: int = 0,
|
|
|
|
|
regex: bool = True,
|
|
|
|
|
) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Replace occurrences of pattern/regex in the Series with some other
|
|
|
|
|
string. Equivalent to :func:`str.replace` or :func:`re.sub`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pat : str or compiled regex
|
|
|
|
|
String can be a character sequence or regular expression.
|
|
|
|
|
repl : str or callable
|
|
|
|
|
Replacement string or a callable. The callable is passed the regex
|
|
|
|
|
match object and must return a replacement string to be used. See
|
|
|
|
|
:func:`re.sub`.
|
|
|
|
|
n : int, default -1 (all)
|
|
|
|
|
Number of replacements to make from start.
|
|
|
|
|
case : boolean, default None
|
|
|
|
|
If True, case sensitive (the default if pat is a string).
|
|
|
|
|
Set to False for case insensitive.
|
|
|
|
|
Cannot be set if pat is a compiled regex.
|
|
|
|
|
flags: int, default 0 (no flags)
|
|
|
|
|
re module flags, e.g. re.IGNORECASE.
|
|
|
|
|
Cannot be set if pat is a compiled regex.
|
|
|
|
|
regex : boolean, default True
|
|
|
|
|
If True, assumes the passed-in pattern is a regular expression.
|
|
|
|
|
If False, treats the pattern as a literal string.
|
|
|
|
|
Cannot be set to False if pat is a compile regex or repl is a
|
|
|
|
|
callable.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
A copy of the string with all matching occurrences of pat replaced
|
|
|
|
|
by repl.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
|
|
|
|
When pat is a string and regex is True (the default), the given pat is
|
|
|
|
|
compiled as a regex. When repl is a string, it replaces matching regex
|
|
|
|
|
patterns as with :func:`re.sub`. NaN value(s) in the Series are changed
|
|
|
|
|
to None:
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> ps.Series(['foo', 'fuz', np.nan]).str.replace('f.', 'ba', regex=True)
|
[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
|
|
|
|
0 bao
|
|
|
|
|
1 baz
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
When pat is a string and regex is False, every pat is replaced with
|
|
|
|
|
repl as with :func:`str.replace`:
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> ps.Series(['f.o', 'fuz', np.nan]).str.replace('f.', 'ba', regex=False)
|
[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
|
|
|
|
0 bao
|
|
|
|
|
1 fuz
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
When repl is a callable, it is called on every pat using
|
|
|
|
|
:func:`re.sub`. The callable should expect one positional argument (a
|
|
|
|
|
regex object) and return a string.
|
|
|
|
|
|
|
|
|
|
Reverse every lowercase alphabetic word:
|
|
|
|
|
|
|
|
|
|
>>> repl = lambda m: m.group(0)[::-1]
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> ps.Series(['foo 123', 'bar baz', np.nan]).str.replace(r'[a-z]+', repl)
|
[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
|
|
|
|
0 oof 123
|
|
|
|
|
1 rab zab
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Using regex groups (extract second group and swap case):
|
|
|
|
|
|
|
|
|
|
>>> pat = r"(?P<one>\\w+) (?P<two>\\w+) (?P<three>\\w+)"
|
|
|
|
|
>>> repl = lambda m: m.group('two').swapcase()
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> ps.Series(['One Two Three', 'Foo Bar Baz']).str.replace(pat, repl)
|
[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
|
|
|
|
0 tWO
|
|
|
|
|
1 bAR
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Using a compiled regex with flags:
|
|
|
|
|
|
|
|
|
|
>>> import re
|
|
|
|
|
>>> regex_pat = re.compile(r'FUZ', flags=re.IGNORECASE)
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> ps.Series(['foo', 'fuz', np.nan]).str.replace(regex_pat, 'bar')
|
[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
|
|
|
|
0 foo
|
|
|
|
|
1 bar
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_replace(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.replace(pat, repl, n=n, case=case, flags=flags, regex=regex)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_replace)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def rfind(self, sub: str, start: int = 0, end: Optional[int] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Return highest indexes in each strings in the Series where the
|
|
|
|
|
substring is fully contained between [start:end].
|
|
|
|
|
|
|
|
|
|
Return -1 on failure. Equivalent to standard :func:`str.rfind`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
sub : str
|
|
|
|
|
Substring being searched.
|
|
|
|
|
start : int
|
|
|
|
|
Left edge index.
|
|
|
|
|
end : int
|
|
|
|
|
Right edge index.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of int
|
|
|
|
|
Series of highest matching indexes.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['apple', 'oranges', 'bananas'])
|
[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
|
|
|
|
|
|
|
|
|
>>> s.str.rfind('a')
|
|
|
|
|
0 0
|
|
|
|
|
1 2
|
|
|
|
|
2 5
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
>>> s.str.rfind('a', start=2)
|
|
|
|
|
0 -1
|
|
|
|
|
1 2
|
|
|
|
|
2 5
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
>>> s.str.rfind('a', end=1)
|
|
|
|
|
0 0
|
|
|
|
|
1 -1
|
|
|
|
|
2 -1
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
>>> s.str.rfind('a', start=2, end=2)
|
|
|
|
|
0 -1
|
|
|
|
|
1 -1
|
|
|
|
|
2 -1
|
|
|
|
|
dtype: int64
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_rfind(s) -> "ps.Series[int]":
|
[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
|
|
|
|
return s.str.rfind(sub, start, end)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_rfind)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def rindex(self, sub: str, start: int = 0, end: Optional[int] = None) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Return highest indexes in each strings where the substring is fully
|
|
|
|
|
contained between [start:end].
|
|
|
|
|
|
|
|
|
|
This is the same as :func:`str.rfind` except instead of returning -1,
|
|
|
|
|
it raises a ValueError when the substring is not found. Equivalent to
|
|
|
|
|
standard :func:`str.rindex`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
sub : str
|
|
|
|
|
Substring being searched.
|
|
|
|
|
start : int
|
|
|
|
|
Left edge index.
|
|
|
|
|
end : int
|
|
|
|
|
Right edge index.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of int
|
|
|
|
|
Series of highest matching indexes.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['apple', 'oranges', 'bananas'])
|
[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
|
|
|
|
|
|
|
|
|
>>> s.str.rindex('a')
|
|
|
|
|
0 0
|
|
|
|
|
1 2
|
|
|
|
|
2 5
|
|
|
|
|
dtype: int64
|
|
|
|
|
|
|
|
|
|
The following expression throws an exception:
|
|
|
|
|
|
|
|
|
|
>>> s.str.rindex('a', start=2) # doctest: +SKIP
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_rindex(s) -> "ps.Series[np.int64]":
|
[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
|
|
|
|
return s.str.rindex(sub, start, end)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_rindex)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def rjust(self, width: int, fillchar: str = " ") -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Filling left side of strings in the Series with an additional
|
|
|
|
|
character. Equivalent to :func:`str.rjust`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
width : int
|
|
|
|
|
Minimum width of resulting string; additional characters will be
|
|
|
|
|
filled with `fillchar`.
|
|
|
|
|
fillchar : str
|
|
|
|
|
Additional character for filling, default is whitespace.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["caribou", "tiger"])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 caribou
|
|
|
|
|
1 tiger
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.rjust(width=10)
|
|
|
|
|
0 caribou
|
|
|
|
|
1 tiger
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.rjust(width=10, fillchar='-')
|
|
|
|
|
0 ---caribou
|
|
|
|
|
1 -----tiger
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_rjust(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.rjust(width, fillchar)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_rjust)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def rpartition(self, sep: str = " ", expand: bool = True) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def slice(
|
|
|
|
|
self, start: Optional[int] = None, stop: Optional[int] = None, step: Optional[int] = None
|
|
|
|
|
) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Slice substrings from each element in the Series.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
start : int, optional
|
|
|
|
|
Start position for slice operation.
|
|
|
|
|
stop : int, optional
|
|
|
|
|
Stop position for slice operation.
|
|
|
|
|
step : int, optional
|
|
|
|
|
Step size for slice operation.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Series from sliced substrings from original string objects.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["koala", "fox", "chameleon"])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 koala
|
|
|
|
|
1 fox
|
|
|
|
|
2 chameleon
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.slice(start=1)
|
|
|
|
|
0 oala
|
|
|
|
|
1 ox
|
|
|
|
|
2 hameleon
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.slice(stop=2)
|
|
|
|
|
0 ko
|
|
|
|
|
1 fo
|
|
|
|
|
2 ch
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.slice(step=2)
|
|
|
|
|
0 kaa
|
|
|
|
|
1 fx
|
|
|
|
|
2 caeen
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.slice(start=0, stop=5, step=3)
|
|
|
|
|
0 kl
|
|
|
|
|
1 f
|
|
|
|
|
2 cm
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_slice(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.slice(start, stop, step)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_slice)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def slice_replace(
|
|
|
|
|
self, start: Optional[int] = None, stop: Optional[int] = None, repl: Optional[str] = None
|
|
|
|
|
) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Slice substrings from each element in the Series.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
start : int, optional
|
|
|
|
|
Start position for slice operation. If not specified (None), the
|
|
|
|
|
slice is unbounded on the left, i.e. slice from the start of the
|
|
|
|
|
string.
|
|
|
|
|
stop : int, optional
|
|
|
|
|
Stop position for slice operation. If not specified (None), the
|
|
|
|
|
slice is unbounded on the right, i.e. slice until the end of the
|
|
|
|
|
string.
|
|
|
|
|
repl : str, optional
|
|
|
|
|
String for replacement. If not specified (None), the sliced region
|
|
|
|
|
is replaced with an empty string.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Series from sliced substrings from original string objects.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['a', 'ab', 'abc', 'abdc', 'abcde'])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 a
|
|
|
|
|
1 ab
|
|
|
|
|
2 abc
|
|
|
|
|
3 abdc
|
|
|
|
|
4 abcde
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Specify just start, meaning replace start until the end of the string
|
|
|
|
|
with repl.
|
|
|
|
|
|
|
|
|
|
>>> s.str.slice_replace(1, repl='X')
|
|
|
|
|
0 aX
|
|
|
|
|
1 aX
|
|
|
|
|
2 aX
|
|
|
|
|
3 aX
|
|
|
|
|
4 aX
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Specify just stop, meaning the start of the string to stop is replaced
|
|
|
|
|
with repl, and the rest of the string is included.
|
|
|
|
|
|
|
|
|
|
>>> s.str.slice_replace(stop=2, repl='X')
|
|
|
|
|
0 X
|
|
|
|
|
1 X
|
|
|
|
|
2 Xc
|
|
|
|
|
3 Xdc
|
|
|
|
|
4 Xcde
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Specify start and stop, meaning the slice from start to stop is
|
|
|
|
|
replaced with repl. Everything before or after start and stop is
|
|
|
|
|
included as is.
|
|
|
|
|
|
|
|
|
|
>>> s.str.slice_replace(start=1, stop=3, repl='X')
|
|
|
|
|
0 aX
|
|
|
|
|
1 aX
|
|
|
|
|
2 aX
|
|
|
|
|
3 aXc
|
|
|
|
|
4 aXde
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_slice_replace(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.slice_replace(start, stop, repl)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_slice_replace)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def split(
|
|
|
|
|
self, pat: Optional[str] = None, n: int = -1, expand: bool = False
|
|
|
|
|
) -> Union["ps.Series", "ps.DataFrame"]:
|
[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
|
|
|
|
"""
|
|
|
|
|
Split strings around given separator/delimiter.
|
|
|
|
|
|
|
|
|
|
Splits the string in the Series from the beginning, at the specified
|
|
|
|
|
delimiter string. Equivalent to :func:`str.split`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pat : str, optional
|
|
|
|
|
String or regular expression to split on. If not specified, split
|
|
|
|
|
on whitespace.
|
|
|
|
|
n : int, default -1 (all)
|
|
|
|
|
Limit number of splits in output. None, 0 and -1 will be
|
|
|
|
|
interpreted as return all splits.
|
|
|
|
|
expand : bool, default False
|
|
|
|
|
Expand the splitted strings into separate columns.
|
|
|
|
|
|
|
|
|
|
* If ``True``, `n` must be a positive integer, and return DataFrame expanding
|
|
|
|
|
dimensionality.
|
|
|
|
|
* If ``False``, return Series, containing lists of strings.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series, DataFrame
|
|
|
|
|
Type matches caller unless `expand=True` (see Notes).
|
|
|
|
|
|
|
|
|
|
See Also
|
|
|
|
|
--------
|
|
|
|
|
str.rsplit : Splits string around given separator/delimiter,
|
|
|
|
|
starting from the right.
|
|
|
|
|
str.join : Join lists contained as elements in the Series/Index
|
|
|
|
|
with passed delimiter.
|
|
|
|
|
|
|
|
|
|
Notes
|
|
|
|
|
-----
|
|
|
|
|
The handling of the `n` keyword depends on the number of found splits:
|
|
|
|
|
|
|
|
|
|
- If found splits > `n`, make first `n` splits only
|
|
|
|
|
- If found splits <= `n`, make all splits
|
|
|
|
|
- If for a certain row the number of found splits < `n`,
|
|
|
|
|
append `None` for padding up to `n` if ``expand=True``
|
|
|
|
|
|
|
|
|
|
If using ``expand=True``, Series callers return DataFrame objects with `n + 1` columns.
|
|
|
|
|
|
|
|
|
|
.. note:: Even if `n` is much larger than found splits, the number of columns does NOT
|
|
|
|
|
shrink unlike pandas.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["this is a regular sentence",
|
[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
|
|
|
|
... "https://docs.python.org/3/tutorial/index.html",
|
|
|
|
|
... np.nan])
|
|
|
|
|
|
|
|
|
|
In the default setting, the string is split by whitespace.
|
|
|
|
|
|
|
|
|
|
>>> s.str.split()
|
|
|
|
|
0 [this, is, a, regular, sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Without the n parameter, the outputs of rsplit and split are identical.
|
|
|
|
|
|
|
|
|
|
>>> s.str.rsplit()
|
|
|
|
|
0 [this, is, a, regular, sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
The n parameter can be used to limit the number of splits on the
|
|
|
|
|
delimiter. The outputs of split and rsplit are different.
|
|
|
|
|
|
|
|
|
|
>>> s.str.split(n=2)
|
|
|
|
|
0 [this, is, a regular sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.rsplit(n=2)
|
|
|
|
|
0 [this is a, regular, sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
The pat parameter can be used to split by other characters.
|
|
|
|
|
|
|
|
|
|
>>> s.str.split(pat = "/")
|
|
|
|
|
0 [this is a regular sentence]
|
|
|
|
|
1 [https:, , docs.python.org, 3, tutorial, index...
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
When using ``expand=True``, the split elements will expand out into
|
|
|
|
|
separate columns. If NaN is present, it is propagated throughout
|
|
|
|
|
the columns during the split.
|
|
|
|
|
|
|
|
|
|
>>> s.str.split(n=4, expand=True)
|
|
|
|
|
0 1 2 3 4
|
|
|
|
|
0 this is a regular sentence
|
|
|
|
|
1 https://docs.python.org/3/tutorial/index.html None None None None
|
|
|
|
|
2 None None None None None
|
|
|
|
|
|
|
|
|
|
For slightly more complex use cases like splitting the html document name
|
|
|
|
|
from a url, a combination of parameter settings can be used.
|
|
|
|
|
|
|
|
|
|
>>> s.str.rsplit("/", n=1, expand=True)
|
|
|
|
|
0 1
|
|
|
|
|
0 this is a regular sentence None
|
|
|
|
|
1 https://docs.python.org/3/tutorial index.html
|
|
|
|
|
2 None None
|
|
|
|
|
|
|
|
|
|
Remember to escape special characters when explicitly using regular
|
|
|
|
|
expressions.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["1+1=2"])
|
[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
|
|
|
|
>>> s.str.split(r"\\+|=", n=2, expand=True)
|
|
|
|
|
0 1 2
|
|
|
|
|
0 1 1 2
|
|
|
|
|
"""
|
|
|
|
|
from pyspark.pandas.frame import DataFrame
|
|
|
|
|
|
|
|
|
|
if expand and n <= 0:
|
|
|
|
|
raise NotImplementedError("expand=True is currently only supported with n > 0.")
|
|
|
|
|
|
|
|
|
|
# type hint does not support to specify array type yet.
|
[SPARK-35638][PYTHON] Introduce InternalField to manage dtypes and StructFields
### What changes were proposed in this pull request?
Introduces `InternalField` to manage dtypes and `StructField`s.
`InternalFrame` is already managing dtypes, but when it checks the Spark's data types, column names, and nullabilities, it tries to run the analysis phase each time it needs, which will cause a performance issue.
It will use `InternalField` class which stores the retrieved Spark's data types, column names, and nullabilities, and reuse them. Also, in case those can be known, just update and reuse them without asking Spark.
### Why are the changes needed?
Currently there are some performance issues in the pandas-on-Spark layer.
One of them is accessing Java DataFrame and run analysis phase too many times, especially just for retrieving the current column names or data types.
We should reduce the amount of unnecessary access.
### Does this PR introduce _any_ user-facing change?
Improves the performance in pandas-on-Spark layer:
```py
df = ps.read_parquet("/path/to/test.parquet") # contains ~75 columns
df = df[(df["col"] > 0) & (df["col"] < 10000)]
```
Before the PR, it took about **2.15 sec** and after **1.15 sec**.
### How was this patch tested?
Existing tests.
Closes #32775 from ueshin/issues/SPARK-35638/field.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-07 22:57:28 -04:00
|
|
|
|
return_type = ArrayType(StringType(), containsNull=True)
|
2021-06-14 22:17:56 -04:00
|
|
|
|
|
|
|
|
|
@pandas_udf(returnType=return_type) # type: ignore
|
|
|
|
|
def pudf(s: pd.Series) -> pd.Series:
|
|
|
|
|
return s.str.split(pat, n)
|
|
|
|
|
|
[SPARK-35638][PYTHON] Introduce InternalField to manage dtypes and StructFields
### What changes were proposed in this pull request?
Introduces `InternalField` to manage dtypes and `StructField`s.
`InternalFrame` is already managing dtypes, but when it checks the Spark's data types, column names, and nullabilities, it tries to run the analysis phase each time it needs, which will cause a performance issue.
It will use `InternalField` class which stores the retrieved Spark's data types, column names, and nullabilities, and reuse them. Also, in case those can be known, just update and reuse them without asking Spark.
### Why are the changes needed?
Currently there are some performance issues in the pandas-on-Spark layer.
One of them is accessing Java DataFrame and run analysis phase too many times, especially just for retrieving the current column names or data types.
We should reduce the amount of unnecessary access.
### Does this PR introduce _any_ user-facing change?
Improves the performance in pandas-on-Spark layer:
```py
df = ps.read_parquet("/path/to/test.parquet") # contains ~75 columns
df = df[(df["col"] > 0) & (df["col"] < 10000)]
```
Before the PR, it took about **2.15 sec** and after **1.15 sec**.
### How was this patch tested?
Existing tests.
Closes #32775 from ueshin/issues/SPARK-35638/field.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-07 22:57:28 -04:00
|
|
|
|
psser = self._data._with_new_scol(
|
|
|
|
|
pudf(self._data.spark.column).alias(self._data._internal.data_spark_column_names[0]),
|
|
|
|
|
field=self._data._internal.data_fields[0].copy(spark_type=return_type, nullable=True),
|
|
|
|
|
)
|
[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
|
|
|
|
|
|
|
|
|
if expand:
|
2021-05-20 18:08:30 -04:00
|
|
|
|
psdf = psser.to_frame()
|
|
|
|
|
scol = psdf._internal.data_spark_columns[0]
|
[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
|
|
|
|
spark_columns = [scol[i].alias(str(i)) for i in range(n + 1)]
|
|
|
|
|
column_labels = [(i,) for i in range(n + 1)]
|
2021-05-20 18:08:30 -04:00
|
|
|
|
internal = psdf._internal.with_new_columns(
|
[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
|
|
|
|
spark_columns,
|
|
|
|
|
column_labels=cast(Optional[List], column_labels),
|
[SPARK-35638][PYTHON] Introduce InternalField to manage dtypes and StructFields
### What changes were proposed in this pull request?
Introduces `InternalField` to manage dtypes and `StructField`s.
`InternalFrame` is already managing dtypes, but when it checks the Spark's data types, column names, and nullabilities, it tries to run the analysis phase each time it needs, which will cause a performance issue.
It will use `InternalField` class which stores the retrieved Spark's data types, column names, and nullabilities, and reuse them. Also, in case those can be known, just update and reuse them without asking Spark.
### Why are the changes needed?
Currently there are some performance issues in the pandas-on-Spark layer.
One of them is accessing Java DataFrame and run analysis phase too many times, especially just for retrieving the current column names or data types.
We should reduce the amount of unnecessary access.
### Does this PR introduce _any_ user-facing change?
Improves the performance in pandas-on-Spark layer:
```py
df = ps.read_parquet("/path/to/test.parquet") # contains ~75 columns
df = df[(df["col"] > 0) & (df["col"] < 10000)]
```
Before the PR, it took about **2.15 sec** and after **1.15 sec**.
### How was this patch tested?
Existing tests.
Closes #32775 from ueshin/issues/SPARK-35638/field.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-07 22:57:28 -04:00
|
|
|
|
data_fields=[
|
|
|
|
|
self._data._internal.data_fields[0].copy(name=str(i), nullable=True)
|
|
|
|
|
for i in range(n + 1)
|
|
|
|
|
],
|
[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
|
|
|
|
)
|
|
|
|
|
return DataFrame(internal)
|
|
|
|
|
else:
|
2021-05-20 18:08:30 -04:00
|
|
|
|
return psser
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def rsplit(
|
|
|
|
|
self, pat: Optional[str] = None, n: int = -1, expand: bool = False
|
|
|
|
|
) -> Union["ps.Series", "ps.DataFrame"]:
|
[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
|
|
|
|
"""
|
|
|
|
|
Split strings around given separator/delimiter.
|
|
|
|
|
|
|
|
|
|
Splits the string in the Series from the end, at the specified
|
|
|
|
|
delimiter string. Equivalent to :func:`str.rsplit`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
pat : str, optional
|
|
|
|
|
String or regular expression to split on. If not specified, split
|
|
|
|
|
on whitespace.
|
|
|
|
|
n : int, default -1 (all)
|
|
|
|
|
Limit number of splits in output. None, 0 and -1 will be
|
|
|
|
|
interpreted as return all splits.
|
|
|
|
|
expand : bool, default False
|
|
|
|
|
Expand the splitted strings into separate columns.
|
|
|
|
|
|
|
|
|
|
* If ``True``, `n` must be a positive integer, and return DataFrame expanding
|
|
|
|
|
dimensionality.
|
|
|
|
|
* If ``False``, return Series, containing lists of strings.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series, DataFrame
|
|
|
|
|
Type matches caller unless `expand=True` (see Notes).
|
|
|
|
|
|
|
|
|
|
See Also
|
|
|
|
|
--------
|
|
|
|
|
str.split : Split strings around given separator/delimiter.
|
|
|
|
|
str.join : Join lists contained as elements in the Series/Index
|
|
|
|
|
with passed delimiter.
|
|
|
|
|
|
|
|
|
|
Notes
|
|
|
|
|
-----
|
|
|
|
|
The handling of the `n` keyword depends on the number of found splits:
|
|
|
|
|
|
|
|
|
|
- If found splits > `n`, make first `n` splits only
|
|
|
|
|
- If found splits <= `n`, make all splits
|
|
|
|
|
- If for a certain row the number of found splits < `n`,
|
|
|
|
|
append `None` for padding up to `n` if ``expand=True``
|
|
|
|
|
|
|
|
|
|
If using ``expand=True``, Series callers return DataFrame objects with `n + 1` columns.
|
|
|
|
|
|
|
|
|
|
.. note:: Even if `n` is much larger than found splits, the number of columns does NOT
|
|
|
|
|
shrink unlike pandas.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["this is a regular sentence",
|
[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
|
|
|
|
... "https://docs.python.org/3/tutorial/index.html",
|
|
|
|
|
... np.nan])
|
|
|
|
|
|
|
|
|
|
In the default setting, the string is split by whitespace.
|
|
|
|
|
|
|
|
|
|
>>> s.str.split()
|
|
|
|
|
0 [this, is, a, regular, sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Without the n parameter, the outputs of rsplit and split are identical.
|
|
|
|
|
|
|
|
|
|
>>> s.str.rsplit()
|
|
|
|
|
0 [this, is, a, regular, sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
The n parameter can be used to limit the number of splits on the
|
|
|
|
|
delimiter. The outputs of split and rsplit are different.
|
|
|
|
|
|
|
|
|
|
>>> s.str.split(n=2)
|
|
|
|
|
0 [this, is, a regular sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
>>> s.str.rsplit(n=2)
|
|
|
|
|
0 [this is a, regular, sentence]
|
|
|
|
|
1 [https://docs.python.org/3/tutorial/index.html]
|
|
|
|
|
2 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
When using ``expand=True``, the split elements will expand out into
|
|
|
|
|
separate columns. If NaN is present, it is propagated throughout
|
|
|
|
|
the columns during the split.
|
|
|
|
|
|
|
|
|
|
>>> s.str.split(n=4, expand=True)
|
|
|
|
|
0 1 2 3 4
|
|
|
|
|
0 this is a regular sentence
|
|
|
|
|
1 https://docs.python.org/3/tutorial/index.html None None None None
|
|
|
|
|
2 None None None None None
|
|
|
|
|
|
|
|
|
|
For slightly more complex use cases like splitting the html document name
|
|
|
|
|
from a url, a combination of parameter settings can be used.
|
|
|
|
|
|
|
|
|
|
>>> s.str.rsplit("/", n=1, expand=True)
|
|
|
|
|
0 1
|
|
|
|
|
0 this is a regular sentence None
|
|
|
|
|
1 https://docs.python.org/3/tutorial index.html
|
|
|
|
|
2 None None
|
|
|
|
|
|
|
|
|
|
Remember to escape special characters when explicitly using regular
|
|
|
|
|
expressions.
|
|
|
|
|
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["1+1=2"])
|
[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
|
|
|
|
>>> s.str.split(r"\\+|=", n=2, expand=True)
|
|
|
|
|
0 1 2
|
|
|
|
|
0 1 1 2
|
|
|
|
|
"""
|
|
|
|
|
from pyspark.pandas.frame import DataFrame
|
|
|
|
|
|
|
|
|
|
if expand and n <= 0:
|
|
|
|
|
raise NotImplementedError("expand=True is currently only supported with n > 0.")
|
|
|
|
|
|
|
|
|
|
# type hint does not support to specify array type yet.
|
[SPARK-35638][PYTHON] Introduce InternalField to manage dtypes and StructFields
### What changes were proposed in this pull request?
Introduces `InternalField` to manage dtypes and `StructField`s.
`InternalFrame` is already managing dtypes, but when it checks the Spark's data types, column names, and nullabilities, it tries to run the analysis phase each time it needs, which will cause a performance issue.
It will use `InternalField` class which stores the retrieved Spark's data types, column names, and nullabilities, and reuse them. Also, in case those can be known, just update and reuse them without asking Spark.
### Why are the changes needed?
Currently there are some performance issues in the pandas-on-Spark layer.
One of them is accessing Java DataFrame and run analysis phase too many times, especially just for retrieving the current column names or data types.
We should reduce the amount of unnecessary access.
### Does this PR introduce _any_ user-facing change?
Improves the performance in pandas-on-Spark layer:
```py
df = ps.read_parquet("/path/to/test.parquet") # contains ~75 columns
df = df[(df["col"] > 0) & (df["col"] < 10000)]
```
Before the PR, it took about **2.15 sec** and after **1.15 sec**.
### How was this patch tested?
Existing tests.
Closes #32775 from ueshin/issues/SPARK-35638/field.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-07 22:57:28 -04:00
|
|
|
|
return_type = ArrayType(StringType(), containsNull=True)
|
2021-06-14 22:17:56 -04:00
|
|
|
|
|
|
|
|
|
@pandas_udf(returnType=return_type) # type: ignore
|
|
|
|
|
def pudf(s: pd.Series) -> pd.Series:
|
|
|
|
|
return s.str.rsplit(pat, n)
|
|
|
|
|
|
[SPARK-35638][PYTHON] Introduce InternalField to manage dtypes and StructFields
### What changes were proposed in this pull request?
Introduces `InternalField` to manage dtypes and `StructField`s.
`InternalFrame` is already managing dtypes, but when it checks the Spark's data types, column names, and nullabilities, it tries to run the analysis phase each time it needs, which will cause a performance issue.
It will use `InternalField` class which stores the retrieved Spark's data types, column names, and nullabilities, and reuse them. Also, in case those can be known, just update and reuse them without asking Spark.
### Why are the changes needed?
Currently there are some performance issues in the pandas-on-Spark layer.
One of them is accessing Java DataFrame and run analysis phase too many times, especially just for retrieving the current column names or data types.
We should reduce the amount of unnecessary access.
### Does this PR introduce _any_ user-facing change?
Improves the performance in pandas-on-Spark layer:
```py
df = ps.read_parquet("/path/to/test.parquet") # contains ~75 columns
df = df[(df["col"] > 0) & (df["col"] < 10000)]
```
Before the PR, it took about **2.15 sec** and after **1.15 sec**.
### How was this patch tested?
Existing tests.
Closes #32775 from ueshin/issues/SPARK-35638/field.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-07 22:57:28 -04:00
|
|
|
|
psser = self._data._with_new_scol(
|
|
|
|
|
pudf(self._data.spark.column).alias(self._data._internal.data_spark_column_names[0]),
|
|
|
|
|
field=self._data._internal.data_fields[0].copy(spark_type=return_type, nullable=True),
|
|
|
|
|
)
|
[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
|
|
|
|
|
|
|
|
|
if expand:
|
2021-05-20 18:08:30 -04:00
|
|
|
|
psdf = psser.to_frame()
|
|
|
|
|
scol = psdf._internal.data_spark_columns[0]
|
[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
|
|
|
|
spark_columns = [scol[i].alias(str(i)) for i in range(n + 1)]
|
|
|
|
|
column_labels = [(i,) for i in range(n + 1)]
|
2021-05-20 18:08:30 -04:00
|
|
|
|
internal = psdf._internal.with_new_columns(
|
[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
|
|
|
|
spark_columns,
|
|
|
|
|
column_labels=cast(Optional[List], column_labels),
|
[SPARK-35638][PYTHON] Introduce InternalField to manage dtypes and StructFields
### What changes were proposed in this pull request?
Introduces `InternalField` to manage dtypes and `StructField`s.
`InternalFrame` is already managing dtypes, but when it checks the Spark's data types, column names, and nullabilities, it tries to run the analysis phase each time it needs, which will cause a performance issue.
It will use `InternalField` class which stores the retrieved Spark's data types, column names, and nullabilities, and reuse them. Also, in case those can be known, just update and reuse them without asking Spark.
### Why are the changes needed?
Currently there are some performance issues in the pandas-on-Spark layer.
One of them is accessing Java DataFrame and run analysis phase too many times, especially just for retrieving the current column names or data types.
We should reduce the amount of unnecessary access.
### Does this PR introduce _any_ user-facing change?
Improves the performance in pandas-on-Spark layer:
```py
df = ps.read_parquet("/path/to/test.parquet") # contains ~75 columns
df = df[(df["col"] > 0) & (df["col"] < 10000)]
```
Before the PR, it took about **2.15 sec** and after **1.15 sec**.
### How was this patch tested?
Existing tests.
Closes #32775 from ueshin/issues/SPARK-35638/field.
Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-06-07 22:57:28 -04:00
|
|
|
|
data_fields=[
|
|
|
|
|
self._data._internal.data_fields[0].copy(name=str(i), nullable=True)
|
|
|
|
|
for i in range(n + 1)
|
|
|
|
|
],
|
[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
|
|
|
|
)
|
|
|
|
|
return DataFrame(internal)
|
|
|
|
|
else:
|
2021-05-20 18:08:30 -04:00
|
|
|
|
return psser
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def translate(self, table: Dict) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Map all characters in the string through the given mapping table.
|
|
|
|
|
Equivalent to standard :func:`str.translate`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
table : dict
|
|
|
|
|
Table is a mapping of Unicode ordinals to Unicode ordinals,
|
|
|
|
|
strings, or None. Unmapped characters are left untouched.
|
|
|
|
|
Characters mapped to None are deleted. :func:`str.maketrans` is a
|
|
|
|
|
helper function for making translation tables.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Series with translated strings.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(["dog", "cat", "bird"])
|
[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
|
|
|
|
>>> m = str.maketrans({'a': 'X', 'i': 'Y', 'o': None})
|
|
|
|
|
>>> s.str.translate(m)
|
|
|
|
|
0 dg
|
|
|
|
|
1 cXt
|
|
|
|
|
2 bYrd
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_translate(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.translate(table)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_translate)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def wrap(self, width: int, **kwargs: bool) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Wrap long strings in the Series to be formatted in paragraphs with
|
|
|
|
|
length less than a given width.
|
|
|
|
|
|
|
|
|
|
This method has the same keyword parameters and defaults as
|
|
|
|
|
:class:`textwrap.TextWrapper`.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
width : int
|
|
|
|
|
Maximum line-width. Lines separated with newline char.
|
|
|
|
|
expand_tabs : bool, optional
|
|
|
|
|
If true, tab characters will be expanded to spaces (default: True).
|
|
|
|
|
replace_whitespace : bool, optional
|
|
|
|
|
If true, each whitespace character remaining after tab expansion
|
|
|
|
|
will be replaced by a single space (default: True).
|
|
|
|
|
drop_whitespace : bool, optional
|
|
|
|
|
If true, whitespace that, after wrapping, happens to end up at the
|
|
|
|
|
beginning or end of a line is dropped (default: True).
|
|
|
|
|
break_long_words : bool, optional
|
|
|
|
|
If true, then words longer than width will be broken in order to
|
|
|
|
|
ensure that no lines are longer than width. If it is false, long
|
|
|
|
|
words will not be broken, and some lines may be longer than width
|
|
|
|
|
(default: True).
|
|
|
|
|
break_on_hyphens : bool, optional
|
|
|
|
|
If true, wrapping will occur preferably on whitespace and right
|
|
|
|
|
after hyphens in compound words, as it is customary in English.
|
|
|
|
|
If false, only whitespaces will be considered as potentially good
|
|
|
|
|
places for line breaks, but you need to set break_long_words to
|
|
|
|
|
false if you want truly insecable words (default: True).
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Series with wrapped strings.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['line to be wrapped', 'another line to be wrapped'])
|
[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
|
|
|
|
>>> s.str.wrap(12)
|
|
|
|
|
0 line to be\\nwrapped
|
|
|
|
|
1 another line\\nto be\\nwrapped
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_wrap(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.wrap(width, **kwargs)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_wrap)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def zfill(self, width: int) -> "ps.Series":
|
[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
|
|
|
|
"""
|
|
|
|
|
Pad strings in the Series by prepending ‘0’ characters.
|
|
|
|
|
|
|
|
|
|
Strings in the Series are padded with ‘0’ characters on the left of the
|
|
|
|
|
string to reach a total string length width. Strings in the Series with
|
|
|
|
|
length greater or equal to width are unchanged.
|
|
|
|
|
|
|
|
|
|
Differs from :func:`str.zfill` which has special handling for ‘+’/’-‘
|
|
|
|
|
in the string.
|
|
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
|
----------
|
|
|
|
|
width : int
|
|
|
|
|
Minimum length of resulting string; strings with length less than
|
|
|
|
|
width be prepended with ‘0’ characters.
|
|
|
|
|
|
|
|
|
|
Returns
|
|
|
|
|
-------
|
|
|
|
|
Series of object
|
|
|
|
|
Series with '0' left-padded strings.
|
|
|
|
|
|
|
|
|
|
Examples
|
|
|
|
|
--------
|
2021-04-11 22:18:08 -04:00
|
|
|
|
>>> s = ps.Series(['-1', '1', '1000', np.nan])
|
[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
|
|
|
|
>>> s
|
|
|
|
|
0 -1
|
|
|
|
|
1 1
|
|
|
|
|
2 1000
|
|
|
|
|
3 None
|
|
|
|
|
dtype: object
|
|
|
|
|
|
|
|
|
|
Note that NaN is not a string, therefore it is converted to NaN. The
|
|
|
|
|
minus sign in '-1' is treated as a regular character and the zero is
|
|
|
|
|
added to the left of it (:func:`str.zfill` would have moved it to the
|
|
|
|
|
left). 1000 remains unchanged as it is longer than width.
|
|
|
|
|
|
|
|
|
|
>>> s.str.zfill(3)
|
|
|
|
|
0 0-1
|
|
|
|
|
1 001
|
|
|
|
|
2 1000
|
|
|
|
|
3 None
|
|
|
|
|
dtype: object
|
|
|
|
|
"""
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
2021-04-11 22:18:08 -04:00
|
|
|
|
def pandas_zfill(s) -> "ps.Series[str]":
|
[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
|
|
|
|
return s.str.zfill(width)
|
|
|
|
|
|
2021-05-31 21:33:10 -04:00
|
|
|
|
return self._data.pandas_on_spark.transform_batch(pandas_zfill)
|
[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
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
@no_type_check
|
|
|
|
|
def get_dummies(self, sep: str = "|") -> "ps.DataFrame":
|
[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
|
|
|
|
"""
|
|
|
|
|
Not supported.
|
|
|
|
|
"""
|
|
|
|
|
raise NotImplementedError()
|
2021-04-07 07:50:41 -04:00
|
|
|
|
|
|
|
|
|
|
2021-05-21 14:03:35 -04:00
|
|
|
|
def _test() -> None:
|
2021-04-07 07:50:41 -04:00
|
|
|
|
import os
|
|
|
|
|
import doctest
|
|
|
|
|
import sys
|
|
|
|
|
from pyspark.sql import SparkSession
|
|
|
|
|
import pyspark.pandas.strings
|
|
|
|
|
|
|
|
|
|
os.chdir(os.environ["SPARK_HOME"])
|
|
|
|
|
|
|
|
|
|
globs = pyspark.pandas.strings.__dict__.copy()
|
2021-04-11 22:18:08 -04:00
|
|
|
|
globs["ps"] = pyspark.pandas
|
2021-04-07 07:50:41 -04:00
|
|
|
|
spark = (
|
|
|
|
|
SparkSession.builder.master("local[4]")
|
|
|
|
|
.appName("pyspark.pandas.strings tests")
|
|
|
|
|
.getOrCreate()
|
|
|
|
|
)
|
|
|
|
|
(failure_count, test_count) = doctest.testmod(
|
|
|
|
|
pyspark.pandas.strings,
|
|
|
|
|
globs=globs,
|
|
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE,
|
|
|
|
|
)
|
|
|
|
|
spark.stop()
|
|
|
|
|
if failure_count:
|
|
|
|
|
sys.exit(-1)
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
|
_test()
|