Commit graph

230 commits

Author SHA1 Message Date
HyukjinKwon 6fb22aa42d
[SPARK-31748][PYTHON] Document resource module in PySpark doc and rename/move classes
### What changes were proposed in this pull request?

This PR is kind of a followup for SPARK-29641 and SPARK-28234. This PR proposes:

1.. Document the new `pyspark.resource` module introduced at 95aec091e4, in PySpark API docs.

2.. Move classes into fewer and simpler modules

Before:

```
pyspark
├── resource
│   ├── executorrequests.py
│   │   ├── class ExecutorResourceRequest
│   │   └── class ExecutorResourceRequests
│   ├── taskrequests.py
│   │   ├── class TaskResourceRequest
│   │   └── class TaskResourceRequests
│   ├── resourceprofilebuilder.py
│   │   └── class ResourceProfileBuilder
│   ├── resourceprofile.py
│   │   └── class ResourceProfile
└── resourceinformation
    └── class ResourceInformation
```

After:

```
pyspark
└── resource
    ├── requests.py
    │   ├── class ExecutorResourceRequest
    │   ├── class ExecutorResourceRequests
    │   ├── class TaskResourceRequest
    │   └── class TaskResourceRequests
    ├── profile.py
    │   ├── class ResourceProfileBuilder
    │   └── class ResourceProfile
    └── information.py
        └── class ResourceInformation
```

3.. Minor docstring fix e.g.:

```diff
-     param name the name of the resource
-     param addresses an array of strings describing the addresses of the resource
+     :param name: the name of the resource
+     :param addresses: an array of strings describing the addresses of the resource
+
+     .. versionadded:: 3.0.0
```

### Why are the changes needed?

To document APIs, and move Python modules to fewer and simpler modules.

### Does this PR introduce _any_ user-facing change?

No, the changes are in unreleased branches.

### How was this patch tested?

Manually tested via:

```bash
cd python
./run-tests --python-executables=python3 --modules=pyspark-core
./run-tests --python-executables=python3 --modules=pyspark-resource
```

Closes #28569 from HyukjinKwon/SPARK-28234-SPARK-29641-followup.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-19 17:09:37 -07:00
Weichen Xu ee1de66fe4 [SPARK-31549][PYSPARK] Add a develop API invoking collect on Python RDD with user-specified job group
### What changes were proposed in this pull request?
I add a new API in pyspark RDD class:

def collectWithJobGroup(self, groupId, description, interruptOnCancel=False)

This API do the same thing with `rdd.collect`, but it can specify the job group when do collect.
The purpose of adding this API is, if we use:

```
sc.setJobGroup("group-id...")
rdd.collect()
```
The `setJobGroup` API in pyspark won't work correctly. This related to a bug discussed in
https://issues.apache.org/jira/browse/SPARK-31549

Note:

This PR is a rather temporary workaround for `PYSPARK_PIN_THREAD`, and as a step to migrate to  `PYSPARK_PIN_THREAD` smoothly. It targets Spark 3.0.

- `PYSPARK_PIN_THREAD` is unstable at this moment that affects whole PySpark applications.
- It is impossible to make it runtime configuration as it has to be set before JVM is launched.
- There is a thread leak issue between Python and JVM. We should address but it's not a release blocker for Spark 3.0 since the feature is experimental. I plan to handle this after Spark 3.0 due to stability.

Once `PYSPARK_PIN_THREAD` is enabled by default, we should remove this API out ideally. I will target to deprecate this API in Spark 3.1.

### Why are the changes needed?
Fix bug.

### Does this PR introduce any user-facing change?
A develop API in pyspark: `pyspark.RDD. collectWithJobGroup`

### How was this patch tested?
Unit test.

Closes #28395 from WeichenXu123/collect_with_job_group.

Authored-by: Weichen Xu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-01 10:08:16 +09:00
Thomas Graves 95aec091e4 [SPARK-29641][PYTHON][CORE] Stage Level Sched: Add python api's and tests
### What changes were proposed in this pull request?

As part of the Stage level scheduling features, add the Python api's to set resource profiles.
This also adds the functionality to properly apply the pyspark memory configuration when specified in the ResourceProfile. The pyspark memory configuration is being passed in the task local properties. This was an easy way to get it to the PythonRunner that needs it. I modeled this off how the barrier task scheduling is passing the addresses. As part of this I added in the JavaRDD api's because those are needed by python.

### Why are the changes needed?

python api for this feature

### Does this PR introduce any user-facing change?

Yes adds the java and python apis for user to specify a ResourceProfile to use stage level scheduling.

### How was this patch tested?

unit tests and manually tested on yarn. Tests also run to verify it errors properly on standalone and local mode where its not yet supported.

Closes #28085 from tgravescs/SPARK-29641-pr-base.

Lead-authored-by: Thomas Graves <tgraves@nvidia.com>
Co-authored-by: Thomas Graves <tgraves@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-04-23 10:20:39 +09:00
Xianyang Liu 0a7095156b [SPARK-29499][CORE][PYSPARK] Add mapPartitionsWithIndex for RDDBarrier
### What changes were proposed in this pull request?

Add mapPartitionsWithIndex for RDDBarrier.

### Why are the changes needed?

There is only one method in `RDDBarrier`. We often use the partition index as a label for the current partition. We need to get the index from `TaskContext`  index in the method of `mapPartitions` which is not convenient.
### Does this PR introduce any user-facing change?

No

### How was this patch tested?

New UT.

Closes #26148 from ConeyLiu/barrier-index.

Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-10-23 13:46:09 +02:00
Holden Karau 42050c3f4f [SPARK-27659][PYTHON] Allow PySpark to prefetch during toLocalIterator
### What changes were proposed in this pull request?

This PR allows Python toLocalIterator to prefetch the next partition while the first partition is being collected. The PR also adds a demo micro bench mark in the examples directory, we may wish to keep this or not.

### Why are the changes needed?

In https://issues.apache.org/jira/browse/SPARK-23961 / 5e79ae3b40 we changed PySpark to only pull one partition at a time. This is memory efficient, but if partitions take time to compute this can mean we're spending more time blocking.

### Does this PR introduce any user-facing change?

A new param is added to toLocalIterator

### How was this patch tested?

New unit test inside of `test_rdd.py` checks the time that the elements are evaluated at. Another test that the results remain the same are added to `test_dataframe.py`.

I also ran a micro benchmark in the examples directory `prefetch.py` which shows an improvement of ~40% in this specific use case.

>
> 19/08/16 17:11:36 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
> Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
> Setting default log level to "WARN".
> To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
> Running timers:
>
> [Stage 32:>                                                         (0 + 1) / 1]
> Results:
>
> Prefetch time:
>
> 100.228110831
>
>
> Regular time:
>
> 188.341721614
>
>
>

Closes #25515 from holdenk/SPARK-27659-allow-pyspark-tolocalitr-to-prefetch.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2019-09-20 09:59:31 -07:00
Chris Martin 05988b256e [SPARK-27463][PYTHON] Support Dataframe Cogroup via Pandas UDFs
### What changes were proposed in this pull request?

Adds a new cogroup Pandas UDF.  This allows two grouped dataframes to be cogrouped together and apply a (pandas.DataFrame, pandas.DataFrame) -> pandas.DataFrame UDF to each cogroup.

**Example usage**

```
from pyspark.sql.functions import pandas_udf, PandasUDFType
df1 = spark.createDataFrame(
   [(20000101, 1, 1.0), (20000101, 2, 2.0), (20000102, 1, 3.0), (20000102, 2, 4.0)],
   ("time", "id", "v1"))

df2 = spark.createDataFrame(
   [(20000101, 1, "x"), (20000101, 2, "y")],
    ("time", "id", "v2"))

pandas_udf("time int, id int, v1 double, v2 string", PandasUDFType.COGROUPED_MAP)
   def asof_join(l, r):
      return pd.merge_asof(l, r, on="time", by="id")

df1.groupby("id").cogroup(df2.groupby("id")).apply(asof_join).show()

```

        +--------+---+---+---+
        |    time| id| v1| v2|
        +--------+---+---+---+
        |20000101|  1|1.0|  x|
        |20000102|  1|3.0|  x|
        |20000101|  2|2.0|  y|
        |20000102|  2|4.0|  y|
        +--------+---+---+---+

### How was this patch tested?

Added unit test test_pandas_udf_cogrouped_map

Closes #24981 from d80tb7/SPARK-27463-poc-arrow-stream.

Authored-by: Chris Martin <chris@cmartinit.co.uk>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-09-17 17:13:50 -07:00
Sean Owen eb037a8180 [SPARK-28855][CORE][ML][SQL][STREAMING] Remove outdated usages of Experimental, Evolving annotations
### What changes were proposed in this pull request?

The Experimental and Evolving annotations are both (like Unstable) used to express that a an API may change. However there are many things in the code that have been marked that way since even Spark 1.x. Per the dev thread, anything introduced at or before Spark 2.3.0 is pretty much 'stable' in that it would not change without a deprecation cycle. Therefore I'd like to remove most of these annotations. And, remove the `:: Experimental ::` scaladoc tag too. And likewise for Python, R.

The changes below can be summarized as:
- Generally, anything introduced at or before Spark 2.3.0 has been unmarked as neither Evolving nor Experimental
- Obviously experimental items like DSv2, Barrier mode, ExperimentalMethods are untouched
- I _did_ unmark a few MLlib classes introduced in 2.4, as I am quite confident they're not going to change (e.g. KolmogorovSmirnovTest, PowerIterationClustering)

It's a big change to review, so I'd suggest scanning the list of _files_ changed to see if any area seems like it should remain partly experimental and examine those.

### Why are the changes needed?

Many of these annotations are incorrect; the APIs are de facto stable. Leaving them also makes legitimate usages of the annotations less meaningful.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #25558 from srowen/SPARK-28855.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-01 10:15:00 -05:00
wangguangxin.cn fbaa177d2a [MINOR][PYTHON] Use _memory_limit to get worker memory conf in rdd.py
## What changes were proposed in this pull request?

Replace duplicate code by function `_memory_limit`

## How was this patch tested?

Existing UTs

Closes #25273 from WangGuangxin/python_memory_limit.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-27 11:58:50 -07:00
Jesse Cai 79e2047703 [SPARK-28355][CORE][PYTHON] Use Spark conf for threshold at which command is compressed by broadcast
## What changes were proposed in this pull request?

The `_prepare_for_python_RDD` method currently broadcasts a pickled command if its length is greater than the hardcoded value `1 << 20` (1M). This change sets this value as a Spark conf instead.

## How was this patch tested?

Unit tests, manual tests.

Closes #25123 from jessecai/SPARK-28355.

Authored-by: Jesse Cai <jesse.cai@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-13 08:44:16 -07:00
HyukjinKwon fe75ff8bea [SPARK-28206][PYTHON] Remove the legacy Epydoc in PySpark API documentation
## What changes were proposed in this pull request?

Seems like we used to generate PySpark API documentation by Epydoc almost at the very first place (see 85b8f2c64f).

This fixes an actual issue:

Before:

![Screen Shot 2019-07-05 at 8 20 01 PM](https://user-images.githubusercontent.com/6477701/60720491-e9879180-9f65-11e9-9562-100830a456cd.png)

After:

![Screen Shot 2019-07-05 at 8 20 05 PM](https://user-images.githubusercontent.com/6477701/60720495-ec828200-9f65-11e9-8277-8f689e292cb0.png)

It seems apparently a bug within `epytext` plugin during the conversion between`param` and `:param` syntax. See also [Epydoc syntax](http://epydoc.sourceforge.net/manual-epytext.html).

Actually, Epydoc syntax violates [PEP-257](https://www.python.org/dev/peps/pep-0257/) IIRC and blocks us to enable some rules for doctest linter as well.

We should remove this legacy away and I guess Spark 3 is good timing to do it.

## How was this patch tested?

Manually built the doc and check each.

I had to manually find the Epydoc syntax by `git grep -r "{L"`, for instance.

Closes #25060 from HyukjinKwon/SPARK-28206.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-07-05 10:08:22 -07:00
HyukjinKwon 02f4763286 [SPARK-28198][PYTHON] Add mapPartitionsInPandas to allow an iterator of DataFrames
## What changes were proposed in this pull request?

This PR proposes to add `mapPartitionsInPandas` API to DataFrame by using existing `SCALAR_ITER` as below:

1. Filtering via setting the column

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType

df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age"))

pandas_udf(df.schema, PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
    for pdf in iterator:
        yield pdf[pdf.id == 1]

df.mapPartitionsInPandas(filter_func).show()
```

```
+---+---+
| id|age|
+---+---+
|  1| 21|
+---+---+
```

2. `DataFrame.loc`

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
import pandas as pd

df = spark.createDataFrame([['aa'], ['bb'], ['cc'], ['aa'], ['aa'], ['aa']], ["value"])

pandas_udf(df.schema, PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
    for pdf in iterator:
        yield pdf.loc[pdf.value.str.contains('^a'), :]

df.mapPartitionsInPandas(filter_func).show()
```

```
+-----+
|value|
+-----+
|   aa|
|   aa|
|   aa|
|   aa|
+-----+
```

3. `pandas.melt`

```python
from pyspark.sql.functions import pandas_udf, PandasUDFType
import pandas as pd

df = spark.createDataFrame(
    pd.DataFrame({'A': {0: 'a', 1: 'b', 2: 'c'},
                  'B': {0: 1, 1: 3, 2: 5},
                  'C': {0: 2, 1: 4, 2: 6}}))

pandas_udf("A string, variable string, value long", PandasUDFType.SCALAR_ITER)
def filter_func(iterator):
    for pdf in iterator:
        import pandas as pd
        yield pd.melt(pdf, id_vars=['A'], value_vars=['B', 'C'])

df.mapPartitionsInPandas(filter_func).show()
```

```
+---+--------+-----+
|  A|variable|value|
+---+--------+-----+
|  a|       B|    1|
|  a|       C|    2|
|  b|       B|    3|
|  b|       C|    4|
|  c|       B|    5|
|  c|       C|    6|
+---+--------+-----+
```

The current limitation of `SCALAR_ITER` is that it doesn't allow different length of result, which is pretty critical in practice - for instance, we cannot simply filter by using Pandas APIs but we merely just map N to N. This PR allows map N to M like flatMap.

This API mimics the way of `mapPartitions` but keeps API shape of `SCALAR_ITER` by allowing different results.

### How does this PR implement?

This PR adds mimics both `dapply` with Arrow optimization and Grouped Map Pandas UDF. At Python execution side, it reuses existing `SCALAR_ITER` code path.

Therefore, externally, we don't introduce any new type of Pandas UDF but internally we use another evaluation type code `205` (`SQL_MAP_PANDAS_ITER_UDF`).

This approach is similar with Pandas' Windows function implementation with Grouped Aggregation Pandas UDF functions - internally we have `203` (`SQL_WINDOW_AGG_PANDAS_UDF`) but externally we just share the same `GROUPED_AGG`.

## How was this patch tested?

Manually tested and unittests were added.

Closes #24997 from HyukjinKwon/scalar-udf-iter.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-02 10:54:16 +09:00
Bryan Cutler c277afb12b [SPARK-27992][PYTHON] Allow Python to join with connection thread to propagate errors
## What changes were proposed in this pull request?

Currently with `toLocalIterator()` and `toPandas()` with Arrow enabled, if the Spark job being run in the background serving thread errors, it will be caught and sent to Python through the PySpark serializer.
This is not the ideal solution because it is only catch a SparkException, it won't handle an error that occurs in the serializer, and each method has to have it's own special handling to propagate the error.

This PR instead returns the Python Server object along with the serving port and authentication info, so that it allows the Python caller to join with the serving thread. During the call to join, the serving thread Future is completed either successfully or with an exception. In the latter case, the exception will be propagated to Python through the Py4j call.

## How was this patch tested?

Existing tests

Closes #24834 from BryanCutler/pyspark-propagate-server-error-SPARK-27992.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-06-26 13:05:41 -07:00
WeichenXu 6d441dcdc6 [SPARK-26412][PYSPARK][SQL] Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series
## What changes were proposed in this pull request?

Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series.
Note the UDF input args will be always one iterator:
* if the udf take only column as input, the iterator's element will be pd.Series (corresponding to the column values batch)
* if the udf take multiple columns as inputs, the iterator's element will be a tuple composed of multiple `pd.Series`s, each one corresponding to the multiple columns as inputs (keep the same order). For example:
```
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def the_udf(iterator):
    for col1_batch, col2_batch in iterator:
        yield col1_batch + col2_batch

df.select(the_udf("col1", "col2"))
```
The udf above will add col1 and col2.

I haven't add unit tests, but manually tests show it works fine. So it is ready for first pass review.
We can test several typical cases:

```
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import udf
from pyspark.taskcontext import TaskContext

df = spark.createDataFrame([(1, 20), (3, 40)], ["a", "b"])

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi1: do init stuff, partitionId=" + str(pid))
    for batch in it:
        yield batch + 100
    print("DBG: fi1: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi2(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi2: do init stuff, partitionId=" + str(pid))
    for batch in it:
        yield batch + 10000
    print("DBG: fi2: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi3(it):
    pid = TaskContext.get().partitionId()
    print("DBG: fi3: do init stuff, partitionId=" + str(pid))
    for x, y in it:
        yield x + y * 10 + 100000
    print("DBG: fi3: do close stuff, partitionId=" + str(pid))

pandas_udf("int", PandasUDFType.SCALAR)
def fp1(x):
    return x + 1000

udf("int")
def fu1(x):
    return x + 10

# test select "pandas iter udf/pandas udf/sql udf" expressions at the same time.
# Note this case the `fi1("a"), fi2("b"), fi3("a", "b")` will generate only one plan,
# and `fu1("a")`, `fp1("a")` will generate another two separate plans.
df.select(fi1("a"), fi2("b"), fi3("a", "b"), fu1("a"), fp1("a")).show()

# test chain two pandas iter udf together
# Note this case `fi2(fi1("a"))` will generate only one plan
# Also note the init stuff/close stuff call order will be like:
# (debug output following)
#     DBG: fi2: do init stuff, partitionId=0
#     DBG: fi1: do init stuff, partitionId=0
#     DBG: fi1: do close stuff, partitionId=0
#     DBG: fi2: do close stuff, partitionId=0
df.select(fi2(fi1("a"))).show()

# test more complex chain
# Note this case `fi1("a"), fi2("a")` will generate one plan,
# and `fi3(fi1_output, fi2_output)` will generate another plan
df.select(fi3(fi1("a"), fi2("a"))).show()
```

## How was this patch tested?

To be added.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #24643 from WeichenXu123/pandas_udf_iter.

Lead-authored-by: WeichenXu <weichen.xu@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-15 08:29:20 -07:00
Bryan Cutler 5e79ae3b40 [SPARK-23961][SPARK-27548][PYTHON] Fix error when toLocalIterator goes out of scope and properly raise errors from worker
## What changes were proposed in this pull request?

This fixes an error when a PySpark local iterator, for both RDD and DataFrames, goes out of scope and the connection is closed before fully consuming the iterator. The error occurs on the JVM in the serving thread, when Python closes the local socket while the JVM is writing to it. This usually happens when there is enough data to fill the socket read buffer, causing the write call to block.

Additionally, this fixes a problem when an error occurs in the Python worker and the collect job is cancelled with an exception. Previously, the Python driver was never notified of the error so the user could get a partial result (iteration until the error) and the application will continue. With this change, an error in the worker is sent to the Python iterator and is then raised.

The change here introduces a protocol for PySpark local iterators that work as follows:

1) The local socket connection is made when the iterator is created
2) When iterating, Python first sends a request for partition data as a non-zero integer
3) While the JVM local iterator over partitions has next, it triggers a job to collect the next partition
4) The JVM sends a nonzero response to indicate it has the next partition to send
5) The next partition is sent to Python and read by the PySpark deserializer
6) After sending the entire partition, an `END_OF_DATA_SECTION` is sent to Python which stops the deserializer and allows to make another request
7) When the JVM gets a request from Python but has already consumed it's local iterator, it will send a zero response to Python and both will close the socket cleanly
8) If an error occurs in the worker, a negative response is sent to Python followed by the error message. Python will then raise a RuntimeError with the message, stopping iteration.
9) When the PySpark local iterator is garbage-collected, it will read any remaining data from the current partition (this is data that has already been collected) and send a request of zero to tell the JVM to stop collection jobs and close the connection.

Steps 1, 3, 5, 6 are the same as before. Step 8 was completely missing before because errors in the worker were never communicated back to Python. The other steps add synchronization to allow for a clean closing of the socket, with a small trade-off in performance for each partition. This is mainly because the JVM does not start collecting partition data until it receives a request to do so, where before it would eagerly write all data until the socket receive buffer is full.

## How was this patch tested?

Added new unit tests for DataFrame and RDD `toLocalIterator` and tested not fully consuming the iterator. Manual tests with Python 2.7  and 3.6.

Closes #24070 from BryanCutler/pyspark-toLocalIterator-clean-stop-SPARK-23961.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-05-07 14:47:39 -07:00
Sean Owen 8171b156eb [SPARK-26771][CORE][GRAPHX] Make .unpersist(), .destroy() consistently non-blocking by default
## What changes were proposed in this pull request?

Make .unpersist(), .destroy() non-blocking by default and adjust callers to request blocking only where important.

This also adds an optional blocking argument to Pyspark's RDD.unpersist(), which never had one.

## How was this patch tested?

Existing tests.

Closes #23685 from srowen/SPARK-26771.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-01 18:29:55 -06:00
Hyukjin Kwon c08021cd87 [SPARK-26776][PYTHON] Reduce Py4J communication cost in PySpark's execution barrier check
## What changes were proposed in this pull request?

I am investigating flaky tests. I realised that:

```
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/rdd.py", line 2512, in __init__
        self.is_barrier = prev._is_barrier() or isFromBarrier
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/rdd.py", line 2412, in _is_barrier
        return self._jrdd.rdd().isBarrier()
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
        answer, self.gateway_client, self.target_id, self.name)
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 342, in get_return_value
        return OUTPUT_CONVERTER[type](answer[2:], gateway_client)
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 2492, in <lambda>
        lambda target_id, gateway_client: JavaObject(target_id, gateway_client))
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1324, in __init__
        ThreadSafeFinalizer.add_finalizer(key, value)
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/finalizer.py", line 43, in add_finalizer
        cls.finalizers[id] = weak_ref
      File "/usr/lib64/pypy-2.5.1/lib-python/2.7/threading.py", line 216, in __exit__
        self.release()
      File "/usr/lib64/pypy-2.5.1/lib-python/2.7/threading.py", line 208, in release
        self.__block.release()
    error: release unlocked lock
```

I assume it might not be directly related with the test itself but I noticed that it `prev._is_barrier()` attempts to access via Py4J.

Accessing via Py4J is expensive. Therefore, this PR proposes to avoid Py4J access when `isFromBarrier` is `True`.

## How was this patch tested?

Unittests should cover this.

Closes #23690 from HyukjinKwon/minor-barrier.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-30 12:24:27 +08:00
Sean Owen c2d0d700b5 [SPARK-26640][CORE][ML][SQL][STREAMING][PYSPARK] Code cleanup from lgtm.com analysis
## What changes were proposed in this pull request?

Misc code cleanup from lgtm.com analysis. See comments below for details.

## How was this patch tested?

Existing tests.

Closes #23571 from srowen/SPARK-26640.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 19:40:39 -06:00
韩田田00222924 82c1ac48a3 [SPARK-25696] The storage memory displayed on spark Application UI is…
… incorrect.

## What changes were proposed in this pull request?
In the reported heartbeat information, the unit of the memory data is bytes, which is converted by the formatBytes() function in the utils.js file before being displayed in the interface. The cardinality of the unit conversion in the formatBytes function is 1000, which should be 1024.
Change the cardinality of the unit conversion in the formatBytes function to 1024.

## How was this patch tested?
 manual tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Closes #22683 from httfighter/SPARK-25696.

Lead-authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Co-authored-by: han.tiantian@zte.com.cn <han.tiantian@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-10 18:27:01 -06:00
Katrin Leinweber c5daccb1da [MINOR] Update all DOI links to preferred resolver
## What changes were proposed in this pull request?

The DOI foundation recommends [this new resolver](https://www.doi.org/doi_handbook/3_Resolution.html#3.8). Accordingly, this PR re`sed`s all static DOI links ;-)

## How was this patch tested?

It wasn't, since it seems as safe as a "[typo fix](https://spark.apache.org/contributing.html)".

In case any of the files is included from other projects, and should be updated there, please let me know.

Closes #23129 from katrinleinweber/resolve-DOIs-securely.

Authored-by: Katrin Leinweber <9948149+katrinleinweber@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-25 17:43:55 -06:00
Sean Owen 08c76b5d39 [SPARK-25238][PYTHON] lint-python: Fix W605 warnings for pycodestyle 2.4
(This change is a subset of the changes needed for the JIRA; see https://github.com/apache/spark/pull/22231)

## What changes were proposed in this pull request?

Use raw strings and simpler regex syntax consistently in Python, which also avoids warnings from pycodestyle about accidentally relying Python's non-escaping of non-reserved chars in normal strings. Also, fix a few long lines.

## How was this patch tested?

Existing tests, and some manual double-checking of the behavior of regexes in Python 2/3 to be sure.

Closes #22400 from srowen/SPARK-25238.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-13 11:19:43 +08:00
Xiangrui Meng 20b7c684cc [SPARK-25248][.1][PYSPARK] update barrier Python API
## What changes were proposed in this pull request?

I made one pass over the Python APIs for barrier mode and updated them to match the Scala doc in #22240 . Major changes:

* export the public classes
* expand the docs
* add doc for BarrierTaskInfo.addresss

cc: jiangxb1987

Closes #22261 from mengxr/SPARK-25248.1.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-29 07:22:03 -07:00
Imran Rashid 38391c9aa8 [SPARK-25253][PYSPARK] Refactor local connection & auth code
This eliminates some duplication in the code to connect to a server on localhost to talk directly to the jvm.  Also it gives consistent ipv6 and error handling.  Two other incidental changes, that shouldn't matter:
1) python barrier tasks perform authentication immediately (rather than waiting for the BARRIER_FUNCTION indicator)
2) for `rdd._load_from_socket`, the timeout is only increased after authentication.

Closes #22247 from squito/py_connection_refactor.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 09:47:38 +08:00
Takuya UESHIN 4dd87d8ff5 [SPARK-25142][PYSPARK] Add error messages when Python worker could not open socket in _load_from_socket.
## What changes were proposed in this pull request?

Sometimes Python worker can't open socket in `_load_from_socket` for some reason, but it's difficult to figure out the reason because the exception doesn't even contain the messages from `socket.error`s.
We should at least add the error messages when raising the exception.

## How was this patch tested?

Manually in my local environment.

Closes #22132 from ueshin/issues/SPARK-25142/socket_error.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-18 17:24:06 +08:00
Kazuhiro Sera 8ec25cd67e Fix typos detected by github.com/client9/misspell
## What changes were proposed in this pull request?

Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell).

This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know.

## How was this patch tested?

### before

```
$ misspell . | grep -v '.js'
R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition"
NOTICE-binary:454:16: "containd" is a misspelling of "contained"
R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition"
R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition"
R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence"
R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred"
R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output"
R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent"
common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin"
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden"
core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments"
dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual"
dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across"
dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across"
dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments"
docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden"
docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes"
docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN"
docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior"
examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract"
examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions"
python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment"
python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability"
python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter"
python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability"
python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter"
python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns"
python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization"
python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary"
resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints"
resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when"
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp"
sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage"
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred"
sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing"
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with"
sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring"
```

### after

```
$ misspell . | grep -v '.js'
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
```

Closes #22070 from seratch/fix-typo.

Authored-by: Kazuhiro Sera <seratch@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 21:23:36 -05:00
Xingbo Jiang 4855d5c4b9 [SPARK-24822][PYSPARK] Python support for barrier execution mode
## What changes were proposed in this pull request?

This PR add python support for barrier execution mode, thus enable launch a job containing barrier stage(s) from PySpark.

We just forked the existing `RDDBarrier` and `RDD.barrier()` in Python api.

## How was this patch tested?

Manually tested:
```
>>> rdd = sc.parallelize([1, 2, 3, 4])
>>> def f(iterator): yield sum(iterator)
...
>>> rdd.barrier().mapPartitions(f).isBarrier() == True
True
```

Unit tests will be added in a follow-up PR that implements BarrierTaskContext on python side.

Closes #22011 from jiangxb1987/python.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-11 21:44:45 +08:00
hyukjinkwon 74f6a92fce [SPARK-24739][PYTHON] Make PySpark compatible with Python 3.7
## What changes were proposed in this pull request?

This PR proposes to make PySpark compatible with Python 3.7.  There are rather radical change in semantic of `StopIteration` within a generator. It now throws it as a `RuntimeError`.

To make it compatible, we should fix it:

```python
try:
    next(...)
except StopIteration
    return
```

See [release note](https://docs.python.org/3/whatsnew/3.7.html#porting-to-python-3-7) and [PEP 479](https://www.python.org/dev/peps/pep-0479/).

## How was this patch tested?

Manually tested:

```
 $ ./run-tests --python-executables=python3.7
Running PySpark tests. Output is in /.../spark/python/unit-tests.log
Will test against the following Python executables: ['python3.7']
Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming']
Starting test(python3.7): pyspark.mllib.tests
Starting test(python3.7): pyspark.sql.tests
Starting test(python3.7): pyspark.streaming.tests
Starting test(python3.7): pyspark.tests
Finished test(python3.7): pyspark.streaming.tests (130s)
Starting test(python3.7): pyspark.accumulators
Finished test(python3.7): pyspark.accumulators (8s)
Starting test(python3.7): pyspark.broadcast
Finished test(python3.7): pyspark.broadcast (9s)
Starting test(python3.7): pyspark.conf
Finished test(python3.7): pyspark.conf (6s)
Starting test(python3.7): pyspark.context
Finished test(python3.7): pyspark.context (27s)
Starting test(python3.7): pyspark.ml.classification
Finished test(python3.7): pyspark.tests (200s) ... 3 tests were skipped
Starting test(python3.7): pyspark.ml.clustering
Finished test(python3.7): pyspark.mllib.tests (244s)
Starting test(python3.7): pyspark.ml.evaluation
Finished test(python3.7): pyspark.ml.classification (63s)
Starting test(python3.7): pyspark.ml.feature
Finished test(python3.7): pyspark.ml.clustering (48s)
Starting test(python3.7): pyspark.ml.fpm
Finished test(python3.7): pyspark.ml.fpm (0s)
Starting test(python3.7): pyspark.ml.image
Finished test(python3.7): pyspark.ml.evaluation (23s)
Starting test(python3.7): pyspark.ml.linalg.__init__
Finished test(python3.7): pyspark.ml.linalg.__init__ (0s)
Starting test(python3.7): pyspark.ml.recommendation
Finished test(python3.7): pyspark.ml.image (20s)
Starting test(python3.7): pyspark.ml.regression
Finished test(python3.7): pyspark.ml.regression (58s)
Starting test(python3.7): pyspark.ml.stat
Finished test(python3.7): pyspark.ml.feature (90s)
Starting test(python3.7): pyspark.ml.tests
Finished test(python3.7): pyspark.ml.recommendation (82s)
Starting test(python3.7): pyspark.ml.tuning
Finished test(python3.7): pyspark.ml.stat (27s)
Starting test(python3.7): pyspark.mllib.classification
Finished test(python3.7): pyspark.sql.tests (362s) ... 102 tests were skipped
Starting test(python3.7): pyspark.mllib.clustering
Finished test(python3.7): pyspark.ml.tuning (29s)
Starting test(python3.7): pyspark.mllib.evaluation
Finished test(python3.7): pyspark.mllib.classification (39s)
Starting test(python3.7): pyspark.mllib.feature
Finished test(python3.7): pyspark.mllib.evaluation (30s)
Starting test(python3.7): pyspark.mllib.fpm
Finished test(python3.7): pyspark.mllib.feature (44s)
Starting test(python3.7): pyspark.mllib.linalg.__init__
Finished test(python3.7): pyspark.mllib.linalg.__init__ (0s)
Starting test(python3.7): pyspark.mllib.linalg.distributed
Finished test(python3.7): pyspark.mllib.clustering (78s)
Starting test(python3.7): pyspark.mllib.random
Finished test(python3.7): pyspark.mllib.fpm (33s)
Starting test(python3.7): pyspark.mllib.recommendation
Finished test(python3.7): pyspark.mllib.random (12s)
Starting test(python3.7): pyspark.mllib.regression
Finished test(python3.7): pyspark.mllib.linalg.distributed (45s)
Starting test(python3.7): pyspark.mllib.stat.KernelDensity
Finished test(python3.7): pyspark.mllib.stat.KernelDensity (0s)
Starting test(python3.7): pyspark.mllib.stat._statistics
Finished test(python3.7): pyspark.mllib.recommendation (41s)
Starting test(python3.7): pyspark.mllib.tree
Finished test(python3.7): pyspark.mllib.regression (44s)
Starting test(python3.7): pyspark.mllib.util
Finished test(python3.7): pyspark.mllib.stat._statistics (20s)
Starting test(python3.7): pyspark.profiler
Finished test(python3.7): pyspark.mllib.tree (26s)
Starting test(python3.7): pyspark.rdd
Finished test(python3.7): pyspark.profiler (11s)
Starting test(python3.7): pyspark.serializers
Finished test(python3.7): pyspark.mllib.util (24s)
Starting test(python3.7): pyspark.shuffle
Finished test(python3.7): pyspark.shuffle (0s)
Starting test(python3.7): pyspark.sql.catalog
Finished test(python3.7): pyspark.serializers (15s)
Starting test(python3.7): pyspark.sql.column
Finished test(python3.7): pyspark.rdd (27s)
Starting test(python3.7): pyspark.sql.conf
Finished test(python3.7): pyspark.sql.catalog (24s)
Starting test(python3.7): pyspark.sql.context
Finished test(python3.7): pyspark.sql.conf (8s)
Starting test(python3.7): pyspark.sql.dataframe
Finished test(python3.7): pyspark.sql.column (29s)
Starting test(python3.7): pyspark.sql.functions
Finished test(python3.7): pyspark.sql.context (26s)
Starting test(python3.7): pyspark.sql.group
Finished test(python3.7): pyspark.sql.dataframe (51s)
Starting test(python3.7): pyspark.sql.readwriter
Finished test(python3.7): pyspark.ml.tests (266s)
Starting test(python3.7): pyspark.sql.session
Finished test(python3.7): pyspark.sql.group (36s)
Starting test(python3.7): pyspark.sql.streaming
Finished test(python3.7): pyspark.sql.functions (57s)
Starting test(python3.7): pyspark.sql.types
Finished test(python3.7): pyspark.sql.session (25s)
Starting test(python3.7): pyspark.sql.udf
Finished test(python3.7): pyspark.sql.types (10s)
Starting test(python3.7): pyspark.sql.window
Finished test(python3.7): pyspark.sql.readwriter (31s)
Starting test(python3.7): pyspark.streaming.util
Finished test(python3.7): pyspark.sql.streaming (22s)
Starting test(python3.7): pyspark.util
Finished test(python3.7): pyspark.util (0s)
Finished test(python3.7): pyspark.streaming.util (0s)
Finished test(python3.7): pyspark.sql.udf (16s)
Finished test(python3.7): pyspark.sql.window (12s)
```

In my local (I have two Macs but both have the same issues), I currently faced some issues for now to install both extra dependencies PyArrow and Pandas same as Jenkins's, against Python 3.7.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21714 from HyukjinKwon/SPARK-24739.
2018-07-07 11:37:41 +08:00
Li Jin 9786ce66c5 [SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.

```
       >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
       >>> from pyspark.sql import Window
       >>> df = spark.createDataFrame(
       ...     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
       ...     ("id", "v"))
       >>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
       ... def mean_udf(v):
       ...     return v.mean()
       >>> w = Window.partitionBy('id')
       >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
       +---+----+------+
       | id|   v|mean_v|
       +---+----+------+
       |  1| 1.0|   1.5|
       |  1| 2.0|   1.5|
       |  2| 3.0|   6.0|
       |  2| 5.0|   6.0|
       |  2|10.0|   6.0|
       +---+----+------+
```

The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)

Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.

## How was this patch tested?

WindowPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-13 09:10:52 +08:00
e-dorigatti 0ebb0c0d4d [SPARK-23754][PYTHON] Re-raising StopIteration in client code
## What changes were proposed in this pull request?

Make sure that `StopIteration`s raised in users' code do not silently interrupt processing by spark, but are raised as exceptions to the users. The users' functions are wrapped in `safe_iter` (in `shuffle.py`), which re-raises `StopIteration`s as `RuntimeError`s

## How was this patch tested?

Unit tests, making sure that the exceptions are indeed raised. I am not sure how to check whether a `Py4JJavaError` contains my exception, so I simply looked for the exception message in the java exception's `toString`. Can you propose a better way?

## License

This is my original work, licensed in the same way as spark

Author: e-dorigatti <emilio.dorigatti@gmail.com>
Author: edorigatti <emilio.dorigatti@gmail.com>

Closes #21383 from e-dorigatti/fix_spark_23754.
2018-05-30 18:11:33 +08:00
Marcelo Vanzin cc613b552e [PYSPARK] Update py4j to version 0.10.7. 2018-05-09 10:47:35 -07:00
Benjamin Peterson 7013eea11c [SPARK-23522][PYTHON] always use sys.exit over builtin exit
The exit() builtin is only for interactive use. applications should use sys.exit().

## What changes were proposed in this pull request?

All usage of the builtin `exit()` function is replaced by `sys.exit()`.

## How was this patch tested?

I ran `python/run-tests`.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Benjamin Peterson <benjamin@python.org>

Closes #20682 from benjaminp/sys-exit.
2018-03-08 20:38:34 +09:00
gatorsmile 7a2ada223e [SPARK-23261][PYSPARK] Rename Pandas UDFs
## What changes were proposed in this pull request?
Rename the public APIs and names of pandas udfs.

- `PANDAS SCALAR UDF` -> `SCALAR PANDAS UDF`
- `PANDAS GROUP MAP UDF` -> `GROUPED MAP PANDAS UDF`
- `PANDAS GROUP AGG UDF` -> `GROUPED AGG PANDAS UDF`

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20428 from gatorsmile/renamePandasUDFs.
2018-01-30 21:55:55 +09:00
Li Jin b2ce17b4c9 [SPARK-22274][PYTHON][SQL] User-defined aggregation functions with pandas udf (full shuffle)
## What changes were proposed in this pull request?

Add support for using pandas UDFs with groupby().agg().

This PR introduces a new type of pandas UDF - group aggregate pandas UDF. This type of UDF defines a transformation of multiple pandas Series -> a scalar value. Group aggregate pandas UDFs can be used with groupby().agg(). Note group aggregate pandas UDF doesn't support partial aggregation, i.e., a full shuffle is required.

This PR doesn't support group aggregate pandas UDFs that return ArrayType, StructType or MapType. Support for these types is left for future PR.

## How was this patch tested?

GroupbyAggPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19872 from icexelloss/SPARK-22274-groupby-agg.
2018-01-23 14:11:30 +09:00
王晓哲 602c6d82d8 [SPARK-20947][PYTHON] Fix encoding/decoding error in pipe action
## What changes were proposed in this pull request?

Pipe action convert objects into strings using a way that was affected by the default encoding setting of Python environment.

This patch fixed the problem. The detailed description is added here:

https://issues.apache.org/jira/browse/SPARK-20947

## How was this patch tested?

Run the following statement in pyspark-shell, and it will NOT raise exception if this patch is applied:

```python
sc.parallelize([u'\u6d4b\u8bd5']).pipe('cat').collect()
```

Author: 王晓哲 <wxz@linkdoc.com>

Closes #18277 from chaoslawful/fix_pipe_encoding_error.
2018-01-22 10:43:12 +09:00
Li Jin 7d039e0c0a [SPARK-22409] Introduce function type argument in pandas_udf
## What changes were proposed in this pull request?

* Add a "function type" argument to pandas_udf.
* Add a new public enum class `PandasUdfType` in pyspark.sql.functions
* Refactor udf related code from pyspark.sql.functions to pyspark.sql.udf
* Merge "PythonUdfType" and "PythonEvalType" into a single enum class "PythonEvalType"

Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType

pandas_udf('double', PandasUDFType.SCALAR):
def plus_one(v):
    return v + 1
```

## Design doc
https://docs.google.com/document/d/1KlLaa-xJ3oz28xlEJqXyCAHU3dwFYkFs_ixcUXrJNTc/edit

## How was this patch tested?

Added PandasUDFTests

## TODO:
* [x] Implement proper enum type for `PandasUDFType`
* [x] Update documentation
* [x] Add more tests in PandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19630 from icexelloss/spark-22409-pandas-udf-type.
2017-11-17 16:43:08 +01:00
peay c06f3f5ac5 [SPARK-21551][PYTHON] Increase timeout for PythonRDD.serveIterator
## What changes were proposed in this pull request?

This modification increases the timeout for `serveIterator` (which is not dynamically configurable). This fixes timeout issues in pyspark when using `collect` and similar functions, in cases where Python may take more than a couple seconds to connect.

See https://issues.apache.org/jira/browse/SPARK-21551

## How was this patch tested?

Ran the tests.

cc rxin

Author: peay <peay@protonmail.com>

Closes #18752 from peay/spark-21551.
2017-08-09 14:03:18 -07:00
chie8842 c3713fde86 [SPARK-21358][EXAMPLES] Argument of repartitionandsortwithinpartitions at pyspark
## What changes were proposed in this pull request?
At example of repartitionAndSortWithinPartitions at rdd.py, third argument should be True or False.
I proposed fix of example code.

## How was this patch tested?
* I rename test_repartitionAndSortWithinPartitions to test_repartitionAndSortWIthinPartitions_asc to specify boolean argument.
* I added test_repartitionAndSortWithinPartitions_desc to test False pattern at third argument.

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: chie8842 <chie8842@gmail.com>

Closes #18586 from chie8842/SPARK-21358.
2017-07-10 18:56:54 -07:00
hyukjinkwon d492cc5a21 [SPARK-19507][SPARK-21296][PYTHON] Avoid per-record type dispatch in schema verification and improve exception message
## What changes were proposed in this pull request?
**Context**

While reviewing https://github.com/apache/spark/pull/17227, I realised here we type-dispatch per record. The PR itself is fine in terms of performance as is but this prints a prefix, `"obj"` in exception message as below:

```
from pyspark.sql.types import *
schema = StructType([StructField('s', IntegerType(), nullable=False)])
spark.createDataFrame([["1"]], schema)
...
TypeError: obj.s: IntegerType can not accept object '1' in type <type 'str'>
```

I suggested to get rid of this but during investigating this, I realised my approach might bring a performance regression as it is a hot path.

Only for SPARK-19507 and https://github.com/apache/spark/pull/17227, It needs more changes to cleanly get rid of the prefix and I rather decided to fix both issues together.

**Propersal**

This PR tried to

  - get rid of per-record type dispatch as we do in many code paths in Scala  so that it improves the performance (roughly ~25% improvement) - SPARK-21296

    This was tested with a simple code `spark.createDataFrame(range(1000000), "int")`. However, I am quite sure the actual improvement in practice is larger than this, in particular, when the schema is complicated.

   - improve error message in exception describing field information as prose - SPARK-19507

## How was this patch tested?

Manually tested and unit tests were added in `python/pyspark/sql/tests.py`.

Benchmark - codes: https://gist.github.com/HyukjinKwon/c3397469c56cb26c2d7dd521ed0bc5a3
Error message - codes: https://gist.github.com/HyukjinKwon/b1b2c7f65865444c4a8836435100e398

**Before**

Benchmark:
  - Results: https://gist.github.com/HyukjinKwon/4a291dab45542106301a0c1abcdca924

Error message
  - Results: https://gist.github.com/HyukjinKwon/57b1916395794ce924faa32b14a3fe19

**After**

Benchmark
  - Results: https://gist.github.com/HyukjinKwon/21496feecc4a920e50c4e455f836266e

Error message
  - Results: https://gist.github.com/HyukjinKwon/7a494e4557fe32a652ce1236e504a395

Closes #17227

Author: hyukjinkwon <gurwls223@gmail.com>
Author: David Gingrich <david@textio.com>

Closes #18521 from HyukjinKwon/python-type-dispatch.
2017-07-04 20:45:58 +08:00
David Gingrich 8ddf0d2a60 [SPARK-20232][PYTHON] Improve combineByKey docs
## What changes were proposed in this pull request?

Improve combineByKey documentation:

* Add note on memory allocation
* Change example code to use different mergeValue and mergeCombiners

## How was this patch tested?

Doctest.

## Legal

This is my original work and I license the work to the project under the project’s open source license.

Author: David Gingrich <david@textio.com>

Closes #17545 from dgingrich/topic-spark-20232-combinebykey-docs.
2017-04-13 12:43:28 -07:00
hyukjinkwon 7387126f83 [SPARK-19872] [PYTHON] Use the correct deserializer for RDD construction for coalesce/repartition
## What changes were proposed in this pull request?

This PR proposes to use the correct deserializer, `BatchedSerializer` for RDD construction for coalesce/repartition when the shuffle is enabled. Currently, it is passing `UTF8Deserializer` as is not `BatchedSerializer` from the copied one.

with the file, `text.txt` below:

```
a
b

d
e
f
g
h
i
j
k
l

```

- Before

```python
>>> sc.textFile('text.txt').repartition(1).collect()
```

```
UTF8Deserializer(True)
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/rdd.py", line 811, in collect
    return list(_load_from_socket(port, self._jrdd_deserializer))
  File ".../spark/python/pyspark/serializers.py", line 549, in load_stream
    yield self.loads(stream)
  File ".../spark/python/pyspark/serializers.py", line 544, in loads
    return s.decode("utf-8") if self.use_unicode else s
  File "/System/Library/Frameworks/Python.framework/Versions/2.7/lib/python2.7/encodings/utf_8.py", line 16, in decode
    return codecs.utf_8_decode(input, errors, True)
UnicodeDecodeError: 'utf8' codec can't decode byte 0x80 in position 0: invalid start byte
```

- After

```python
>>> sc.textFile('text.txt').repartition(1).collect()
```

```
[u'a', u'b', u'', u'd', u'e', u'f', u'g', u'h', u'i', u'j', u'k', u'l', u'']
```

## How was this patch tested?

Unit test in `python/pyspark/tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17282 from HyukjinKwon/SPARK-19872.
2017-03-15 10:17:18 -07:00
Jeff Zhang 330c3e33bd [SPARK-13330][PYSPARK] PYTHONHASHSEED is not propgated to python worker
## What changes were proposed in this pull request?
self.environment will be propagated to executor. Should set PYTHONHASHSEED as long as the python version is greater than 3.3

## How was this patch tested?
Manually tested it.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #11211 from zjffdu/SPARK-13330.
2017-02-24 15:04:42 -08:00
Liang-Chi Hsieh 95c95b71ed [SPARK-18281] [SQL] [PYSPARK] Remove timeout for reading data through socket for local iterator
## What changes were proposed in this pull request?

There is a timeout failure when using `rdd.toLocalIterator()` or `df.toLocalIterator()` for a PySpark RDD and DataFrame:

    df = spark.createDataFrame([[1],[2],[3]])
    it = df.toLocalIterator()
    row = next(it)

    df2 = df.repartition(1000)  # create many empty partitions which increase materialization time so causing timeout
    it2 = df2.toLocalIterator()
    row = next(it2)

The cause of this issue is, we open a socket to serve the data from JVM side. We set timeout for connection and reading through the socket in Python side. In Python we use a generator to read the data, so we only begin to connect the socket once we start to ask data from it. If we don't consume it immediately, there is connection timeout.

In the other side, the materialization time for RDD partitions is unpredictable. So we can't set a timeout for reading data through the socket. Otherwise, it is very possibly to fail.

## How was this patch tested?

Added tests into PySpark.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16263 from viirya/fix-pyspark-localiterator.
2016-12-20 13:12:16 -08:00
hyukjinkwon 933a6548d4
[SPARK-18447][DOCS] Fix the markdown for Note:/NOTE:/Note that across Python API documentation
## What changes were proposed in this pull request?

It seems in Python, there are

- `Note:`
- `NOTE:`
- `Note that`
- `.. note::`

This PR proposes to fix those to `.. note::` to be consistent.

**Before**

<img width="567" alt="2016-11-21 1 18 49" src="https://cloud.githubusercontent.com/assets/6477701/20464305/85144c86-af88-11e6-8ee9-90f584dd856c.png">

<img width="617" alt="2016-11-21 12 42 43" src="https://cloud.githubusercontent.com/assets/6477701/20464263/27be5022-af88-11e6-8577-4bbca7cdf36c.png">

**After**

<img width="554" alt="2016-11-21 1 18 42" src="https://cloud.githubusercontent.com/assets/6477701/20464306/8fe48932-af88-11e6-83e1-fc3cbf74407d.png">

<img width="628" alt="2016-11-21 12 42 51" src="https://cloud.githubusercontent.com/assets/6477701/20464264/2d3e156e-af88-11e6-93f3-cab8d8d02983.png">

## How was this patch tested?

The notes were found via

```bash
grep -r "Note: " .
grep -r "NOTE: " .
grep -r "Note that " .
```

And then fixed one by one comparing with API documentation.

After that, manually tested via `make html` under `./python/docs`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15947 from HyukjinKwon/SPARK-18447.
2016-11-22 11:40:18 +00:00
Gabriel Huang 70176871ae [SPARK-18361][PYSPARK] Expose RDD localCheckpoint in PySpark
## What changes were proposed in this pull request?

Expose RDD's localCheckpoint() and associated functions in PySpark.

## How was this patch tested?

I added a UnitTest in python/pyspark/tests.py which passes.

I certify that this is my original work, and I license it to the project under the project's open source license.

Gabriel HUANG
Developer at Cardabel (http://cardabel.com/)

Author: Gabriel Huang <gabi.xiaohuang@gmail.com>

Closes #15811 from gabrielhuang/pyspark-localcheckpoint.
2016-11-21 16:08:34 -05:00
hyukjinkwon d5b1d5fc80
[SPARK-18445][BUILD][DOCS] Fix the markdown for Note:/NOTE:/Note that/'''Note:''' across Scala/Java API documentation
## What changes were proposed in this pull request?

It seems in Scala/Java,

- `Note:`
- `NOTE:`
- `Note that`
- `'''Note:'''`
- `note`

This PR proposes to fix those to `note` to be consistent.

**Before**

- Scala
  ![2016-11-17 6 16 39](https://cloud.githubusercontent.com/assets/6477701/20383180/1a7aed8c-acf2-11e6-9611-5eaf6d52c2e0.png)

- Java
  ![2016-11-17 6 14 41](https://cloud.githubusercontent.com/assets/6477701/20383096/c8ffc680-acf1-11e6-914a-33460bf1401d.png)

**After**

- Scala
  ![2016-11-17 6 16 44](https://cloud.githubusercontent.com/assets/6477701/20383167/09940490-acf2-11e6-937a-0d5e1dc2cadf.png)

- Java
  ![2016-11-17 6 13 39](https://cloud.githubusercontent.com/assets/6477701/20383132/e7c2a57e-acf1-11e6-9c47-b849674d4d88.png)

## How was this patch tested?

The notes were found via

```bash
grep -r "NOTE: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// NOTE: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \ # note that this is a regular expression. So actual matches were mostly `org/apache/spark/api/java/functions ...`
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note that " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note that " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "'''Note:'''" . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// '''Note:''' " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

And then fixed one by one comparing with API documentation/access modifiers.

After that, manually tested via `jekyll build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15889 from HyukjinKwon/SPARK-18437.
2016-11-19 11:24:15 +00:00
anabranch 49b6f456ac
[SPARK-18365][DOCS] Improve Sample Method Documentation
## What changes were proposed in this pull request?

I found the documentation for the sample method to be confusing, this adds more clarification across all languages.

- [x] Scala
- [x] Python
- [x] R
- [x] RDD Scala
- [ ] RDD Python with SEED
- [X] RDD Java
- [x] RDD Java with SEED
- [x] RDD Python

## How was this patch tested?

NA

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>

Closes #15815 from anabranch/SPARK-18365.
2016-11-17 11:34:55 +00:00
Liang-Chi Hsieh 1e35e96930 [SPARK-17817] [PYSPARK] [FOLLOWUP] PySpark RDD Repartitioning Results in Highly Skewed Partition Sizes
## What changes were proposed in this pull request?

This change is a followup for #15389 which calls `_to_java_object_rdd()` to solve this issue. Due to the concern of the possible expensive cost of the call, we can choose to decrease the batch size to solve this issue too.

Simple benchmark:

    import time
    num_partitions = 20000
    a = sc.parallelize(range(int(1e6)), 2)
    start = time.time()
    l = a.repartition(num_partitions).glom().map(len).collect()
    end = time.time()
    print(end - start)

Before: 419.447577953
_to_java_object_rdd(): 421.916361094
decreasing the batch size: 423.712255955

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #15445 from viirya/repartition-batch-size.
2016-10-18 14:25:10 -07:00
Liang-Chi Hsieh 07508bd01d [SPARK-17817][PYSPARK] PySpark RDD Repartitioning Results in Highly Skewed Partition Sizes
## What changes were proposed in this pull request?

Quoted from JIRA description:

Calling repartition on a PySpark RDD to increase the number of partitions results in highly skewed partition sizes, with most having 0 rows. The repartition method should evenly spread out the rows across the partitions, and this behavior is correctly seen on the Scala side.

Please reference the following code for a reproducible example of this issue:

    num_partitions = 20000
    a = sc.parallelize(range(int(1e6)), 2)  # start with 2 even partitions
    l = a.repartition(num_partitions).glom().map(len).collect()  # get length of each partition
    min(l), max(l), sum(l)/len(l), len(l)  # skewed!

In Scala's `repartition` code, we will distribute elements evenly across output partitions. However, the RDD from Python is serialized as a single binary data, so the distribution fails. We need to convert the RDD in Python to java object before repartitioning.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #15389 from viirya/pyspark-rdd-repartition.
2016-10-11 11:43:24 -07:00
Jason White 1f31bdaef6 [SPARK-17679] [PYSPARK] remove unnecessary Py4J ListConverter patch
## What changes were proposed in this pull request?

This PR removes a patch on ListConverter from https://github.com/apache/spark/pull/5570, as it is no longer necessary. The underlying issue in Py4J https://github.com/bartdag/py4j/issues/160 was patched in 224b94b666 and is present in 0.10.3, the version currently in use in Spark.

## How was this patch tested?

The original test added in https://github.com/apache/spark/pull/5570 remains.

Author: Jason White <jason.white@shopify.com>

Closes #15254 from JasonMWhite/remove_listconverter_patch.
2016-10-03 14:12:03 -07:00
hyukjinkwon 2190037757
[MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation
## What changes were proposed in this pull request?

This PR proposes to fix wrongly indented examples in PySpark documentation

```
-        >>> json_sdf = spark.readStream.format("json")\
-                                       .schema(sdf_schema)\
-                                       .load(tempfile.mkdtemp())
+        >>> json_sdf = spark.readStream.format("json") \\
+        ...     .schema(sdf_schema) \\
+        ...     .load(tempfile.mkdtemp())
```

```
-        people.filter(people.age > 30).join(department, people.deptId == department.id)\
+        people.filter(people.age > 30).join(department, people.deptId == department.id) \\
```

```
-        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \
-                        LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])),
+        ...             LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```

```
-        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \
-                        LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])),
+        ...             LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```

```
-        ...      for x in iterator:
-        ...           print(x)
+        ...     for x in iterator:
+        ...          print(x)
```

## How was this patch tested?

Manually tested.

**Before**

![2016-09-26 8 36 02](https://cloud.githubusercontent.com/assets/6477701/18834471/05c7a478-8431-11e6-94bb-09aa37b12ddb.png)

![2016-09-26 9 22 16](https://cloud.githubusercontent.com/assets/6477701/18834472/06c8735c-8431-11e6-8775-78631eab0411.png)

<img width="601" alt="2016-09-27 2 29 27" src="https://cloud.githubusercontent.com/assets/6477701/18861294/29c0d5b4-84bf-11e6-99c5-3c9d913c125d.png">

<img width="1056" alt="2016-09-27 2 29 58" src="https://cloud.githubusercontent.com/assets/6477701/18861298/31694cd8-84bf-11e6-9e61-9888cb8c2089.png">

<img width="1079" alt="2016-09-27 2 30 05" src="https://cloud.githubusercontent.com/assets/6477701/18861301/359722da-84bf-11e6-97f9-5f5365582d14.png">

**After**

![2016-09-26 9 29 47](https://cloud.githubusercontent.com/assets/6477701/18834467/0367f9da-8431-11e6-86d9-a490d3297339.png)

![2016-09-26 9 30 24](https://cloud.githubusercontent.com/assets/6477701/18834463/f870fae0-8430-11e6-9482-01fc47898492.png)

<img width="515" alt="2016-09-27 2 28 19" src="https://cloud.githubusercontent.com/assets/6477701/18861305/3ff88b88-84bf-11e6-902c-9f725e8a8b10.png">

<img width="652" alt="2016-09-27 3 50 59" src="https://cloud.githubusercontent.com/assets/6477701/18863053/592fbc74-84ca-11e6-8dbf-99cf57947de8.png">

<img width="709" alt="2016-09-27 3 51 03" src="https://cloud.githubusercontent.com/assets/6477701/18863060/601607be-84ca-11e6-80aa-a401df41c321.png">

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15242 from HyukjinKwon/minor-example-pyspark.
2016-09-28 06:19:04 -04:00
Mortada Mehyar 6ee40d2cc5 [DOC] improve python doc for rdd.histogram and dataframe.join
## What changes were proposed in this pull request?

doc change only

## How was this patch tested?

doc change only

Author: Mortada Mehyar <mortada.mehyar@gmail.com>

Closes #14253 from mortada/histogram_typos.
2016-07-18 23:49:47 -07:00