Commit graph

1367 commits

Author SHA1 Message Date
Jason Lee 3d66a2ce9b [SPARK-14564][ML][MLLIB][PYSPARK] Python Word2Vec missing setWindowSize method
## What changes were proposed in this pull request?
Added windowSize getter/setter to ML/MLlib

## How was this patch tested?
Added test cases in tests.py under both ML and MLlib

Author: Jason Lee <cjlee@us.ibm.com>

Closes #12428 from jasoncl/SPARK-14564.
2016-04-18 12:47:14 -07:00
Xusen Yin b64482f49f [SPARK-14306][ML][PYSPARK] PySpark ml.classification OneVsRest support export/import
## What changes were proposed in this pull request?

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

Add PySpark OneVsRest save/load supports.

## How was this patch tested?

Test with Python unit test.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #12439 from yinxusen/SPARK-14306-0415.
2016-04-18 11:52:29 -07:00
Joseph K. Bradley 36da5e3234 [SPARK-14605][ML][PYTHON] Changed Python to use unicode UIDs for spark.ml Identifiable
## What changes were proposed in this pull request?

Python spark.ml Identifiable classes use UIDs of type str, but they should use unicode (in Python 2.x) to match Java. This could be a problem if someone created a class in Java with odd unicode characters, saved it, and loaded it in Python.

This PR: Use unicode everywhere in Python.

## How was this patch tested?

Updated persistence unit test to check uid type

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #12368 from jkbradley/python-uid-unicode.
2016-04-16 11:23:28 -07:00
Xusen Yin 90b46e014a [SPARK-7861][ML] PySpark OneVsRest
## What changes were proposed in this pull request?

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

Add PySpark OneVsRest. I implement it with Python since it's a meta-pipeline.

## How was this patch tested?

Test with doctest.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #12124 from yinxusen/SPARK-14306-7861.
2016-04-15 12:58:38 -07:00
sethah 129f2f455d [SPARK-14104][PYSPARK][ML] All Python param setters should use the _set method
## What changes were proposed in this pull request?

Param setters in python previously accessed the _paramMap directly to update values. The `_set` method now implements type checking, so it should be used to update all parameters. This PR eliminates all direct accesses to `_paramMap` besides the one in the `_set` method to ensure type checking happens.

Additional changes:
* [SPARK-13068](https://github.com/apache/spark/pull/11663) missed adding type converters in evaluation.py so those are done here
* An incorrect `toBoolean` type converter was used for StringIndexer `handleInvalid` param in previous PR. This is fixed here.

## How was this patch tested?

Existing unit tests verify that parameters are still set properly. No new functionality is actually added in this PR.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #11939 from sethah/SPARK-14104.
2016-04-15 12:14:41 -07:00
Joseph K. Bradley d6ae7d4637 [SPARK-14665][ML][PYTHON] Fixed bug with StopWordsRemover default stopwords
## What changes were proposed in this pull request?

The default stopwords were a Java object.  They are no longer.

## How was this patch tested?

Unit test which failed before the fix

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #12422 from jkbradley/pyspark-stopwords.
2016-04-15 11:50:21 -07:00
Yanbo Liang b9613239d3 [SPARK-14374][ML][PYSPARK] PySpark ml GBTClassifier, Regressor support export/import
## What changes were proposed in this pull request?
PySpark ml GBTClassifier, Regressor support export/import.

## How was this patch tested?
Doc test.

cc jkbradley

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #12383 from yanboliang/spark-14374.
2016-04-14 21:36:03 -07:00
Yong Tang bc748b7b8f [SPARK-14238][ML][MLLIB][PYSPARK] Add binary toggle Param to PySpark HashingTF in ML & MLlib
## What changes were proposed in this pull request?

This fix tries to add binary toggle Param to PySpark HashingTF in ML & MLlib. If this toggle is set, then all non-zero counts will be set to 1.

Note: This fix (SPARK-14238) is extended from SPARK-13963 where Scala implementation was done.

## How was this patch tested?

This fix adds two tests to cover the code changes. One for HashingTF in PySpark's ML and one for HashingTF in PySpark's MLLib.

Author: Yong Tang <yong.tang.github@outlook.com>

Closes #12079 from yongtang/SPARK-14238.
2016-04-14 21:53:32 +02:00
Bryan Cutler c5172f8205 [SPARK-13967][PYSPARK][ML] Added binary Param to Python CountVectorizer
Added binary toggle param to CountVectorizer feature transformer in PySpark.

Created a unit test for using CountVectorizer with the binary toggle on.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #12308 from BryanCutler/binary-param-python-CountVectorizer-SPARK-13967.
2016-04-14 20:47:31 +02:00
Holden Karau 478af2f455 [SPARK-14573][PYSPARK][BUILD] Fix PyDoc Makefile & highlighting issues
## What changes were proposed in this pull request?

The PyDoc Makefile used "=" rather than "?=" for setting env variables so it overwrote the user values. This ignored the environment variables we set for linting allowing warnings through. This PR also fixes the warnings that had been introduced.

## How was this patch tested?

manual local export & make

Author: Holden Karau <holden@us.ibm.com>

Closes #12336 from holdenk/SPARK-14573-fix-pydoc-makefile.
2016-04-14 09:42:15 +01:00
Bryan Cutler fc3cd2f509 [SPARK-14472][PYSPARK][ML] Cleanup ML JavaWrapper and related class hierarchy
Currently, JavaWrapper is only a wrapper class for pipeline classes that have Params and JavaCallable is a separate mixin that provides methods to make Java calls.  This change simplifies the class structure and to define the Java wrapper in a plain base class along with methods to make Java calls.  Also, renames Java wrapper classes to better reflect their purpose.

Ran existing Python ml tests and generated documentation to test this change.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #12304 from BryanCutler/pyspark-cleanup-JavaWrapper-SPARK-14472.
2016-04-13 14:08:57 -07:00
Liwei Lin 23f93f559c [SPARK-13992][CORE][PYSPARK][FOLLOWUP] Update OFF_HEAP semantics for Java api and Python api
## What changes were proposed in this pull request?

- updated `OFF_HEAP` semantics for `StorageLevels.java`
- updated `OFF_HEAP` semantics for `storagelevel.py`

## How was this patch tested?

no need to test

Author: Liwei Lin <lwlin7@gmail.com>

Closes #12126 from lw-lin/storagelevel.py.
2016-04-12 23:06:55 -07:00
Kai Jiang 7f024c4744 [SPARK-13597][PYSPARK][ML] Python API for GeneralizedLinearRegression
## What changes were proposed in this pull request?

Python API for GeneralizedLinearRegression
JIRA: https://issues.apache.org/jira/browse/SPARK-13597

## How was this patch tested?

The patch is tested with Python doctest.

Author: Kai Jiang <jiangkai@gmail.com>

Closes #11468 from vectorijk/spark-13597.
2016-04-12 11:29:12 -07:00
Holden Karau 00288ea2a4 [SPARK-13687][PYTHON] Cleanup PySpark parallelize temporary files
## What changes were proposed in this pull request?

Eagerly cleanup PySpark's temporary parallelize cleanup files rather than waiting for shut down.

## How was this patch tested?

Unit tests

Author: Holden Karau <holden@us.ibm.com>

Closes #12233 from holdenk/SPARK-13687-cleanup-pyspark-temporary-files.
2016-04-10 02:34:54 +01:00
Joseph K. Bradley d7af736b2c [SPARK-14498][ML][PYTHON][SQL] Many cleanups to ML and ML-related docs
## What changes were proposed in this pull request?

Cleanups to documentation.  No changes to code.
* GBT docs: Move Scala doc for private object GradientBoostedTrees to public docs for GBTClassifier,Regressor
* GLM regParam: needs doc saying it is for L2 only
* TrainValidationSplitModel: add .. versionadded:: 2.0.0
* Rename “_transformer_params_from_java” to “_transfer_params_from_java”
* LogReg Summary classes: “probability” col should not say “calibrated”
* LR summaries: coefficientStandardErrors —> document that intercept stderr comes last.  Same for t,p-values
* approxCountDistinct: Document meaning of “rsd" argument.
* LDA: note which params are for online LDA only

## How was this patch tested?

Doc build

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #12266 from jkbradley/ml-doc-cleanups.
2016-04-08 20:15:44 -07:00
wm624@hotmail.com e0ad75f2b5 [SPARK-12569][PYSPARK][ML] DecisionTreeRegressor: provide variance of prediction: Python API
## What changes were proposed in this pull request?

A new column VarianceCol has been added to DecisionTreeRegressor in ML scala code.

This patch adds the corresponding Python API, HasVarianceCol, to class DecisionTreeRegressor.

## How was this patch tested?
./dev/lint-python
PEP8 checks passed.
rm -rf _build/*
pydoc checks passed.

./python/run-tests --python-executables=python2.7 --modules=pyspark-ml
Running PySpark tests. Output is in /Users/mwang/spark_ws_0904/python/unit-tests.log
Will test against the following Python executables: ['python2.7']
Will test the following Python modules: ['pyspark-ml']
Finished test(python2.7): pyspark.ml.evaluation (12s)
Finished test(python2.7): pyspark.ml.clustering (18s)
Finished test(python2.7): pyspark.ml.classification (30s)
Finished test(python2.7): pyspark.ml.recommendation (28s)
Finished test(python2.7): pyspark.ml.feature (43s)
Finished test(python2.7): pyspark.ml.regression (31s)
Finished test(python2.7): pyspark.ml.tuning (19s)
Finished test(python2.7): pyspark.ml.tests (34s)

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #12116 from wangmiao1981/fix_api.
2016-04-08 10:47:05 -07:00
Kai Jiang e5d8d6e09c [SPARK-14373][PYSPARK] PySpark RandomForestClassifier, Regressor support export/import
## What changes were proposed in this pull request?
supporting `RandomForest{Classifier, Regressor}` save/load for Python API.
[JIRA](https://issues.apache.org/jira/browse/SPARK-14373)
## How was this patch tested?
doctest

Author: Kai Jiang <jiangkai@gmail.com>

Closes #12238 from vectorijk/spark-14373.
2016-04-08 10:39:12 -07:00
Bryan Cutler 9c6556c5f8 [SPARK-13430][PYSPARK][ML] Python API for training summaries of linear and logistic regression
## What changes were proposed in this pull request?

Adding Python API for training summaries of LogisticRegression and LinearRegression in PySpark ML.

## How was this patch tested?
Added unit tests to exercise the api calls for the summary classes.  Also, manually verified values are expected and match those from Scala directly.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #11621 from BryanCutler/pyspark-ml-summary-SPARK-13430.
2016-04-06 12:07:47 -07:00
Xusen Yin db0b06c6ea [SPARK-13786][ML][PYSPARK] Add save/load for pyspark.ml.tuning
## What changes were proposed in this pull request?

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

Add save/load for Python CrossValidator/Model and TrainValidationSplit/Model.

## How was this patch tested?

Test with Python doctest.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #12020 from yinxusen/SPARK-13786.
2016-04-06 11:24:11 -07:00
Davies Liu 90ca184486 [SPARK-14418][PYSPARK] fix unpersist of Broadcast in Python
## What changes were proposed in this pull request?

Currently, Broaccast.unpersist() will remove the file of broadcast, which should be the behavior of destroy().

This PR added destroy() for Broadcast in Python, to match the sematics in Scala.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #12189 from davies/py_unpersist.
2016-04-06 10:46:34 -07:00
Burak Yavuz 9ee5c25717 [SPARK-14353] Dataset Time Window window API for Python, and SQL
## What changes were proposed in this pull request?

The `window` function was added to Dataset with [this PR](https://github.com/apache/spark/pull/12008).
This PR adds the Python, and SQL, API for this function.

With this PR, SQL, Java, and Scala will share the same APIs as in users can use:
 - `window(timeColumn, windowDuration)`
 - `window(timeColumn, windowDuration, slideDuration)`
 - `window(timeColumn, windowDuration, slideDuration, startTime)`

In Python, users can access all APIs above, but in addition they can do
 - In Python:
   `window(timeColumn, windowDuration, startTime=...)`

that is, they can provide the startTime without providing the `slideDuration`. In this case, we will generate tumbling windows.

## How was this patch tested?

Unit tests + manual tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #12136 from brkyvz/python-windows.
2016-04-05 13:18:39 -07:00
Yong Tang 7db56244fa [SPARK-14368][PYSPARK] Support python.spark.worker.memory with upper-case unit.
## What changes were proposed in this pull request?

This fix tries to address the issue in PySpark where `spark.python.worker.memory`
could only be configured with a lower case unit (`k`, `m`, `g`, `t`). This fix
allows the upper case unit (`K`, `M`, `G`, `T`) to be used as well. This is to
conform to the JVM memory string as is specified in the documentation .

## How was this patch tested?

This fix adds additional test to cover the changes.

Author: Yong Tang <yong.tang.github@outlook.com>

Closes #12163 from yongtang/SPARK-14368.
2016-04-05 12:19:20 +09:00
Marcelo Vanzin 24d7d2e453 [SPARK-13579][BUILD] Stop building the main Spark assembly.
This change modifies the "assembly/" module to just copy needed
dependencies to its build directory, and modifies the packaging
script to pick those up (and remove duplicate jars packages in the
examples module).

I also made some minor adjustments to dependencies to remove some
test jars from the final packaging, and remove jars that conflict with each
other when packaged separately (e.g. servlet api).

Also note that this change restores guava in applications' classpaths, even
though it's still shaded inside Spark. This is now needed for the Hadoop
libraries that are packaged with Spark, which now are not processed by
the shade plugin.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #11796 from vanzin/SPARK-13579.
2016-04-04 16:52:22 -07:00
Davies Liu cc70f17416 [SPARK-14334] [SQL] add toLocalIterator for Dataset/DataFrame
## What changes were proposed in this pull request?

RDD.toLocalIterator() could be used to fetch one partition at a time to reduce the memory usage. Right now, for Dataset/Dataframe we have to use df.rdd.toLocalIterator, which is super slow also requires lots of memory (because of the Java serializer or even Kyro serializer).

This PR introduce an optimized toLocalIterator for Dataset/DataFrame, which is much faster and requires much less memory. For a partition with 5 millions rows, `df.rdd.toIterator` took about 100 seconds, but df.toIterator took less than 7 seconds. For 10 millions row, rdd.toIterator will crash (not enough memory) with 4G heap, but df.toLocalIterator could finished in 12 seconds.

The JDBC server has been updated to use DataFrame.toIterator.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #12114 from davies/local_iterator.
2016-04-04 13:31:44 -07:00
Davies Liu 5743c6476d [SPARK-12981] [SQL] extract Pyhton UDF in physical plan
## What changes were proposed in this pull request?

Currently we extract Python UDFs into a special logical plan EvaluatePython in analyzer, But EvaluatePython is not part of catalyst, many rules have no knowledge of it , which will break many things (for example, filter push down or column pruning).

We should treat Python UDFs as normal expressions, until we want to evaluate in physical plan, we could extract them in end of optimizer, or physical plan.

This PR extract Python UDFs in physical plan.

Closes #10935

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #12127 from davies/py_udf.
2016-04-04 10:56:26 -07:00
hyukjinkwon 2262a93358 [SPARK-14231] [SQL] JSON data source infers floating-point values as a double when they do not fit in a decimal
## What changes were proposed in this pull request?

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

Currently, JSON data source supports to infer `DecimalType` for big numbers and `floatAsBigDecimal` option which reads floating-point values as `DecimalType`.

But there are few restrictions in Spark `DecimalType` below:

1. The precision cannot be bigger than 38.
2. scale cannot be bigger than precision.

Currently, both restrictions are not being handled.

This PR handles the cases by inferring them as `DoubleType`. Also, the option name was changed from `floatAsBigDecimal` to `prefersDecimal` as suggested [here](https://issues.apache.org/jira/browse/SPARK-14231?focusedCommentId=15215579&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15215579).

So, the codes below:

```scala
def doubleRecords: RDD[String] =
  sqlContext.sparkContext.parallelize(
    s"""{"a": 1${"0" * 38}, "b": 0.01}""" ::
    s"""{"a": 2${"0" * 38}, "b": 0.02}""" :: Nil)

val jsonDF = sqlContext.read
  .option("prefersDecimal", "true")
  .json(doubleRecords)
jsonDF.printSchema()
```

produces below:

- **Before**

```scala
org.apache.spark.sql.AnalysisException: Decimal scale (2) cannot be greater than precision (1).;
	at org.apache.spark.sql.types.DecimalType.<init>(DecimalType.scala:44)
	at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:144)
	at org.apache.spark.sql.execution.datasources.json.InferSchema$.org$apache$spark$sql$execution$datasources$json$InferSchema$$inferField(InferSchema.scala:108)
	at
...
```

- **After**

```scala
root
 |-- a: double (nullable = true)
 |-- b: double (nullable = true)
```

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for coding style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12030 from HyukjinKwon/SPARK-14231.
2016-04-02 23:12:04 -07:00
Liwei Lin 03d130f973 [SPARK-14342][CORE][DOCS][TESTS] Remove straggler references to Tachyon
## What changes were proposed in this pull request?

Straggler references to Tachyon were removed:
- for docs, `tachyon` has been generalized as `off-heap memory`;
- for Mesos test suits, the key-value `tachyon:true`/`tachyon:false` has been changed to `os:centos`/`os:ubuntu`, since `os` is an example constrain used by the [Mesos official docs](http://mesos.apache.org/documentation/attributes-resources/).

## How was this patch tested?

Existing test suites.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #12129 from lw-lin/tachyon-cleanup.
2016-04-02 17:55:46 -07:00
Yanbo Liang 381358fbe9 [SPARK-14305][ML][PYSPARK] PySpark ml.clustering BisectingKMeans support export/import
## What changes were proposed in this pull request?
PySpark ml.clustering BisectingKMeans support export/import
## How was this patch tested?
doc test.

cc jkbradley

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #12112 from yanboliang/spark-14305.
2016-04-01 12:53:39 -07:00
Alexander Ulanov 26867ebc67 [SPARK-11262][ML] Unit test for gradient, loss layers, memory management for multilayer perceptron
1.Implement LossFunction trait and implement squared error and cross entropy
loss with it
2.Implement unit test for gradient and loss
3.Implement InPlace trait and in-place layer evaluation
4.Refactor interface for ActivationFunction
5.Update of Layer and LayerModel interfaces
6.Fix random weights assignment
7.Implement memory allocation by MLP model instead of individual layers

These features decreased the memory usage and increased flexibility of
internal API.

Author: Alexander Ulanov <nashb@yandex.ru>
Author: avulanov <avulanov@gmail.com>

Closes #9229 from avulanov/mlp-refactoring.
2016-03-31 23:48:36 -07:00
Davies Liu f0afafdc5d [SPARK-14267] [SQL] [PYSPARK] execute multiple Python UDFs within single batch
## What changes were proposed in this pull request?

This PR support multiple Python UDFs within single batch, also improve the performance.

```python
>>> from pyspark.sql.types import IntegerType
>>> sqlContext.registerFunction("double", lambda x: x * 2, IntegerType())
>>> sqlContext.registerFunction("add", lambda x, y: x + y, IntegerType())
>>> sqlContext.sql("SELECT double(add(1, 2)), add(double(2), 1)").explain(True)
== Parsed Logical Plan ==
'Project [unresolvedalias('double('add(1, 2)), None),unresolvedalias('add('double(2), 1), None)]
+- OneRowRelation$

== Analyzed Logical Plan ==
double(add(1, 2)): int, add(double(2), 1): int
Project [double(add(1, 2))#14,add(double(2), 1)#15]
+- Project [double(add(1, 2))#14,add(double(2), 1)#15]
   +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
      +- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
         +- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
            +- OneRowRelation$

== Optimized Logical Plan ==
Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
+- EvaluatePython [add(pythonUDF1#17, 1)], [pythonUDF0#18]
   +- EvaluatePython [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
      +- OneRowRelation$

== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF0#16 AS double(add(1, 2))#14,pythonUDF0#18 AS add(double(2), 1)#15]
:     +- INPUT
+- !BatchPythonEvaluation [add(pythonUDF1#17, 1)], [pythonUDF0#16,pythonUDF1#17,pythonUDF0#18]
   +- !BatchPythonEvaluation [double(add(1, 2)),double(2)], [pythonUDF0#16,pythonUDF1#17]
      +- Scan OneRowRelation[]
```

## How was this patch tested?

Added new tests.

Using the following script to benchmark 1, 2 and 3 udfs,
```
df = sqlContext.range(1, 1 << 23, 1, 4)
double = F.udf(lambda x: x * 2, LongType())
print df.select(double(df.id)).count()
print df.select(double(df.id), double(df.id + 1)).count()
print df.select(double(df.id), double(df.id + 1), double(df.id + 2)).count()
```
Here is the results:

N | Before | After  | speed up
---- |------------ | -------------|------
1 | 22 s | 7 s |  3.1X
2 | 38 s | 13 s | 2.9X
3 | 58 s | 16 s | 3.6X

This benchmark ran locally with 4 CPUs. For 3 UDFs, it launched 12 Python before before this patch, 4 process after this patch. After this patch, it will use less memory for multiple UDFs than before (less buffering).

Author: Davies Liu <davies@databricks.com>

Closes #12057 from davies/multi_udfs.
2016-03-31 16:40:20 -07:00
sethah b11887c086 [SPARK-14264][PYSPARK][ML] Add feature importance for GBTs in pyspark
## What changes were proposed in this pull request?

Feature importances are exposed in the python API for GBTs.

Other changes:
* Update the random forest feature importance documentation to not repeat decision tree docstring and instead place a reference to it.

## How was this patch tested?

Python doc tests were updated to validate GBT feature importance.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #12056 from sethah/Pyspark_GBT_feature_importance.
2016-03-31 13:00:10 -07:00
Herman van Hovell a9b93e0739 [SPARK-14211][SQL] Remove ANTLR3 based parser
### What changes were proposed in this pull request?

This PR removes the ANTLR3 based parser, and moves the new ANTLR4 based parser into the `org.apache.spark.sql.catalyst.parser package`.

### How was this patch tested?

Existing unit tests.

cc rxin andrewor14 yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #12071 from hvanhovell/SPARK-14211.
2016-03-31 09:25:09 -07:00
Yanbo Liang f301df37cb [SPARK-14152][ML][PYSPARK] MultilayerPerceptronClassifier supports save/load for Python API
## What changes were proposed in this pull request?
```MultilayerPerceptronClassifier``` supports save/load for Python API.

## How was this patch tested?
doctest.

cc mengxr jkbradley yinxusen

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #11952 from yanboliang/spark-14152.
2016-03-30 15:47:01 -07:00
Davies Liu a7a93a116d [SPARK-14215] [SQL] [PYSPARK] Support chained Python UDFs
## What changes were proposed in this pull request?

This PR brings the support for chained Python UDFs, for example

```sql
select udf1(udf2(a))
select udf1(udf2(a) + 3)
select udf1(udf2(a) + udf3(b))
```

Also directly chained unary Python UDFs are put in single batch of Python UDFs, others may require multiple batches.

For example,
```python
>>> sqlContext.sql("select double(double(1))").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF#10 AS double(double(1))#9]
:     +- INPUT
+- !BatchPythonEvaluation double(double(1)), [pythonUDF#10]
   +- Scan OneRowRelation[]
>>> sqlContext.sql("select double(double(1) + double(2))").explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [pythonUDF#19 AS double((double(1) + double(2)))#16]
:     +- INPUT
+- !BatchPythonEvaluation double((pythonUDF#17 + pythonUDF#18)), [pythonUDF#17,pythonUDF#18,pythonUDF#19]
   +- !BatchPythonEvaluation double(2), [pythonUDF#17,pythonUDF#18]
      +- !BatchPythonEvaluation double(1), [pythonUDF#17]
         +- Scan OneRowRelation[]
```

TODO: will support multiple unrelated Python UDFs in one batch (another PR).

## How was this patch tested?

Added new unit tests for chained UDFs.

Author: Davies Liu <davies@databricks.com>

Closes #12014 from davies/py_udfs.
2016-03-29 15:06:29 -07:00
wm624@hotmail.com 63b200e8d4 [SPARK-14071][PYSPARK][ML] Change MLWritable.write to be a property
Add property to MLWritable.write method, so we can use .write instead of .write()

Add a new test to ml/test.py to check whether the write is a property.
./python/run-tests --python-executables=python2.7 --modules=pyspark-ml

Will test against the following Python executables: ['python2.7']
Will test the following Python modules: ['pyspark-ml']
Finished test(python2.7): pyspark.ml.evaluation (11s)
Finished test(python2.7): pyspark.ml.clustering (16s)
Finished test(python2.7): pyspark.ml.classification (24s)
Finished test(python2.7): pyspark.ml.recommendation (24s)
Finished test(python2.7): pyspark.ml.feature (39s)
Finished test(python2.7): pyspark.ml.regression (26s)
Finished test(python2.7): pyspark.ml.tuning (15s)
Finished test(python2.7): pyspark.ml.tests (30s)
Tests passed in 55 seconds

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #11945 from wangmiao1981/fix_property.
2016-03-28 22:33:25 -07:00
zero323 39f743a623 [SPARK-14202] [PYTHON] Use generator expression instead of list comp in python_full_outer_jo…
## What changes were proposed in this pull request?

This PR replaces list comprehension in python_full_outer_join.dispatch with a generator expression.

## How was this patch tested?

PySpark-Core, PySpark-MLlib test suites against Python 2.7, 3.5.

Author: zero323 <matthew.szymkiewicz@gmail.com>

Closes #11998 from zero323/pyspark-join-generator-expr.
2016-03-28 14:51:36 -07:00
Herman van Hovell 600c0b69ca [SPARK-13713][SQL] Migrate parser from ANTLR3 to ANTLR4
### What changes were proposed in this pull request?
The current ANTLR3 parser is quite complex to maintain and suffers from code blow-ups. This PR introduces a new parser that is based on ANTLR4.

This parser is based on the [Presto's SQL parser](https://github.com/facebook/presto/blob/master/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4). The current implementation can parse and create Catalyst and SQL plans. Large parts of the HiveQl DDL and some of the DML functionality is currently missing, the plan is to add this in follow-up PRs.

This PR is a work in progress, and work needs to be done in the following area's:

- [x] Error handling should be improved.
- [x] Documentation should be improved.
- [x] Multi-Insert needs to be tested.
- [ ] Naming and package locations.

### How was this patch tested?

Catalyst and SQL unit tests.

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #11557 from hvanhovell/ngParser.
2016-03-28 12:31:12 -07:00
Shixiong Zhu d23ad7c1c9 [SPARK-13874][DOC] Remove docs of streaming-akka, streaming-zeromq, streaming-mqtt and streaming-twitter
## What changes were proposed in this pull request?

This PR removes all docs about the old streaming-akka, streaming-zeromq, streaming-mqtt and streaming-twitter projects since I have already copied them to https://github.com/spark-packages

Also remove mqtt_wordcount.py that I forgot to remove previously.

## How was this patch tested?

Jenkins PR Build.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11824 from zsxwing/remove-doc.
2016-03-26 01:47:27 -07:00
Shixiong Zhu 24587ce433 [SPARK-14073][STREAMING][TEST-MAVEN] Move flume back to Spark
## What changes were proposed in this pull request?

This PR moves flume back to Spark as per the discussion in the dev mail-list.

## How was this patch tested?

Existing Jenkins tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11895 from zsxwing/move-flume-back.
2016-03-25 17:37:16 -07:00
Wenchen Fan 43b15e01c4 [SPARK-14061][SQL] implement CreateMap
## What changes were proposed in this pull request?

As we have `CreateArray` and `CreateStruct`, we should also have `CreateMap`.  This PR adds the `CreateMap` expression, and the DataFrame API, and python API.

## How was this patch tested?

various new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11879 from cloud-fan/create_map.
2016-03-25 09:50:06 -07:00
Andrew Or 20ddf5fddf [SPARK-14014][SQL] Integrate session catalog (attempt #2)
## What changes were proposed in this pull request?

This reopens #11836, which was merged but promptly reverted because it introduced flaky Hive tests.

## How was this patch tested?

See `CatalogTestCases`, `SessionCatalogSuite` and `HiveContextSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #11938 from andrewor14/session-catalog-again.
2016-03-24 22:59:35 -07:00
Reynold Xin 3619fec1ec [SPARK-14142][SQL] Replace internal use of unionAll with union
## What changes were proposed in this pull request?
unionAll has been deprecated in SPARK-14088.

## How was this patch tested?
Should be covered by all existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11946 from rxin/SPARK-14142.
2016-03-24 22:34:55 -07:00
GayathriMurali 0874ff3aad [SPARK-13949][ML][PYTHON] PySpark ml DecisionTreeClassifier, Regressor support export/import
## What changes were proposed in this pull request?

Added MLReadable and MLWritable to Decision Tree Classifier and Regressor. Added doctests.

## How was this patch tested?

Python Unit tests. Tests added to check persistence in DecisionTreeClassifier and DecisionTreeRegressor.

Author: GayathriMurali <gayathri.m.softie@gmail.com>

Closes #11892 from GayathriMurali/SPARK-13949.
2016-03-24 19:20:49 -07:00
sethah 585097716c [SPARK-14107][PYSPARK][ML] Add seed as named argument to GBTs in pyspark
## What changes were proposed in this pull request?

GBTs in pyspark previously had seed parameters, but they could not be passed as keyword arguments through the class constructor. This patch adds seed as a keyword argument and also sets default value.

## How was this patch tested?

Doc tests were updated to pass a random seed through the GBTClassifier and GBTRegressor constructors.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #11944 from sethah/SPARK-14107.
2016-03-24 19:14:24 -07:00
Andrew Or c44d140cae Revert "[SPARK-14014][SQL] Replace existing catalog with SessionCatalog"
This reverts commit 5dfc01976b.
2016-03-23 22:21:15 -07:00
Joseph K. Bradley cf823bead1 [SPARK-12183][ML][MLLIB] Remove mllib tree implementation, and wrap spark.ml one
Primary change:
* Removed spark.mllib.tree.DecisionTree implementation of tree and forest learning.
* spark.mllib now calls the spark.ml implementation.
* Moved unit tests (of tree learning internals) from spark.mllib to spark.ml as needed.

ml.tree.DecisionTreeModel
* Added toOld and made ```private[spark]```, implemented for Classifier and Regressor in subclasses.  These methods now use OldInformationGainStats.invalidInformationGainStats for LeafNodes in order to mimic the spark.mllib implementation.

ml.tree.Node
* Added ```private[tree] def deepCopy```, used by unit tests

Copied developer comments from spark.mllib implementation to spark.ml one.

Moving unit tests
* Tree learning internals were tested by spark.mllib.tree.DecisionTreeSuite, or spark.mllib.tree.RandomForestSuite.
* Those tests were all moved to spark.ml.tree.impl.RandomForestSuite.  The order in the file + the test names are the same, so you should be able to compare them by opening them in 2 windows side-by-side.
* I made minimal changes to each test to allow it to run.  Each test makes the same checks as before, except for a few removed assertions which were checking irrelevant values.
* No new unit tests were added.
* mllib.tree.DecisionTreeSuite: I removed some checks of splits and bins which were not relevant to the unit tests they were in.  Those same split calculations were already being tested in other unit tests, for each dataset type.

**Changes of behavior** (to be noted in SPARK-13448 once this PR is merged)
* spark.ml.tree.impl.RandomForest: Rather than throwing an error when maxMemoryInMB is set to too small a value (to split any node), we now allow 1 node to be split, even if its memory requirements exceed maxMemoryInMB.  This involved removing the maxMemoryPerNode check in RandomForest.run, as well as modifying selectNodesToSplit().  Once this PR is merged, I will note the change of behavior on SPARK-13448.
* spark.mllib.tree.DecisionTree: When a tree only has one node (root = leaf node), the "stats" field will now be empty, rather than being set to InformationGainStats.invalidInformationGainStats.  This does not remove information from the tree, and it will save a bit of storage.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #11855 from jkbradley/remove-mllib-tree-impl.
2016-03-23 21:16:00 -07:00
Andrew Or 5dfc01976b [SPARK-14014][SQL] Replace existing catalog with SessionCatalog
## What changes were proposed in this pull request?

`SessionCatalog`, introduced in #11750, is a catalog that keeps track of temporary functions and tables, and delegates metastore operations to `ExternalCatalog`. This functionality overlaps a lot with the existing `analysis.Catalog`.

As of this commit, `SessionCatalog` and `ExternalCatalog` will no longer be dead code. There are still things that need to be done after this patch, namely:
- SPARK-14013: Properly implement temporary functions in `SessionCatalog`
- SPARK-13879: Decide which DDL/DML commands to support natively in Spark
- SPARK-?????: Implement the ones we do want to support through `SessionCatalog`.
- SPARK-?????: Merge SQL/HiveContext

## How was this patch tested?

This is largely a refactoring task so there are no new tests introduced. The particularly relevant tests are `SessionCatalogSuite` and `ExternalCatalogSuite`.

Author: Andrew Or <andrew@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #11836 from andrewor14/use-session-catalog.
2016-03-23 13:34:22 -07:00
sethah 30bdb5cbd9 [SPARK-13068][PYSPARK][ML] Type conversion for Pyspark params
## What changes were proposed in this pull request?

This patch adds type conversion functionality for parameters in Pyspark. A `typeConverter` field is added to the constructor of `Param` class. This argument is a function which converts values passed to this param to the appropriate type if possible. This is beneficial so that the params can fail at set time if they are given inappropriate values, but even more so because coherent error messages are now provided when Py4J cannot cast the python type to the appropriate Java type.

This patch also adds a `TypeConverters` class with factory methods for common type conversions. Most of the changes involve adding these factory type converters to existing params. The previous solution to this issue, `expectedType`, is deprecated and can be removed in 2.1.0 as discussed on the Jira.

## How was this patch tested?

Unit tests were added in python/pyspark/ml/tests.py to test parameter type conversion. These tests check that values that should be convertible are converted correctly, and that the appropriate errors are thrown when invalid values are provided.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #11663 from sethah/SPARK-13068-tc.
2016-03-23 11:20:44 -07:00
Reynold Xin 926a93e54b [SPARK-14088][SQL] Some Dataset API touch-up
## What changes were proposed in this pull request?
1. Deprecated unionAll. It is pretty confusing to have both "union" and "unionAll" when the two do the same thing in Spark but are different in SQL.
2. Rename reduce in KeyValueGroupedDataset to reduceGroups so it is more consistent with rest of the functions in KeyValueGroupedDataset. Also makes it more obvious what "reduce" and "reduceGroups" mean. Previously it was confusing because it could be reducing a Dataset, or just reducing groups.
3. Added a "name" function, which is more natural to name columns than "as" for non-SQL users.
4. Remove "subtract" function since it is just an alias for "except".

## How was this patch tested?
All changes should be covered by existing tests. Also added couple test cases to cover "name".

Author: Reynold Xin <rxin@databricks.com>

Closes #11908 from rxin/SPARK-14088.
2016-03-22 23:43:09 -07:00
Joseph K. Bradley 7e3423b9c0 [SPARK-13951][ML][PYTHON] Nested Pipeline persistence
Adds support for saving and loading nested ML Pipelines from Python.  Pipeline and PipelineModel do not extend JavaWrapper, but they are able to utilize the JavaMLWriter, JavaMLReader implementations.

Also:
* Separates out interfaces from Java wrapper implementations for MLWritable, MLReadable, MLWriter, MLReader.
* Moves methods _stages_java2py, _stages_py2java into Pipeline, PipelineModel as _transfer_stage_from_java, _transfer_stage_to_java

Added new unit test for nested Pipelines.  Abstracted validity check into a helper method for the 2 unit tests.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #11866 from jkbradley/nested-pipeline-io.
Closes #11835
2016-03-22 12:11:37 -07:00
hyukjinkwon 4e09a0d5ea [SPARK-13953][SQL] Specifying the field name for corrupted record via option at JSON datasource
## What changes were proposed in this pull request?

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

Currently, JSON data source creates a new field in `PERMISSIVE` mode for storing malformed string.
This field can be renamed via `spark.sql.columnNameOfCorruptRecord` option but it is a global configuration.

This PR make that option can be applied per read and can be specified via `option()`. This will overwrites `spark.sql.columnNameOfCorruptRecord` if it is set.

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for coding style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11881 from HyukjinKwon/SPARK-13953.
2016-03-22 20:30:48 +08:00
zero323 8193a266b5 [SPARK-14058][PYTHON] Incorrect docstring in Window.order
## What changes were proposed in this pull request?

Replaces current docstring ("Creates a :class:`WindowSpec` with the partitioning defined.") with "Creates a :class:`WindowSpec` with the ordering defined."

## How was this patch tested?

PySpark unit tests (no regression introduced). No changes to the code.

Author: zero323 <matthew.szymkiewicz@gmail.com>

Closes #11877 from zero323/order-by-description.
2016-03-21 23:52:33 -07:00
hyukjinkwon e474088144 [SPARK-13764][SQL] Parse modes in JSON data source
## What changes were proposed in this pull request?

Currently, there is no way to control the behaviour when fails to parse corrupt records in JSON data source .

This PR adds the support for parse modes just like CSV data source. There are three modes below:

- `PERMISSIVE` :  When it fails to parse, this sets `null` to to field. This is a default mode when it has been this mode.
- `DROPMALFORMED`: When it fails to parse, this drops the whole record.
- `FAILFAST`: When it fails to parse, it just throws an exception.

This PR also make JSON data source share the `ParseModes` in CSV data source.

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for code style tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11756 from HyukjinKwon/SPARK-13764.
2016-03-21 15:42:35 +08:00
Xusen Yin 454a00df2a [SPARK-13993][PYSPARK] Add pyspark Rformula/RforumlaModel save/load
## What changes were proposed in this pull request?

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

## How was this patch tested?

doctest

Author: Xusen Yin <yinxusen@gmail.com>

Closes #11807 from yinxusen/SPARK-13993.
2016-03-20 15:34:34 -07:00
Bryan Cutler 828213d4ca [SPARK-13937][PYSPARK][ML] Change JavaWrapper _java_obj from static to member variable
## What changes were proposed in this pull request?
In PySpark wrapper.py JavaWrapper change _java_obj from an unused static variable to a member variable that is consistent with usage in derived classes.

## How was this patch tested?
Ran python tests for ML and MLlib.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #11767 from BryanCutler/JavaWrapper-static-_java_obj-SPARK-13937.
2016-03-17 10:16:51 -07:00
GayathriMurali 27e1f38851 [SPARK-13034] PySpark ml.classification support export/import
## What changes were proposed in this pull request?

Add export/import for all estimators and transformers(which have Scala implementation) under pyspark/ml/classification.py.

## How was this patch tested?

./python/run-tests
./dev/lint-python
Unit tests added to check persistence in Logistic Regression

Author: GayathriMurali <gayathri.m.softie@gmail.com>

Closes #11707 from GayathriMurali/SPARK-13034.
2016-03-16 14:21:42 -07:00
Xusen Yin ae6c677c8a [SPARK-13038][PYSPARK] Add load/save to pipeline
## What changes were proposed in this pull request?

JIRA issue: https://issues.apache.org/jira/browse/SPARK-13038

1. Add load/save to PySpark Pipeline and PipelineModel

2. Add `_transfer_stage_to_java()` and `_transfer_stage_from_java()` for `JavaWrapper`.

## How was this patch tested?

Test with doctest.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #11683 from yinxusen/SPARK-13038-only.
2016-03-16 13:49:40 -07:00
Reynold Xin 8e0b030606 [SPARK-10380][SQL] Fix confusing documentation examples for astype/drop_duplicates.
## What changes were proposed in this pull request?
We have seen users getting confused by the documentation for astype and drop_duplicates, because the examples in them do not use these functions (but do uses their aliases). This patch simply removes all examples for these functions, and say that they are aliases.

## How was this patch tested?
Existing PySpark unit tests.

Closes #11543.

Author: Reynold Xin <rxin@databricks.com>

Closes #11698 from rxin/SPARK-10380.
2016-03-14 19:25:49 -07:00
Shixiong Zhu 06dec37455 [SPARK-13843][STREAMING] Remove streaming-flume, streaming-mqtt, streaming-zeromq, streaming-akka, streaming-twitter to Spark packages
## What changes were proposed in this pull request?

Currently there are a few sub-projects, each for integrating with different external sources for Streaming.  Now that we have better ability to include external libraries (spark packages) and with Spark 2.0 coming up, we can move the following projects out of Spark to https://github.com/spark-packages

- streaming-flume
- streaming-akka
- streaming-mqtt
- streaming-zeromq
- streaming-twitter

They are just some ancillary packages and considering the overhead of maintenance, running tests and PR failures, it's better to maintain them out of Spark. In addition, these projects can have their different release cycles and we can release them faster.

I have already copied these projects to https://github.com/spark-packages

## How was this patch tested?

Jenkins tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11672 from zsxwing/remove-external-pkg.
2016-03-14 16:56:04 -07:00
Josh Rosen 07cb323e7a [SPARK-13848][SPARK-5185] Update to Py4J 0.9.2 in order to fix classloading issue
This patch upgrades Py4J from 0.9.1 to 0.9.2 in order to include a patch which modifies Py4J to use the current thread's ContextClassLoader when performing reflection / class loading. This is necessary in order to fix [SPARK-5185](https://issues.apache.org/jira/browse/SPARK-5185), a longstanding issue affecting the use of `--jars` and `--packages` in PySpark.

In order to demonstrate that the fix works, I removed the workarounds which were added as part of [SPARK-6027](https://issues.apache.org/jira/browse/SPARK-6027) / #4779 and other patches.

Py4J diff: https://github.com/bartdag/py4j/compare/0.9.1...0.9.2

/cc zsxwing tdas davies brkyvz

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11687 from JoshRosen/py4j-0.9.2.
2016-03-14 12:22:02 -07:00
Davies Liu ba8c86d06f [SPARK-13671] [SPARK-13311] [SQL] Use different physical plans for RDD and data sources
## What changes were proposed in this pull request?

This PR split the PhysicalRDD into two classes, PhysicalRDD and PhysicalScan. PhysicalRDD is used for DataFrames that is created from existing RDD. PhysicalScan is used for DataFrame that is created from data sources. This enable use to apply different optimization on both of them.

Also fix the problem for sameResult() on two DataSourceScan.

Also fix the equality check to toString for `In`. It's better to use Seq there, but we can't break this public API (sad).

## How was this patch tested?

Existing tests. Manually tested with TPCDS query Q59 and Q64, all those duplicated exchanges can be re-used now, also saw there are 40+% performance improvement (saving half of the scan).

Author: Davies Liu <davies@databricks.com>

Closes #11514 from davies/existing_rdd.
2016-03-12 00:48:36 -08:00
Josh Rosen 073bf9d4d9 [SPARK-13807] De-duplicate Python*Helper instantiation code in PySpark streaming
This patch de-duplicates code in PySpark streaming which loads the `Python*Helper` classes. I also changed a few `raise e` statements to simply `raise` in order to preserve the full exception stacktrace when re-throwing.

Here's a link to the whitespace-change-free diff: https://github.com/apache/spark/compare/master...JoshRosen:pyspark-reflection-deduplication?w=0

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11641 from JoshRosen/pyspark-reflection-deduplication.
2016-03-11 11:18:51 -08:00
sethah 234f781ae1 [SPARK-13787][ML][PYSPARK] Pyspark feature importances for decision tree and random forest
## What changes were proposed in this pull request?

This patch adds a `featureImportance` property to the Pyspark API for `DecisionTreeRegressionModel`, `DecisionTreeClassificationModel`, `RandomForestRegressionModel` and `RandomForestClassificationModel`.

## How was this patch tested?

Python doc tests for the affected classes were updated to check feature importances.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #11622 from sethah/SPARK-13787.
2016-03-11 09:54:23 +02:00
Zheng RuiFeng d18276cb1d [SPARK-13672][ML] Add python examples of BisectingKMeans in ML and MLLIB
JIRA: https://issues.apache.org/jira/browse/SPARK-13672

## What changes were proposed in this pull request?

add two python examples of BisectingKMeans for ml and mllib

## How was this patch tested?

manual tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #11515 from zhengruifeng/mllib_bkm_pe.
2016-03-11 09:21:12 +02:00
Cheng Lian 1d542785b9 [SPARK-13244][SQL] Migrates DataFrame to Dataset
## What changes were proposed in this pull request?

This PR unifies DataFrame and Dataset by migrating existing DataFrame operations to Dataset and make `DataFrame` a type alias of `Dataset[Row]`.

Most Scala code changes are source compatible, but Java API is broken as Java knows nothing about Scala type alias (mostly replacing `DataFrame` with `Dataset<Row>`).

There are several noticeable API changes related to those returning arrays:

1.  `collect`/`take`

    -   Old APIs in class `DataFrame`:

        ```scala
        def collect(): Array[Row]
        def take(n: Int): Array[Row]
        ```

    -   New APIs in class `Dataset[T]`:

        ```scala
        def collect(): Array[T]
        def take(n: Int): Array[T]

        def collectRows(): Array[Row]
        def takeRows(n: Int): Array[Row]
        ```

    Two specialized methods `collectRows` and `takeRows` are added because Java doesn't support returning generic arrays. Thus, for example, `DataFrame.collect(): Array[T]` actually returns `Object` instead of `Array<T>` from Java side.

    Normally, Java users may fall back to `collectAsList` and `takeAsList`.  The two new specialized versions are added to avoid performance regression in ML related code (but maybe I'm wrong and they are not necessary here).

1.  `randomSplit`

    -   Old APIs in class `DataFrame`:

        ```scala
        def randomSplit(weights: Array[Double], seed: Long): Array[DataFrame]
        def randomSplit(weights: Array[Double]): Array[DataFrame]
        ```

    -   New APIs in class `Dataset[T]`:

        ```scala
        def randomSplit(weights: Array[Double], seed: Long): Array[Dataset[T]]
        def randomSplit(weights: Array[Double]): Array[Dataset[T]]
        ```

    Similar problem as above, but hasn't been addressed for Java API yet.  We can probably add `randomSplitAsList` to fix this one.

1.  `groupBy`

    Some original `DataFrame.groupBy` methods have conflicting signature with original `Dataset.groupBy` methods.  To distinguish these two, typed `Dataset.groupBy` methods are renamed to `groupByKey`.

Other noticeable changes:

1.  Dataset always do eager analysis now

    We used to support disabling DataFrame eager analysis to help reporting partially analyzed malformed logical plan on analysis failure.  However, Dataset encoders requires eager analysi during Dataset construction.  To preserve the error reporting feature, `AnalysisException` now takes an extra `Option[LogicalPlan]` argument to hold the partially analyzed plan, so that we can check the plan tree when reporting test failures.  This plan is passed by `QueryExecution.assertAnalyzed`.

## How was this patch tested?

Existing tests do the work.

## TODO

- [ ] Fix all tests
- [ ] Re-enable MiMA check
- [ ] Update ScalaDoc (`since`, `group`, and example code)

Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Cheng Lian <liancheng@users.noreply.github.com>

Closes #11443 from liancheng/ds-to-df.
2016-03-10 17:00:17 -08:00
Tristan Reid 5f7dbdba6f [MINOR] Fix typo in 'hypot' docstring
Minor typo:  docstring for pyspark.sql.functions: hypot has extra characters

N/A

Author: Tristan Reid <treid@netflix.com>

Closes #11616 from tristanreid/master.
2016-03-09 18:05:03 -08:00
Sean Owen 256704c771 [SPARK-13595][BUILD] Move docker, extras modules into external
## What changes were proposed in this pull request?

Move `docker` dirs out of top level into `external/`; move `extras/*` into `external/`

## How was this patch tested?

This is tested with Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #11523 from srowen/SPARK-13595.
2016-03-09 18:27:44 +00:00
Bryan Cutler d8813fa043 [SPARK-13625][PYSPARK][ML] Added a check to see if an attribute is a property when getting param list
## What changes were proposed in this pull request?

Added a check in pyspark.ml.param.Param.params() to see if an attribute is a property (decorated with `property`) before checking if it is a `Param` instance.  This prevents the property from being invoked to 'get' this attribute, which could possibly cause an error.

## How was this patch tested?

Added a test case with a class has a property that will raise an error when invoked and then call`Param.params` to verify that the property is not invoked, but still able to find another property in the class.  Also ran pyspark-ml test before fix that will trigger an error, and again after the fix to verify that the error was resolved and the method was working properly.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #11476 from BryanCutler/pyspark-ml-property-attr-SPARK-13625.
2016-03-08 17:34:25 -08:00
Wenchen Fan d57daf1f77 [SPARK-13593] [SQL] improve the createDataFrame to accept data type string and verify the data
## What changes were proposed in this pull request?

This PR improves the `createDataFrame` method to make it also accept datatype string, then users can convert python RDD to DataFrame easily, for example, `df = rdd.toDF("a: int, b: string")`.
It also supports flat schema so users can convert an RDD of int to DataFrame directly, we will automatically wrap int to row for users.
If schema is given, now we checks if the real data matches the given schema, and throw error if it doesn't.

## How was this patch tested?

new tests in `test.py` and doc test in `types.py`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11444 from cloud-fan/pyrdd.
2016-03-08 14:00:03 -08:00
Wenchen Fan d5ce61722f [SPARK-13740][SQL] add null check for _verify_type in types.py
## What changes were proposed in this pull request?

This PR adds null check in `_verify_type` according to the nullability information.

## How was this patch tested?

new doc tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11574 from cloud-fan/py-null-check.
2016-03-08 13:46:17 -08:00
Sean Owen 0eea12a3d9 [SPARK-13596][BUILD] Move misc top-level build files into appropriate subdirs
## What changes were proposed in this pull request?

Move many top-level files in dev/ or other appropriate directory. In particular, put `make-distribution.sh` in `dev` and update docs accordingly. Remove deprecated `sbt/sbt`.

I was (so far) unable to figure out how to move `tox.ini`. `scalastyle-config.xml` should be movable but edits to the project `.sbt` files didn't work; config file location is updatable for compile but not test scope.

## How was this patch tested?

`./dev/run-tests` to verify RAT and checkstyle work. Jenkins tests for the rest.

Author: Sean Owen <sowen@cloudera.com>

Closes #11522 from srowen/SPARK-13596.
2016-03-07 14:48:02 -08:00
Dongjoon Hyun e72914f37d [SPARK-12243][BUILD][PYTHON] PySpark tests are slow in Jenkins.
## What changes were proposed in this pull request?

In the Jenkins pull request builder, PySpark tests take around [962 seconds ](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52530/console) of end-to-end time to run, despite the fact that we run four Python test suites in parallel. According to the log, the basic reason is that the long running test starts at the end due to FIFO queue. We first try to reduce the test time by just starting some long running tests first with simple priority queue.

```
========================================================================
Running PySpark tests
========================================================================
...
Finished test(python3.4): pyspark.streaming.tests (213s)
Finished test(pypy): pyspark.sql.tests (92s)
Finished test(pypy): pyspark.streaming.tests (280s)
Tests passed in 962 seconds
```

## How was this patch tested?

Manual check.
Check 'Running PySpark tests' part of the Jenkins log.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11551 from dongjoon-hyun/SPARK-12243.
2016-03-07 12:06:46 -08:00
Shixiong Zhu ee913e6e2d [SPARK-13697] [PYSPARK] Fix the missing module name of TransformFunctionSerializer.loads
## What changes were proposed in this pull request?

Set the function's module name to `__main__` if it's missing in `TransformFunctionSerializer.loads`.

## How was this patch tested?

Manually test in the shell.

Before this patch:
```
>>> from pyspark.streaming import StreamingContext
>>> from pyspark.streaming.util import TransformFunction
>>> ssc = StreamingContext(sc, 1)
>>> func = TransformFunction(sc, lambda x: x, sc.serializer)
>>> func.rdd_wrapper(lambda x: x)
TransformFunction(<function <lambda> at 0x106ac8b18>)
>>> bytes = bytearray(ssc._transformerSerializer.serializer.dumps((func.func, func.rdd_wrap_func, func.deserializers)))
>>> func2 = ssc._transformerSerializer.loads(bytes)
>>> print(func2.func.__module__)
None
>>> print(func2.rdd_wrap_func.__module__)
None
>>>
```
After this patch:
```
>>> from pyspark.streaming import StreamingContext
>>> from pyspark.streaming.util import TransformFunction
>>> ssc = StreamingContext(sc, 1)
>>> func = TransformFunction(sc, lambda x: x, sc.serializer)
>>> func.rdd_wrapper(lambda x: x)
TransformFunction(<function <lambda> at 0x108bf1b90>)
>>> bytes = bytearray(ssc._transformerSerializer.serializer.dumps((func.func, func.rdd_wrap_func, func.deserializers)))
>>> func2 = ssc._transformerSerializer.loads(bytes)
>>> print(func2.func.__module__)
__main__
>>> print(func2.rdd_wrap_func.__module__)
__main__
>>>
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11535 from zsxwing/loads-module.
2016-03-06 08:57:01 -08:00
gatorsmile adce5ee721 [SPARK-12720][SQL] SQL Generation Support for Cube, Rollup, and Grouping Sets
#### What changes were proposed in this pull request?

This PR is for supporting SQL generation for cube, rollup and grouping sets.

For example, a query using rollup:
```SQL
SELECT count(*) as cnt, key % 5, grouping_id() FROM t1 GROUP BY key % 5 WITH ROLLUP
```
Original logical plan:
```
  Aggregate [(key#17L % cast(5 as bigint))#47L,grouping__id#46],
            [(count(1),mode=Complete,isDistinct=false) AS cnt#43L,
             (key#17L % cast(5 as bigint))#47L AS _c1#45L,
             grouping__id#46 AS _c2#44]
  +- Expand [List(key#17L, value#18, (key#17L % cast(5 as bigint))#47L, 0),
             List(key#17L, value#18, null, 1)],
            [key#17L,value#18,(key#17L % cast(5 as bigint))#47L,grouping__id#46]
     +- Project [key#17L,
                 value#18,
                 (key#17L % cast(5 as bigint)) AS (key#17L % cast(5 as bigint))#47L]
        +- Subquery t1
           +- Relation[key#17L,value#18] ParquetRelation
```
Converted SQL:
```SQL
  SELECT count( 1) AS `cnt`,
         (`t1`.`key` % CAST(5 AS BIGINT)),
         grouping_id() AS `_c2`
  FROM `default`.`t1`
  GROUP BY (`t1`.`key` % CAST(5 AS BIGINT))
  GROUPING SETS (((`t1`.`key` % CAST(5 AS BIGINT))), ())
```

#### How was the this patch tested?

Added eight test cases in `LogicalPlanToSQLSuite`.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #11283 from gatorsmile/groupingSetsToSQL.
2016-03-05 19:25:03 +08:00
Xusen Yin 83302c3bff [SPARK-13036][SPARK-13318][SPARK-13319] Add save/load for feature.py
Add save/load for feature.py. Meanwhile, add save/load for `ElementwiseProduct` in Scala side and fix a bug of missing `setDefault` in `VectorSlicer` and `StopWordsRemover`.

In this PR I ignore the `RFormula` and `RFormulaModel` because its Scala implementation is pending in https://github.com/apache/spark/pull/9884. I'll add them in this PR if https://github.com/apache/spark/pull/9884 gets merged first. Or add a follow-up JIRA for `RFormula`.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #11203 from yinxusen/SPARK-13036.
2016-03-04 08:32:24 -08:00
Dongjoon Hyun c8f25459ed [SPARK-13676] Fix mismatched default values for regParam in LogisticRegression
## What changes were proposed in this pull request?

The default value of regularization parameter for `LogisticRegression` algorithm is different in Scala and Python. We should provide the same value.

**Scala**
```
scala> new org.apache.spark.ml.classification.LogisticRegression().getRegParam
res0: Double = 0.0
```

**Python**
```
>>> from pyspark.ml.classification import LogisticRegression
>>> LogisticRegression().getRegParam()
0.1
```

## How was this patch tested?
manual. Check the following in `pyspark`.
```
>>> from pyspark.ml.classification import LogisticRegression
>>> LogisticRegression().getRegParam()
0.0
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11519 from dongjoon-hyun/SPARK-13676.
2016-03-04 08:25:41 -08:00
Wenchen Fan 15d57f9c23 [SPARK-13647] [SQL] also check if numeric value is within allowed range in _verify_type
## What changes were proposed in this pull request?

This PR makes the `_verify_type` in `types.py` more strict, also check if numeric value is within allowed range.

## How was this patch tested?

newly added doc test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11492 from cloud-fan/py-verify.
2016-03-03 20:16:37 -08:00
Dongjoon Hyun 941b270b70 [MINOR] Fix typos in comments and testcase name of code
## What changes were proposed in this pull request?

This PR fixes typos in comments and testcase name of code.

## How was this patch tested?

manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11481 from dongjoon-hyun/minor_fix_typos_in_code.
2016-03-03 22:42:12 +00:00
hyukjinkwon cf95d728c6 [SPARK-13543][SQL] Support for specifying compression codec for Parquet/ORC via option()
## What changes were proposed in this pull request?

This PR adds the support to specify compression codecs for both ORC and Parquet.

## How was this patch tested?

unittests within IDE and code style tests with `dev/run_tests`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #11464 from HyukjinKwon/SPARK-13543.
2016-03-03 10:30:55 -08:00
JeremyNixon 511d4929c8 [SPARK-12877][ML] Add train-validation-split to pyspark
## What changes were proposed in this pull request?
The changes proposed were to add train-validation-split to pyspark.ml.tuning.

## How was the this patch tested?
This patch was tested through unit tests located in pyspark/ml/test.py.

This is my original work and I license it to Spark.

Author: JeremyNixon <jnixon2@gmail.com>

Closes #11335 from JeremyNixon/tvs_pyspark.
2016-03-03 09:50:05 -08:00
Wenchen Fan 4dd24811d9 [SPARK-13594][SQL] remove typed operations(e.g. map, flatMap) from python DataFrame
## What changes were proposed in this pull request?

Remove `map`, `flatMap`, `mapPartitions` from python DataFrame, to prepare for Dataset API in the future.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11445 from cloud-fan/python-clean.
2016-03-02 15:26:34 -08:00
Joseph K. Bradley 9495c40f22 [SPARK-13008][ML][PYTHON] Put one alg per line in pyspark.ml all lists
This is to fix a long-time annoyance: Whenever we add a new algorithm to pyspark.ml, we have to add it to the ```__all__``` list at the top.  Since we keep it alphabetized, it often creates a lot more changes than needed.  It is also easy to add the Estimator and forget the Model.  I'm going to switch it to have one algorithm per line.

This also alphabetizes a few out-of-place classes in pyspark.ml.feature.  No changes have been made to the moved classes.

CC: thunterdb

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #10927 from jkbradley/ml-python-all-list.
2016-03-01 21:26:47 -08:00
hyukjinkwon 02aa499dfb [SPARK-13509][SPARK-13507][SQL] Support for writing CSV with a single function call
https://issues.apache.org/jira/browse/SPARK-13507
https://issues.apache.org/jira/browse/SPARK-13509

## What changes were proposed in this pull request?
This PR adds the support to write CSV data directly by a single call to the given path.

Several unitests were added for each functionality.
## How was this patch tested?

This was tested with unittests and with `dev/run_tests` for coding style

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #11389 from HyukjinKwon/SPARK-13507-13509.
2016-02-29 09:44:29 -08:00
vijaykiran 236e3c8fbc [SPARK-12633][PYSPARK] [DOC] PySpark regression parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent. This is for the regression module.  Also, updated 2 params in classification to read as `Supported values:` to be consistent.

closes #10600

Author: vijaykiran <mail@vijaykiran.com>
Author: Bryan Cutler <cutlerb@gmail.com>

Closes #11404 from BryanCutler/param-desc-consistent-regression-SPARK-12633.
2016-02-29 15:52:41 +02:00
Yanbo Liang d81a71357e [SPARK-13545][MLLIB][PYSPARK] Make MLlib LogisticRegressionWithLBFGS's default parameters consistent in Scala and Python
## What changes were proposed in this pull request?
* The default value of ```regParam``` of PySpark MLlib ```LogisticRegressionWithLBFGS``` should be consistent with Scala which is ```0.0```. (This is also consistent with ML ```LogisticRegression```.)
* BTW, if we use a known updater(L1 or L2) for binary classification, ```LogisticRegressionWithLBFGS``` will call the ML implementation. We should update the API doc to clarifying ```numCorrections``` will have no effect if we fall into that route.
* Make a pass for all parameters of ```LogisticRegressionWithLBFGS```, others are set properly.

cc mengxr dbtsai
## How was this patch tested?
No new tests, it should pass all current tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #11424 from yanboliang/spark-13545.
2016-02-29 00:55:51 -08:00
zlpmichelle 1e5fcdf96c [SPARK-13505][ML] add python api for MaxAbsScaler
## What changes were proposed in this pull request?
After SPARK-13028, we should add Python API for MaxAbsScaler.

## How was this patch tested?
unit test

Author: zlpmichelle <zlpmichelle@gmail.com>

Closes #11393 from zlpmichelle/master.
2016-02-26 14:37:44 -08:00
Bryan Cutler b33261f913 [SPARK-12634][PYSPARK][DOC] PySpark tree parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent.  This is for the tree module.

closes #10601

Author: Bryan Cutler <cutlerb@gmail.com>
Author: vijaykiran <mail@vijaykiran.com>

Closes #11353 from BryanCutler/param-desc-consistent-tree-SPARK-12634.
2016-02-26 08:30:32 -08:00
Tommy YU f3be369ef7 [SPARK-13033] [ML] [PYSPARK] Add import/export for ml.regression
Add export/import for all estimators and transformers(which have Scala implementation) under pyspark/ml/regression.py.

yanboliang Please help to review.
For doctest, I though it's enough to add one since it's common usage. But I can add to all if we want it.

Author: Tommy YU <tummyyu@163.com>

Closes #11000 from Wenpei/spark-13033-ml.regression-exprot-import and squashes the following commits:

3646b36 [Tommy YU] address review comments
9cddc98 [Tommy YU] change base on review and pr 11197
cc61d9d [Tommy YU] remove default parameter set
19535d4 [Tommy YU] add export/import to regression
44a9dc2 [Tommy YU] add import/export for ml.regression
2016-02-25 21:09:02 -08:00
Yu ISHIKAWA 35316cb0b7 [SPARK-13292] [ML] [PYTHON] QuantileDiscretizer should take random seed in PySpark
## What changes were proposed in this pull request?
QuantileDiscretizer in Python should also specify a random seed.

## How was this patch tested?
unit tests

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #11362 from yu-iskw/SPARK-13292 and squashes the following commits:

02ffa76 [Yu ISHIKAWA] [SPARK-13292][ML][PYTHON] QuantileDiscretizer should take random seed in PySpark
2016-02-25 13:29:10 -08:00
Kai Jiang 4d2864b2d7 [SPARK-7106][MLLIB][PYSPARK] Support model save/load in Python's FPGrowth
## What changes were proposed in this pull request?

Python API supports mode save/load in FPGrowth
JIRA: [https://issues.apache.org/jira/browse/SPARK-7106](https://issues.apache.org/jira/browse/SPARK-7106)
## How was the this patch tested?

The patch is tested with Python doctest.

Author: Kai Jiang <jiangkai@gmail.com>

Closes #11321 from vectorijk/spark-7106.
2016-02-24 23:22:14 -08:00
Joseph K. Bradley 13ce10e954 [SPARK-13479][SQL][PYTHON] Added Python API for approxQuantile
## What changes were proposed in this pull request?

* Scala DataFrameStatFunctions: Added version of approxQuantile taking a List instead of an Array, for Python compatbility
* Python DataFrame and DataFrameStatFunctions: Added approxQuantile

## How was this patch tested?

* unit test in sql/tests.py

Documentation was copied from the existing approxQuantile exactly.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #11356 from jkbradley/approx-quantile-python.
2016-02-24 23:15:36 -08:00
Nong Li 5a7af9e7ac [SPARK-13250] [SQL] Update PhysicallRDD to convert to UnsafeRow if using the vectorized scanner.
Some parts of the engine rely on UnsafeRow which the vectorized parquet scanner does not want
to produce. This add a conversion in Physical RDD. In the case where codegen is used (and the
scan is the start of the pipeline), there is no requirement to use UnsafeRow. This patch adds
update PhysicallRDD to support codegen, which eliminates the need for the UnsafeRow conversion
in all cases.

The result of these changes for TPCDS-Q19 at the 10gb sf reduces the query time from 9.5 seconds
to 6.5 seconds.

Author: Nong Li <nong@databricks.com>

Closes #11141 from nongli/spark-13250.
2016-02-24 17:16:45 -08:00
Wenchen Fan a60f91284c [SPARK-13467] [PYSPARK] abstract python function to simplify pyspark code
## What changes were proposed in this pull request?

When we pass a Python function to JVM side, we also need to send its context, e.g. `envVars`, `pythonIncludes`, `pythonExec`, etc. However, it's annoying to pass around so many parameters at many places. This PR abstract python function along with its context, to simplify some pyspark code and make the logic more clear.

## How was the this patch tested?

by existing unit tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11342 from cloud-fan/python-clean.
2016-02-24 12:44:54 -08:00
Davies Liu c481bdf512 [SPARK-13329] [SQL] considering output for statistics of logical plan
The current implementation of statistics of UnaryNode does not considering output (for example, Project may product much less columns than it's child), we should considering it to have a better guess.

We usually only join with few columns from a parquet table, the size of projected plan could be much smaller than the original parquet files. Having a better guess of size help we choose between broadcast join or sort merge join.

After this PR, I saw a few queries choose broadcast join other than sort merge join without turning spark.sql.autoBroadcastJoinThreshold for every query, ended up with about 6-8X improvements on end-to-end time.

We use `defaultSize` of DataType to estimate the size of a column, currently For DecimalType/StringType/BinaryType and UDT, we are over-estimate too much (4096 Bytes), so this PR change them to some more reasonable values. Here are the new defaultSize for them:

DecimalType:  8 or 16 bytes, based on the precision
StringType:  20 bytes
BinaryType: 100 bytes
UDF: default size of SQL type

These numbers are not perfect (hard to have a perfect number for them), but should be better than 4096.

Author: Davies Liu <davies@databricks.com>

Closes #11210 from davies/statics.
2016-02-23 12:55:44 -08:00
Yanbo Liang 72427c3e11 [SPARK-13429][MLLIB] Unify Logistic Regression convergence tolerance of ML & MLlib
## What changes were proposed in this pull request?
In order to provide better and consistent result, let's change the default value of MLlib ```LogisticRegressionWithLBFGS convergenceTol``` from ```1E-4``` to ```1E-6``` which will be equal to ML ```LogisticRegression```.
cc dbtsai
## How was the this patch tested?
unit tests

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #11299 from yanboliang/spark-13429.
2016-02-22 23:37:09 -08:00
Bryan Cutler e298ac91e3 [SPARK-12632][PYSPARK][DOC] PySpark fpm and als parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent.  This is for the fpm and recommendation modules.

Closes #10602
Closes #10897

Author: Bryan Cutler <cutlerb@gmail.com>
Author: somideshmukh <somilde@us.ibm.com>

Closes #11186 from BryanCutler/param-desc-consistent-fpmrecc-SPARK-12632.
2016-02-22 12:48:37 +02:00
Dongjoon Hyun 024482bf51 [MINOR][DOCS] Fix all typos in markdown files of doc and similar patterns in other comments
## What changes were proposed in this pull request?

This PR tries to fix all typos in all markdown files under `docs` module,
and fixes similar typos in other comments, too.

## How was the this patch tested?

manual tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11300 from dongjoon-hyun/minor_fix_typos.
2016-02-22 09:52:07 +00:00
Yong Gang Cao ef1047fca7 [SPARK-12153][SPARK-7617][MLLIB] add support of arbitrary length sentence and other tuning for Word2Vec
add support of arbitrary length sentence by using the nature representation of sentences in the input.

add new similarity functions and add normalization option for distances in synonym finding
add new accessor for internal structure(the vocabulary and wordindex) for convenience

need instructions about how to set value for the Since annotation for newly added public functions. 1.5.3?

jira link: https://issues.apache.org/jira/browse/SPARK-12153

Author: Yong Gang Cao <ygcao@amazon.com>
Author: Yong-Gang Cao <ygcao@users.noreply.github.com>

Closes #10152 from ygcao/improvementForSentenceBoundary.
2016-02-22 09:47:36 +00:00
Franklyn D'souza 0f90f4e6ac [SPARK-13410][SQL] Support unionAll for DataFrames with UDT columns.
## What changes were proposed in this pull request?

This PR adds equality operators to UDT classes so that they can be correctly tested for dataType equality during union operations.

This was previously causing `"AnalysisException: u"unresolved operator 'Union;""` when trying to unionAll two dataframes with UDT columns as below.

```
from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT
from pyspark.sql import types

schema = types.StructType([types.StructField("point", PythonOnlyUDT(), True)])

a = sqlCtx.createDataFrame([[PythonOnlyPoint(1.0, 2.0)]], schema)
b = sqlCtx.createDataFrame([[PythonOnlyPoint(3.0, 4.0)]], schema)

c = a.unionAll(b)
```

## How was the this patch tested?

Tested using two unit tests in sql/test.py and the DataFrameSuite.

Additional information here : https://issues.apache.org/jira/browse/SPARK-13410

Author: Franklyn D'souza <franklynd@gmail.com>

Closes #11279 from damnMeddlingKid/udt-union-all.
2016-02-21 16:58:17 -08:00
Cheng Lian d9efe63ecd [SPARK-12799] Simplify various string output for expressions
This PR introduces several major changes:

1. Replacing `Expression.prettyString` with `Expression.sql`

   The `prettyString` method is mostly an internal, developer faced facility for debugging purposes, and shouldn't be exposed to users.

1. Using SQL-like representation as column names for selected fields that are not named expression (back-ticks and double quotes should be removed)

   Before, we were using `prettyString` as column names when possible, and sometimes the result column names can be weird.  Here are several examples:

   Expression         | `prettyString` | `sql`      | Note
   ------------------ | -------------- | ---------- | ---------------
   `a && b`           | `a && b`       | `a AND b`  |
   `a.getField("f")`  | `a[f]`         | `a.f`      | `a` is a struct

1. Adding trait `NonSQLExpression` extending from `Expression` for expressions that don't have a SQL representation (e.g. Scala UDF/UDAF and Java/Scala object expressions used for encoders)

   `NonSQLExpression.sql` may return an arbitrary user facing string representation of the expression.

Author: Cheng Lian <lian@databricks.com>

Closes #10757 from liancheng/spark-12799.simplify-expression-string-methods.
2016-02-21 22:53:15 +08:00
Holden Karau 9ca79c1ece [SPARK-13302][PYSPARK][TESTS] Move the temp file creation and cleanup outside of the doctests
Some of the new doctests in ml/clustering.py have a lot of setup code, move the setup code to the general test init to keep the doctest more example-style looking.
In part this is a follow up to https://github.com/apache/spark/pull/10999
Note that the same pattern is followed in regression & recommendation - might as well clean up all three at the same time.

Author: Holden Karau <holden@us.ibm.com>

Closes #11197 from holdenk/SPARK-13302-cleanup-doctests-in-ml-clustering.
2016-02-20 09:07:19 +00:00
Reynold Xin 6624a588c1 Revert "[SPARK-12567] [SQL] Add aes_{encrypt,decrypt} UDFs"
This reverts commit 4f9a664818.
2016-02-19 22:44:20 -08:00
Kai Jiang 4f9a664818 [SPARK-12567] [SQL] Add aes_{encrypt,decrypt} UDFs
Author: Kai Jiang <jiangkai@gmail.com>

Closes #10527 from vectorijk/spark-12567.
2016-02-19 22:28:47 -08:00
Sean Owen fb7e21797e [SPARK-13339][DOCS] Clarify commutative / associative operator requirements for reduce, fold
Clarify that reduce functions need to be commutative, and fold functions do not

See https://github.com/apache/spark/pull/11091

Author: Sean Owen <sowen@cloudera.com>

Closes #11217 from srowen/SPARK-13339.
2016-02-19 10:26:38 +00:00
Miles Yucht 827ed1c067 Correct SparseVector.parse documentation
There's a small typo in the SparseVector.parse docstring (which says that it returns a DenseVector rather than a SparseVector), which seems to be incorrect.

Author: Miles Yucht <miles@databricks.com>

Closes #11213 from mgyucht/fix-sparsevector-docs.
2016-02-16 13:01:21 +00:00
Reynold Xin 354d4c24be [SPARK-13296][SQL] Move UserDefinedFunction into sql.expressions.
This pull request has the following changes:

1. Moved UserDefinedFunction into expressions package. This is more consistent with how we structure the packages for window functions and UDAFs.

2. Moved UserDefinedPythonFunction into execution.python package, so we don't have a random private class in the top level sql package.

3. Move everything in execution/python.scala into the newly created execution.python package.

Most of the diffs are just straight copy-paste.

Author: Reynold Xin <rxin@databricks.com>

Closes #11181 from rxin/SPARK-13296.
2016-02-13 21:06:31 -08:00
Liang-Chi Hsieh e3441e3f68 [SPARK-12363][MLLIB] Remove setRun and fix PowerIterationClustering failed test
JIRA: https://issues.apache.org/jira/browse/SPARK-12363

This issue is pointed by yanboliang. When `setRuns` is removed from PowerIterationClustering, one of the tests will be failed. I found that some `dstAttr`s of the normalized graph are not correct values but 0.0. By setting `TripletFields.All` in `mapTriplets` it can work.

Author: Liang-Chi Hsieh <viirya@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #10539 from viirya/fix-poweriter.
2016-02-13 15:56:20 -08:00
vijaykiran 42d656814f [SPARK-12630][PYSPARK] [DOC] PySpark classification parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent. This is for the classification module.

Author: vijaykiran <mail@vijaykiran.com>
Author: Bryan Cutler <cutlerb@gmail.com>

Closes #11183 from BryanCutler/pyspark-consistent-param-classification-SPARK-12630.
2016-02-12 14:24:24 -08:00
Yanbo Liang 90de6b2fae [SPARK-12962] [SQL] [PySpark] PySpark support covar_samp and covar_pop
PySpark support ```covar_samp``` and ```covar_pop```.

cc rxin davies marmbrus

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10876 from yanboliang/spark-12962.
2016-02-12 12:43:13 -08:00
Holden Karau 64515e5fbf [SPARK-13154][PYTHON] Add linting for pydocs
We should have lint rules using sphinx to automatically catch the pydoc issues that are sometimes introduced.

Right now ./dev/lint-python will skip building the docs if sphinx isn't present - but it might make sense to fail hard - just a matter of if we want to insist all PySpark developers have sphinx present.

Author: Holden Karau <holden@us.ibm.com>

Closes #11109 from holdenk/SPARK-13154-add-pydoc-lint-for-docs.
2016-02-12 02:13:06 -08:00
Yanbo Liang a183dda6ab [SPARK-12974][ML][PYSPARK] Add Python API for spark.ml bisecting k-means
Add Python API for spark.ml bisecting k-means.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10889 from yanboliang/spark-12974.
2016-02-12 01:45:45 -08:00
Tommy YU d3e2e20299 [SPARK-13153][PYSPARK] ML persistence failed when handle no default value parameter
Fix this defect by check default value exist or not.

yanboliang Please help to review.

Author: Tommy YU <tummyyu@163.com>

Closes #11043 from Wenpei/spark-13153-handle-param-withnodefaultvalue.
2016-02-11 18:38:49 -08:00
sethah b354673886 [SPARK-13047][PYSPARK][ML] Pyspark Params.hasParam should not throw an error
Pyspark Params class has a method `hasParam(paramName)` which returns `True` if the class has a parameter by that name, but throws an `AttributeError` otherwise. There is not currently a way of getting a Boolean to indicate if a class has a parameter. With Spark 2.0 we could modify the existing behavior of `hasParam` or add an additional method with this functionality.

In Python:
```python
from pyspark.ml.classification import NaiveBayes
nb = NaiveBayes()
print nb.hasParam("smoothing")
print nb.hasParam("notAParam")
```
produces:
> True
> AttributeError: 'NaiveBayes' object has no attribute 'notAParam'

However, in Scala:
```scala
import org.apache.spark.ml.classification.NaiveBayes
val nb  = new NaiveBayes()
nb.hasParam("smoothing")
nb.hasParam("notAParam")
```
produces:
> true
> false

cc holdenk

Author: sethah <seth.hendrickson16@gmail.com>

Closes #10962 from sethah/SPARK-13047.
2016-02-11 16:42:44 -08:00
Yanbo Liang 30e0095566 [SPARK-13035][ML][PYSPARK] PySpark ml.clustering support export/import
PySpark ml.clustering support export/import.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10999 from yanboliang/spark-13035.
2016-02-11 15:55:40 -08:00
Yanbo Liang 2426eb3e16 [MINOR][ML][PYSPARK] Cleanup test cases of clustering.py
Test cases should be removed from annotation of ```setXXX``` function, otherwise it will be parts of [Python API docs](https://spark.apache.org/docs/latest/api/python/pyspark.ml.html#pyspark.ml.clustering.KMeans.setInitMode).
cc mengxr jkbradley

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10975 from yanboliang/clustering-cleanup.
2016-02-11 15:53:45 -08:00
Kai Jiang c8f667d7c1 [SPARK-13037][ML][PYSPARK] PySpark ml.recommendation support export/import
PySpark ml.recommendation support export/import.

Author: Kai Jiang <jiangkai@gmail.com>

Closes #11044 from vectorijk/spark-13037.
2016-02-11 15:50:33 -08:00
Davies Liu b5761d150b [SPARK-12706] [SQL] grouping() and grouping_id()
Grouping() returns a column is aggregated or not, grouping_id() returns the aggregation levels.

grouping()/grouping_id() could be used with window function, but does not work in having/sort clause, will be fixed by another PR.

The GROUPING__ID/grouping_id() in Hive is wrong (according to docs), we also did it wrongly, this PR change that to match the behavior in most databases (also the docs of Hive).

Author: Davies Liu <davies@databricks.com>

Closes #10677 from davies/grouping.
2016-02-10 20:13:38 -08:00
Nam Pham edf4a0e62e [SPARK-12986][DOC] Fix pydoc warnings in mllib/regression.py
I have fixed the warnings by running "make html" under "python/docs/". They are caused by not having blank lines around indented paragraphs.

Author: Nam Pham <phamducnam@gmail.com>

Closes #11025 from nampham2/SPARK-12986.
2016-02-08 11:06:41 -08:00
Tommy YU 81da3bee66 [SPARK-5865][API DOC] Add doc warnings for methods that return local data structures
rxin srowen
I work out note message for rdd.take function, please help to review.

If it's fine, I can apply to all other function later.

Author: Tommy YU <tummyyu@163.com>

Closes #10874 from Wenpei/spark-5865-add-warning-for-localdatastructure.
2016-02-06 17:29:09 +00:00
Shixiong Zhu 335f10edad [SPARK-7997][CORE] Add rpcEnv.awaitTermination() back to SparkEnv
`rpcEnv.awaitTermination()` was not added in #10854 because some Streaming Python tests hung forever.

This patch fixed the hung issue and added rpcEnv.awaitTermination() back to SparkEnv.

Previously, Streaming Kafka Python tests shutdowns the zookeeper server before stopping StreamingContext. Then when stopping StreamingContext, KafkaReceiver may be hung due to https://issues.apache.org/jira/browse/KAFKA-601, hence, some thread of RpcEnv's Dispatcher cannot exit and rpcEnv.awaitTermination is hung.The patch just changed the shutdown order to fix it.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11031 from zsxwing/awaitTermination.
2016-02-02 21:13:54 -08:00
Bryan Cutler cba1d6b659 [SPARK-12631][PYSPARK][DOC] PySpark clustering parameter desc to consistent format
Part of task for [SPARK-11219](https://issues.apache.org/jira/browse/SPARK-11219) to make PySpark MLlib parameter description formatting consistent.  This is for the clustering module.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #10610 from BryanCutler/param-desc-consistent-cluster-SPARK-12631.
2016-02-02 10:50:22 -08:00
Herman van Hovell 5a8b978fab [SPARK-13049] Add First/last with ignore nulls to functions.scala
This PR adds the ability to specify the ```ignoreNulls``` option to the functions dsl, e.g:
```df.select($"id", last($"value", ignoreNulls = true).over(Window.partitionBy($"id").orderBy($"other"))```

This PR is some where between a bug fix (see the JIRA) and a new feature. I am not sure if we should backport to 1.6.

cc yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10957 from hvanhovell/SPARK-13049.
2016-01-31 13:56:13 -08:00
Yanbo Liang e51b6eaa9e [SPARK-13032][ML][PYSPARK] PySpark support model export/import and take LinearRegression as example
* Implement ```MLWriter/MLWritable/MLReader/MLReadable``` for PySpark.
* Making ```LinearRegression``` to support ```save/load``` as example. After this merged, the work for other transformers/estimators will be easy, then we can list and distribute the tasks to the community.

cc mengxr jkbradley

Author: Yanbo Liang <ybliang8@gmail.com>
Author: Joseph K. Bradley <joseph@databricks.com>

Closes #10469 from yanboliang/spark-11939.
2016-01-29 09:22:24 -08:00
Brandon Bradley 3a40c0e575 [SPARK-12749][SQL] add json option to parse floating-point types as DecimalType
I tried to add this via `USE_BIG_DECIMAL_FOR_FLOATS` option from Jackson with no success.

Added test for non-complex types. Should I add a test for complex types?

Author: Brandon Bradley <bradleytastic@gmail.com>

Closes #10936 from blbradley/spark-12749.
2016-01-28 15:25:57 -08:00
Jason Lee edd473751b [SPARK-10847][SQL][PYSPARK] Pyspark - DataFrame - Optional Metadata with None triggers cryptic failure
The error message is now changed from "Do not support type class scala.Tuple2." to "Do not support type class org.json4s.JsonAST$JNull$" to be more informative about what is not supported. Also, StructType metadata now handles JNull correctly, i.e., {'a': None}. test_metadata_null is added to tests.py to show the fix works.

Author: Jason Lee <cjlee@us.ibm.com>

Closes #8969 from jasoncl/SPARK-10847.
2016-01-27 09:55:10 -08:00
Xusen Yin 4db255c7aa [SPARK-12780] Inconsistency returning value of ML python models' properties
https://issues.apache.org/jira/browse/SPARK-12780

Author: Xusen Yin <yinxusen@gmail.com>

Closes #10724 from yinxusen/SPARK-12780.
2016-01-26 21:16:56 -08:00
Holden Karau eb917291ca [SPARK-10509][PYSPARK] Reduce excessive param boiler plate code
The current python ml params require cut-and-pasting the param setup and description between the class & ```__init__``` methods. Remove this possible case of errors & simplify use of custom params by adding a ```_copy_new_parent``` method to param so as to avoid cut and pasting (and cut and pasting at different indentation levels urgh).

Author: Holden Karau <holden@us.ibm.com>

Closes #10216 from holdenk/SPARK-10509-excessive-param-boiler-plate-code.
2016-01-26 15:53:48 -08:00
Jeff Zhang 19fdb21afb [SPARK-12993][PYSPARK] Remove usage of ADD_FILES in pyspark
environment variable ADD_FILES is created for adding python files on spark context to be distributed to executors (SPARK-865), this is deprecated now. User are encouraged to use --py-files for adding python files.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10913 from zjffdu/SPARK-12993.
2016-01-26 14:58:39 -08:00
Xusen Yin 8beab68152 [SPARK-11923][ML] Python API for ml.feature.ChiSqSelector
https://issues.apache.org/jira/browse/SPARK-11923

Author: Xusen Yin <yinxusen@gmail.com>

Closes #10186 from yinxusen/SPARK-11923.
2016-01-26 11:56:46 -08:00
Xiangrui Meng 27c910f7f2 [SPARK-10086][MLLIB][STREAMING][PYSPARK] ignore StreamingKMeans test in PySpark for now
I saw several failures from recent PR builds, e.g., https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/50015/consoleFull. This PR marks the test as ignored and we will fix the flakyness in SPARK-10086.

gliptak Do you know why the test failure didn't show up in the Jenkins "Test Result"?

cc: jkbradley

Author: Xiangrui Meng <meng@databricks.com>

Closes #10909 from mengxr/SPARK-10086.
2016-01-25 22:53:34 -08:00
Holden Karau b66afdeb52 [SPARK-11922][PYSPARK][ML] Python api for ml.feature.quantile discretizer
Add Python API for ml.feature.QuantileDiscretizer.

One open question: Do we want to do this stuff to re-use the java model, create a new model, or use a different wrapper around the java model.
cc brkyvz & mengxr

Author: Holden Karau <holden@us.ibm.com>

Closes #10085 from holdenk/SPARK-11937-SPARK-11922-Python-API-for-ml.feature.QuantileDiscretizer.
2016-01-25 22:38:31 -08:00
Yanbo Liang dcae355c64 [SPARK-12905][ML][PYSPARK] PCAModel return eigenvalues for PySpark
```PCAModel```  can output ```explainedVariance``` at Python side.

cc mengxr srowen

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10830 from yanboliang/spark-12905.
2016-01-25 13:54:21 -08:00
Cheng Lian 3327fd2817 [SPARK-12624][PYSPARK] Checks row length when converting Java arrays to Python rows
When actual row length doesn't conform to specified schema field length, we should give a better error message instead of throwing an unintuitive `ArrayOutOfBoundsException`.

Author: Cheng Lian <lian@databricks.com>

Closes #10886 from liancheng/spark-12624.
2016-01-24 19:40:34 -08:00
Jeff Zhang e789b1d2c1 [SPARK-12120][PYSPARK] Improve exception message when failing to init…
…ialize HiveContext in PySpark

davies Mind to review ?

This is the error message after this PR

```
15/12/03 16:59:53 WARN ObjectStore: Failed to get database default, returning NoSuchObjectException
/Users/jzhang/github/spark/python/pyspark/sql/context.py:689: UserWarning: You must build Spark with Hive. Export 'SPARK_HIVE=true' and run build/sbt assembly
  warnings.warn("You must build Spark with Hive. "
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/Users/jzhang/github/spark/python/pyspark/sql/context.py", line 663, in read
    return DataFrameReader(self)
  File "/Users/jzhang/github/spark/python/pyspark/sql/readwriter.py", line 56, in __init__
    self._jreader = sqlContext._ssql_ctx.read()
  File "/Users/jzhang/github/spark/python/pyspark/sql/context.py", line 692, in _ssql_ctx
    raise e
py4j.protocol.Py4JJavaError: An error occurred while calling None.org.apache.spark.sql.hive.HiveContext.
: java.lang.RuntimeException: java.net.ConnectException: Call From jzhangMBPr.local/127.0.0.1 to 0.0.0.0:9000 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
	at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:522)
	at org.apache.spark.sql.hive.client.ClientWrapper.<init>(ClientWrapper.scala:194)
	at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:238)
	at org.apache.spark.sql.hive.HiveContext.executionHive$lzycompute(HiveContext.scala:218)
	at org.apache.spark.sql.hive.HiveContext.executionHive(HiveContext.scala:208)
	at org.apache.spark.sql.hive.HiveContext.functionRegistry$lzycompute(HiveContext.scala:462)
	at org.apache.spark.sql.hive.HiveContext.functionRegistry(HiveContext.scala:461)
	at org.apache.spark.sql.UDFRegistration.<init>(UDFRegistration.scala:40)
	at org.apache.spark.sql.SQLContext.<init>(SQLContext.scala:330)
	at org.apache.spark.sql.hive.HiveContext.<init>(HiveContext.scala:90)
	at org.apache.spark.sql.hive.HiveContext.<init>(HiveContext.scala:101)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:234)
	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:381)
	at py4j.Gateway.invoke(Gateway.java:214)
	at py4j.commands.ConstructorCommand.invokeConstructor(ConstructorCommand.java:79)
	at py4j.commands.ConstructorCommand.execute(ConstructorCommand.java:68)
	at py4j.GatewayConnection.run(GatewayConnection.java:209)
	at java.lang.Thread.run(Thread.java:745)
```

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10126 from zjffdu/SPARK-12120.
2016-01-24 12:29:26 -08:00
Gábor Lipták 9bb35c5b59 [SPARK-11295][PYSPARK] Add packages to JUnit output for Python tests
This is #9263 from gliptak (improving grouping/display of test case results) with a small fix of bisecting k-means unit test.

Author: Gábor Lipták <gliptak@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #10850 from mengxr/SPARK-11295.
2016-01-20 11:11:10 -08:00
Xiangrui Meng beda901422 Revert "[SPARK-11295] Add packages to JUnit output for Python tests"
This reverts commit c6f971b4ae.
2016-01-19 16:51:17 -08:00
BenFradet f6f7ca9d2e [SPARK-9716][ML] BinaryClassificationEvaluator should accept Double prediction column
This PR aims to allow the prediction column of `BinaryClassificationEvaluator` to be of double type.

Author: BenFradet <benjamin.fradet@gmail.com>

Closes #10472 from BenFradet/SPARK-9716.
2016-01-19 14:59:20 -08:00
Gábor Lipták c6f971b4ae [SPARK-11295] Add packages to JUnit output for Python tests
SPARK-11295 Add packages to JUnit output for Python tests

This improves grouping/display of test case results.

Author: Gábor Lipták <gliptak@gmail.com>

Closes #9263 from gliptak/SPARK-11295.
2016-01-19 14:06:53 -08:00
Holden Karau 0ddba6d88f [SPARK-11944][PYSPARK][MLLIB] python mllib.clustering.bisecting k means
From the coverage issues for 1.6 : Add Python API for mllib.clustering.BisectingKMeans.

Author: Holden Karau <holden@us.ibm.com>

Closes #10150 from holdenk/SPARK-11937-python-api-coverage-SPARK-11944-python-mllib.clustering.BisectingKMeans.
2016-01-19 10:15:54 -08:00
Sean Owen d8c4b00a23 [SPARK-7683][PYSPARK] Confusing behavior of fold function of RDD in pyspark
Fix order of arguments that Pyspark RDD.fold passes to its op -  should be (acc, obj) like other implementations.

Obviously, this is a potentially breaking change, so can only happen for 2.x

CC davies

Author: Sean Owen <sowen@cloudera.com>

Closes #10771 from srowen/SPARK-7683.
2016-01-19 09:34:49 +00:00
Yanbo Liang 5f843781e3 [SPARK-11925][ML][PYSPARK] Add PySpark missing methods for ml.feature during Spark 1.6 QA
Add PySpark missing methods and params for ml.feature:
* ```RegexTokenizer``` should support setting ```toLowercase```.
* ```MinMaxScalerModel``` should support output ```originalMin``` and ```originalMax```.
* ```PCAModel``` should support output ```pc```.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #9908 from yanboliang/spark-11925.
2016-01-15 15:54:19 -08:00
Herman van Hovell 7cd7f22025 [SPARK-12575][SQL] Grammar parity with existing SQL parser
In this PR the new CatalystQl parser stack reaches grammar parity with the old Parser-Combinator based SQL Parser. This PR also replaces all uses of the old Parser, and removes it from the code base.

Although the existing Hive and SQL parser dialects were mostly the same, some kinks had to be worked out:
- The SQL Parser allowed syntax like ```APPROXIMATE(0.01) COUNT(DISTINCT a)```. In order to make this work we needed to hardcode approximate operators in the parser, or we would have to create an approximate expression. ```APPROXIMATE_COUNT_DISTINCT(a, 0.01)``` would also do the job and is much easier to maintain. So, this PR **removes** this keyword.
- The old SQL Parser supports ```LIMIT``` clauses in nested queries. This is **not supported** anymore. See https://github.com/apache/spark/pull/10689 for the rationale for this.
- Hive has a charset name char set literal combination it supports, for instance the following expression ```_ISO-8859-1 0x4341464562616265``` would yield this string: ```CAFEbabe```. Hive will only allow charset names to start with an underscore. This is quite annoying in spark because as soon as you use a tuple names will start with an underscore. In this PR we **remove** this feature from the parser. It would be quite easy to implement such a feature as an Expression later on.
- Hive and the SQL Parser treat decimal literals differently. Hive will turn any decimal into a ```Double``` whereas the SQL Parser would convert a non-scientific decimal into a ```BigDecimal```, and would turn a scientific decimal into a Double. We follow Hive's behavior here. The new parser supports a big decimal literal, for instance: ```81923801.42BD```, which can be used when a big decimal is needed.

cc rxin viirya marmbrus yhuai cloud-fan

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10745 from hvanhovell/SPARK-12575-2.
2016-01-15 15:19:10 -08:00
Wenchen Fan 962e9bcf94 [SPARK-12756][SQL] use hash expression in Exchange
This PR makes bucketing and exchange share one common hash algorithm, so that we can guarantee the data distribution is same between shuffle and bucketed data source, which enables us to only shuffle one side when join a bucketed table and a normal one.

This PR also fixes the tests that are broken by the new hash behaviour in shuffle.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10703 from cloud-fan/use-hash-expr-in-shuffle.
2016-01-13 22:43:28 -08:00
Reynold Xin cbbcd8e425 [SPARK-12791][SQL] Simplify CaseWhen by breaking "branches" into "conditions" and "values"
This pull request rewrites CaseWhen expression to break the single, monolithic "branches" field into a sequence of tuples (Seq[(condition, value)]) and an explicit optional elseValue field.

Prior to this pull request, each even position in "branches" represents the condition for each branch, and each odd position represents the value for each branch. The use of them have been pretty confusing with a lot sliding windows or grouped(2) calls.

Author: Reynold Xin <rxin@databricks.com>

Closes #10734 from rxin/simplify-case.
2016-01-13 12:44:35 -08:00
Wenchen Fan c2ea79f96a [SPARK-12642][SQL] improve the hash expression to be decoupled from unsafe row
https://issues.apache.org/jira/browse/SPARK-12642

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10694 from cloud-fan/hash-expr.
2016-01-13 12:29:02 -08:00
Erik Selin e4e0b3f7b2 [SPARK-12268][PYSPARK] Make pyspark shell pythonstartup work under python3
This replaces the `execfile` used for running custom python shell scripts
with explicit open, compile and exec (as recommended by 2to3). The reason
for this change is to make the pythonstartup option compatible with python3.

Author: Erik Selin <erik.selin@gmail.com>

Closes #10255 from tyro89/pythonstartup-python3.
2016-01-13 12:21:45 -08:00
Shixiong Zhu 4f60651cbe [SPARK-12652][PYSPARK] Upgrade Py4J to 0.9.1
- [x] Upgrade Py4J to 0.9.1
- [x] SPARK-12657: Revert SPARK-12617
- [x] SPARK-12658: Revert SPARK-12511
  - Still keep the change that only reading checkpoint once. This is a manual change and worth to take a look carefully. bfd4b5c040
- [x] Verify no leak any more after reverting our workarounds

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10692 from zsxwing/py4j-0.9.1.
2016-01-12 14:27:05 -08:00
Yanbo Liang ee4ee02b86 [SPARK-12603][MLLIB] PySpark MLlib GaussianMixtureModel should support single instance predict/predictSoft
PySpark MLlib ```GaussianMixtureModel``` should support single instance ```predict/predictSoft``` just like Scala do.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10552 from yanboliang/spark-12603.
2016-01-11 14:43:25 -08:00
Sean Owen b9c8353378 [SPARK-12618][CORE][STREAMING][SQL] Clean up build warnings: 2.0.0 edition
Fix most build warnings: mostly deprecated API usages. I'll annotate some of the changes below. CC rxin who is leading the charge to remove the deprecated APIs.

Author: Sean Owen <sowen@cloudera.com>

Closes #10570 from srowen/SPARK-12618.
2016-01-08 17:47:44 +00:00
zero323 592f64985d [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None
If initial model passed to GMM is not empty it causes net.razorvine.pickle.PickleException. It can be fixed by converting initialModel.weights to list.

Author: zero323 <matthew.szymkiewicz@gmail.com>

Closes #10644 from zero323/SPARK-12006.
2016-01-07 10:32:56 -08:00
Yin Huai e5cde7ab11 Revert "[SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None"
This reverts commit fcd013cf70.

Author: Yin Huai <yhuai@databricks.com>

Closes #10632 from yhuai/pythonStyle.
2016-01-06 22:03:31 -08:00
Shixiong Zhu 1e6648d62f [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to Streaming
Move Py4jCallbackConnectionCleaner to Streaming because the callback server starts only in StreamingContext.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10621 from zsxwing/SPARK-12617-2.
2016-01-06 12:03:01 -08:00
zero323 fcd013cf70 [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None
If initial model passed to GMM is not empty it causes `net.razorvine.pickle.PickleException`. It can be fixed by converting `initialModel.weights` to `list`.

Author: zero323 <matthew.szymkiewicz@gmail.com>

Closes #9986 from zero323/SPARK-12006.
2016-01-06 11:58:33 -08:00
Yanbo Liang 3aa3488225 [SPARK-11815][ML][PYSPARK] PySpark DecisionTreeClassifier & DecisionTreeRegressor should support setSeed
PySpark ```DecisionTreeClassifier``` & ```DecisionTreeRegressor``` should support ```setSeed``` like what we do at Scala side.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #9807 from yanboliang/spark-11815.
2016-01-06 10:52:25 -08:00
Yanbo Liang 95eb651633 [SPARK-11945][ML][PYSPARK] Add computeCost to KMeansModel for PySpark spark.ml
Add ```computeCost``` to ```KMeansModel``` as evaluator for PySpark spark.ml.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #9931 from yanboliang/SPARK-11945.
2016-01-06 10:50:02 -08:00
Joshi 007da1a9dc [SPARK-11531][ML] SparseVector error Msg
PySpark SparseVector should have "Found duplicate indices" error message

Author: Joshi <rekhajoshm@gmail.com>
Author: Rekha Joshi <rekhajoshm@gmail.com>

Closes #9525 from rekhajoshm/SPARK-11531.
2016-01-06 10:48:14 -08:00
Holden Karau 3b29004d24 [SPARK-7675][ML][PYSPARK] sparkml params type conversion
From JIRA:
Currently, PySpark wrappers for spark.ml Scala classes are brittle when accepting Param types. E.g., Normalizer's "p" param cannot be set to "2" (an integer); it must be set to "2.0" (a float). Fixing this is not trivial since there does not appear to be a natural place to insert the conversion before Python wrappers call Java's Params setter method.

A possible fix will be to include a method "_checkType" to PySpark's Param class which checks the type, prints an error if needed, and converts types when relevant (e.g., int to float, or scipy matrix to array). The Java wrapper method which copies params to Scala can call this method when available.

This fix instead checks the types at set time since I think failing sooner is better, but I can switch it around to check at copy time if that would be better. So far this only converts int to float and other conversions (like scipymatrix to array) are left for the future.

Author: Holden Karau <holden@us.ibm.com>

Closes #9581 from holdenk/SPARK-7675-PySpark-sparkml-Params-type-conversion.
2016-01-06 10:43:03 -08:00
Kai Jiang 1537e55604 [SPARK-12041][ML][PYSPARK] Add columnSimilarities to IndexedRowMatrix
Add `columnSimilarities` to IndexedRowMatrix for PySpark spark.mllib.linalg.

Author: Kai Jiang <jiangkai@gmail.com>

Closes #10158 from vectorijk/spark-12041.
2016-01-05 15:33:27 -08:00
Shixiong Zhu 6cfe341ee8 [SPARK-12511] [PYSPARK] [STREAMING] Make sure PythonDStream.registerSerializer is called only once
There is an issue that Py4J's PythonProxyHandler.finalize blocks forever. (https://github.com/bartdag/py4j/pull/184)

Py4j will create a PythonProxyHandler in Java for "transformer_serializer" when calling "registerSerializer". If we call "registerSerializer" twice, the second PythonProxyHandler will override the first one, then the first one will be GCed and trigger "PythonProxyHandler.finalize". To avoid that, we should not call"registerSerializer" more than once, so that "PythonProxyHandler" in Java side won't be GCed.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10514 from zsxwing/SPARK-12511.
2016-01-05 13:48:47 -08:00
Shixiong Zhu 047a31bb10 [SPARK-12617] [PYSPARK] Clean up the leak sockets of Py4J
This patch added Py4jCallbackConnectionCleaner to clean the leak sockets of Py4J every 30 seconds. This is a workaround before Py4J fixes the leak issue https://github.com/bartdag/py4j/issues/187

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10579 from zsxwing/SPARK-12617.
2016-01-05 13:10:46 -08:00
Wenchen Fan 76768337be [SPARK-12480][FOLLOW-UP] use a single column vararg for hash
address comments in #10435

This makes the API easier to use if user programmatically generate the call to hash, and they will get analysis exception if the arguments of hash is empty.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10588 from cloud-fan/hash.
2016-01-05 10:23:36 -08:00
Reynold Xin 77ab49b857 [SPARK-12600][SQL] Remove deprecated methods in Spark SQL
Author: Reynold Xin <rxin@databricks.com>

Closes #10559 from rxin/remove-deprecated-sql.
2016-01-04 18:02:38 -08:00
Holden Karau 13dab9c386 [SPARK-12611][SQL][PYSPARK][TESTS] Fix test_infer_schema_to_local
Previously (when the PR was first created) not specifying b= explicitly was fine (and treated as default null) - instead be explicit about b being None in the test.

Author: Holden Karau <holden@us.ibm.com>

Closes #10564 from holdenk/SPARK-12611-fix-test-infer-schema-local.
2016-01-03 17:04:35 -08:00
Cazen b8410ff9ce [SPARK-12537][SQL] Add option to accept quoting of all character backslash quoting mechanism
We can provides the option to choose JSON parser can be enabled to accept quoting of all character or not.

Author: Cazen <Cazen@korea.com>
Author: Cazen Lee <cazen.lee@samsung.com>
Author: Cazen Lee <Cazen@korea.com>
Author: cazen.lee <cazen.lee@samsung.com>

Closes #10497 from Cazen/master.
2016-01-03 17:01:19 -08:00
Holden Karau d1ca634db4 [SPARK-12300] [SQL] [PYSPARK] fix schema inferance on local collections
Current schema inference for local python collections halts as soon as there are no NullTypes. This is different than when we specify a sampling ratio of 1.0 on a distributed collection. This could result in incomplete schema information.

Author: Holden Karau <holden@us.ibm.com>

Closes #10275 from holdenk/SPARK-12300-fix-schmea-inferance-on-local-collections.
2015-12-30 11:14:47 -08:00
jerryshao 8d49400921 [SPARK-12353][STREAMING][PYSPARK] Fix countByValue inconsistent output in Python API
The semantics of Python countByValue is different from Scala API, it is more like countDistinctValue, so here change to make it consistent with Scala/Java API.

Author: jerryshao <sshao@hortonworks.com>

Closes #10350 from jerryshao/SPARK-12353.
2015-12-28 10:43:23 +00:00
gatorsmile 9ab296ecdc [SPARK-12520] [PYSPARK] Correct Descriptions and Add Use Cases in Equi-Join
After reading the JIRA https://issues.apache.org/jira/browse/SPARK-12520, I double checked the code.

For example, users can do the Equi-Join like
  ```df.join(df2, 'name', 'outer').select('name', 'height').collect()```
- There exists a bug in 1.5 and 1.4. The code just ignores the third parameter (join type) users pass. However, the join type we called is `Inner`, even if the user-specified type is the other type (e.g., `Outer`).
- After a PR: https://github.com/apache/spark/pull/8600, the 1.6 does not have such an issue, but the description has not been updated.

Plan to submit another PR to fix 1.5 and issue an error message if users specify a non-inner join type when using Equi-Join.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10477 from gatorsmile/pyOuterJoin.
2015-12-27 23:18:48 -08:00
Holden Karau 969d5665bb [SPARK-12296][PYSPARK][MLLIB] Feature parity for pyspark mllib standard scaler model
Some methods are missing, such as ways to access the std, mean, etc. This PR is for feature parity for pyspark.mllib.feature.StandardScaler & StandardScalerModel.

Author: Holden Karau <holden@us.ibm.com>

Closes #10298 from holdenk/SPARK-12296-feature-parity-pyspark-mllib-StandardScalerModel.
2015-12-22 09:14:12 +02:00
pshearer fc6dbcc703 Doc typo: ltrim = trim from left end, not right
Author: pshearer <pshearer@massmutual.com>

Closes #10414 from pshearer/patch-1.
2015-12-21 14:04:59 -08:00
Jeff Zhang 1920d72a1f [PYSPARK] Pyspark typo & Add missing abstractmethod annotation
No jira is created since this is a trivial change.

davies  Please help review it

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10143 from zjffdu/pyspark_typo.
2015-12-21 08:53:46 -08:00
Bryan Cutler ce1798b3af [SPARK-10158][PYSPARK][MLLIB] ALS better error message when using Long IDs
Added catch for casting Long to Int exception when PySpark ALS Ratings are serialized.  It is easy to accidentally use Long IDs for user/product and before, it would fail with a somewhat cryptic "ClassCastException: java.lang.Long cannot be cast to java.lang.Integer."  Now if this is done, a more descriptive error is shown, e.g. "PickleException: Ratings id 1205640308657491975 exceeds max integer value of 2147483647."

Author: Bryan Cutler <bjcutler@us.ibm.com>

Closes #9361 from BryanCutler/als-pyspark-long-id-error-SPARK-10158.
2015-12-20 09:08:23 +00:00
Yanbo Liang a073a73a56 [SQL] Fix mistake doc of join type for dataframe.join
Fix mistake doc of join type for ```dataframe.join```.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10378 from yanboliang/leftsemi.
2015-12-19 00:34:30 -08:00
gatorsmile 499ac3e69a [SPARK-12091] [PYSPARK] Deprecate the JAVA-specific deserialized storage levels
The current default storage level of Python persist API is MEMORY_ONLY_SER. This is different from the default level MEMORY_ONLY in the official document and RDD APIs.

davies Is this inconsistency intentional? Thanks!

Updates: Since the data is always serialized on the Python side, the storage levels of JAVA-specific deserialization are not removed, such as MEMORY_ONLY.

Updates: Based on the reviewers' feedback. In Python, stored objects will always be serialized with the [Pickle](https://docs.python.org/2/library/pickle.html) library, so it does not matter whether you choose a serialized level. The available storage levels in Python include `MEMORY_ONLY`, `MEMORY_ONLY_2`, `MEMORY_AND_DISK`, `MEMORY_AND_DISK_2`, `DISK_ONLY`, `DISK_ONLY_2` and `OFF_HEAP`.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10092 from gatorsmile/persistStorageLevel.
2015-12-18 20:06:05 -08:00
Yanbo Liang 6e0771665b [SQL] Update SQLContext.read.text doc
Since we rename the column name from ```text``` to ```value``` for DataFrame load by ```SQLContext.read.text```, we need to update doc.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10349 from yanboliang/text-value.
2015-12-17 09:19:46 -08:00
David Tolpin 437583f692 [SPARK-11904][PYSPARK] reduceByKeyAndWindow does not require checkpointing when invFunc is None
when invFunc is None, `reduceByKeyAndWindow(func, None, winsize, slidesize)` is equivalent to

     reduceByKey(func).window(winsize, slidesize).reduceByKey(winsize, slidesize)

and no checkpoint is necessary. The corresponding Scala code does exactly that, but Python code always creates a windowed stream with obligatory checkpointing. The patch fixes this.

I do not know how to unit-test this.

Author: David Tolpin <david.tolpin@gmail.com>

Closes #9888 from dtolpin/master.
2015-12-16 22:10:24 -08:00
Davies Liu 27b98e99d2 [SPARK-12380] [PYSPARK] use SQLContext.getOrCreate in mllib
MLlib should use SQLContext.getOrCreate() instead of creating new SQLContext.

Author: Davies Liu <davies@databricks.com>

Closes #10338 from davies/create_context.
2015-12-16 15:48:11 -08:00
Martin Menestret 3a44aebd0c [SPARK-9690][ML][PYTHON] pyspark CrossValidator random seed
Extend CrossValidator with HasSeed in PySpark.

This PR replaces [https://github.com/apache/spark/pull/7997]

CC: yanboliang thunterdb mmenestret  Would one of you mind taking a look?  Thanks!

Author: Joseph K. Bradley <joseph@databricks.com>
Author: Martin MENESTRET <mmenestret@ippon.fr>

Closes #10268 from jkbradley/pyspark-cv-seed.
2015-12-16 14:05:35 -08:00
Jeff Zhang 6a880afa83 [SPARK-12361][PYSPARK][TESTS] Should set PYSPARK_DRIVER_PYTHON before Python tests
Although this patch still doesn't solve the issue why the return code is 0 (see JIRA description), it resolves the issue of python version mismatch.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10322 from zjffdu/SPARK-12361.
2015-12-16 11:29:51 -08:00
Liang-Chi Hsieh b51a4cdff3 [SPARK-12016] [MLLIB] [PYSPARK] Wrap Word2VecModel when loading it in pyspark
JIRA: https://issues.apache.org/jira/browse/SPARK-12016

We should not directly use Word2VecModel in pyspark. We need to wrap it in a Word2VecModelWrapper when loading it in pyspark.

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

Closes #10100 from viirya/fix-load-py-wordvecmodel.
2015-12-14 09:59:42 -08:00
Bryan Cutler 6a6c1fc5c8 [SPARK-11713] [PYSPARK] [STREAMING] Initial RDD updateStateByKey for PySpark
Adding ability to define an initial state RDD for use with updateStateByKey PySpark.  Added unit test and changed stateful_network_wordcount example to use initial RDD.

Author: Bryan Cutler <bjcutler@us.ibm.com>

Closes #10082 from BryanCutler/initial-rdd-updateStateByKey-SPARK-11713.
2015-12-10 14:21:15 -08:00
Cheng Lian 6e1c55eac4 [SPARK-12012][SQL] Show more comprehensive PhysicalRDD metadata when visualizing SQL query plan
This PR adds a `private[sql]` method `metadata` to `SparkPlan`, which can be used to describe detail information about a physical plan during visualization. Specifically, this PR uses this method to provide details of `PhysicalRDD`s translated from a data source relation. For example, a `ParquetRelation` converted from Hive metastore table `default.psrc` is now shown as the following screenshot:

![image](https://cloud.githubusercontent.com/assets/230655/11526657/e10cb7e6-9916-11e5-9afa-f108932ec890.png)

And here is the screenshot for a regular `ParquetRelation` (not converted from Hive metastore table) loaded from a really long path:

![output](https://cloud.githubusercontent.com/assets/230655/11680582/37c66460-9e94-11e5-8f50-842db5309d5a.png)

Author: Cheng Lian <lian@databricks.com>

Closes #10004 from liancheng/spark-12012.physical-rdd-metadata.
2015-12-09 23:30:42 +08:00
Andrew Ray 36282f78b8 [SPARK-12184][PYTHON] Make python api doc for pivot consistant with scala doc
In SPARK-11946 the API for pivot was changed a bit and got updated doc, the doc changes were not made for the python api though. This PR updates the python doc to be consistent.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #10176 from aray/sql-pivot-python-doc.
2015-12-07 15:01:00 -08:00
Davies Liu ef3f047c07 [SPARK-12132] [PYSPARK] raise KeyboardInterrupt inside SIGINT handler
Currently, the current line is not cleared by Cltr-C

After this patch
```
>>> asdfasdf^C
Traceback (most recent call last):
  File "~/spark/python/pyspark/context.py", line 225, in signal_handler
    raise KeyboardInterrupt()
KeyboardInterrupt
```

It's still worse than 1.5 (and before).

Author: Davies Liu <davies@databricks.com>

Closes #10134 from davies/fix_cltrc.
2015-12-07 11:00:25 -08:00
Burak Yavuz 302d68de87 [SPARK-12058][STREAMING][KINESIS][TESTS] fix Kinesis python tests
Python tests require access to the `KinesisTestUtils` file. When this file exists under src/test, python can't access it, since it is not available in the assembly jar.

However, if we move KinesisTestUtils to src/main, we need to add the KinesisProducerLibrary as a dependency. In order to avoid this, I moved KinesisTestUtils to src/main, and extended it with ExtendedKinesisTestUtils which is under src/test that adds support for the KPL.

cc zsxwing tdas

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #10050 from brkyvz/kinesis-py.
2015-12-04 12:08:42 -08:00
Yanbo Liang d576e76bba [MINOR][ML] Use coefficients replace weights
Use ```coefficients``` replace ```weights```, I wish they are the last two.
mengxr

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10065 from yanboliang/coefficients.
2015-12-03 11:37:34 -08:00
Davies Liu 4375eb3f48 [SPARK-12090] [PYSPARK] consider shuffle in coalesce()
Author: Davies Liu <davies@databricks.com>

Closes #10090 from davies/fix_coalesce.
2015-12-01 22:41:48 -08:00
jerryshao f292018f8e [SPARK-12002][STREAMING][PYSPARK] Fix python direct stream checkpoint recovery issue
Fixed a minor race condition in #10017

Closes #10017

Author: jerryshao <sshao@hortonworks.com>
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10074 from zsxwing/review-pr10017.
2015-12-01 15:26:10 -08:00
Shixiong Zhu edb26e7f4e [SPARK-12058][HOTFIX] Disable KinesisStreamTests
KinesisStreamTests in test.py is broken because of #9403. See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/46896/testReport/(root)/KinesisStreamTests/test_kinesis_stream/

Because Streaming Python didn’t work when merging https://github.com/apache/spark/pull/9403, the PR build didn’t report the Python test failure actually.

This PR just disabled the test to unblock #10039

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10047 from zsxwing/disable-python-kinesis-test.
2015-11-30 16:31:59 -08:00
Jeff Zhang d8220885c4 [SPARK-11917][PYSPARK] Add SQLContext#dropTempTable to PySpark
Author: Jeff Zhang <zjffdu@apache.org>

Closes #9903 from zjffdu/SPARK-11917.
2015-11-26 19:15:22 -08:00
gatorsmile 068b6438d6 [SPARK-11980][SPARK-10621][SQL] Fix json_tuple and add test cases for
Added Python test cases for the function `isnan`, `isnull`, `nanvl` and `json_tuple`.

Fixed a bug in the function `json_tuple`

rxin , could you help me review my changes? Please let me know anything is missing.

Thank you! Have a good Thanksgiving day!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9977 from gatorsmile/json_tuple.
2015-11-25 23:24:33 -08:00
Shixiong Zhu d29e2ef4cf [SPARK-11935][PYSPARK] Send the Python exceptions in TransformFunction and TransformFunctionSerializer to Java
The Python exception track in TransformFunction and TransformFunctionSerializer is not sent back to Java. Py4j just throws a very general exception, which is hard to debug.

This PRs adds `getFailure` method to get the failure message in Java side.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9922 from zsxwing/SPARK-11935.
2015-11-25 11:47:21 -08:00
Davies Liu dc1d324fdf [SPARK-11969] [SQL] [PYSPARK] visualization of SQL query for pyspark
Currently, we does not have visualization for SQL query from Python, this PR fix that.

cc zsxwing

Author: Davies Liu <davies@databricks.com>

Closes #9949 from davies/pyspark_sql_ui.
2015-11-25 11:11:39 -08:00
felixcheung faabdfa2bd [SPARK-11984][SQL][PYTHON] Fix typos in doc for pivot for scala and python
Author: felixcheung <felixcheung_m@hotmail.com>

Closes #9967 from felixcheung/pypivotdoc.
2015-11-25 10:36:35 -08:00
Jeff Zhang b9b6fbe89b [SPARK-11860][PYSAPRK][DOCUMENTATION] Invalid argument specification …
…for registerFunction [Python]

Straightforward change on the python doc

Author: Jeff Zhang <zjffdu@apache.org>

Closes #9901 from zjffdu/SPARK-11860.
2015-11-25 13:49:58 +00:00
Reynold Xin 151d7c2baf [SPARK-10621][SQL] Consistent naming for functions in SQL, Python, Scala
Author: Reynold Xin <rxin@databricks.com>

Closes #9948 from rxin/SPARK-10621.
2015-11-24 21:30:53 -08:00
Reynold Xin 25bbd3c16e [SPARK-11967][SQL] Consistent use of varargs for multiple paths in DataFrameReader
This patch makes it consistent to use varargs in all DataFrameReader methods, including Parquet, JSON, text, and the generic load function.

Also added a few more API tests for the Java API.

Author: Reynold Xin <rxin@databricks.com>

Closes #9945 from rxin/SPARK-11967.
2015-11-24 18:16:07 -08:00
Reynold Xin f315272279 [SPARK-11946][SQL] Audit pivot API for 1.6.
Currently pivot's signature looks like

```scala
scala.annotation.varargs
def pivot(pivotColumn: Column, values: Column*): GroupedData

scala.annotation.varargs
def pivot(pivotColumn: String, values: Any*): GroupedData
```

I think we can remove the one that takes "Column" types, since callers should always be passing in literals. It'd also be more clear if the values are not varargs, but rather Seq or java.util.List.

I also made similar changes for Python.

Author: Reynold Xin <rxin@databricks.com>

Closes #9929 from rxin/SPARK-11946.
2015-11-24 12:54:37 -08:00
Bryan Cutler 105745645b [SPARK-10560][PYSPARK][MLLIB][DOCS] Make StreamingLogisticRegressionWithSGD Python API equal to Scala one
This is to bring the API documentation of StreamingLogisticReressionWithSGD and StreamingLinearRegressionWithSGC in line with the Scala versions.

-Fixed the algorithm descriptions
-Added default values to parameter descriptions
-Changed StreamingLogisticRegressionWithSGD regParam to default to 0, as in the Scala version

Author: Bryan Cutler <bjcutler@us.ibm.com>

Closes #9141 from BryanCutler/StreamingLogisticRegressionWithSGD-python-api-sync.
2015-11-23 17:11:51 -08:00
Davies Liu 1d91202010 [SPARK-11836][SQL] udf/cast should not create new SQLContext
They should use the existing SQLContext.

Author: Davies Liu <davies@databricks.com>

Closes #9914 from davies/create_udf.
2015-11-23 13:44:30 -08:00
Shixiong Zhu be7a2cfd97 [SPARK-11870][STREAMING][PYSPARK] Rethrow the exceptions in TransformFunction and TransformFunctionSerializer
TransformFunction and TransformFunctionSerializer don't rethrow the exception, so when any exception happens, it just return None. This will cause some weird NPE and confuse people.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9847 from zsxwing/pyspark-streaming-exception.
2015-11-20 14:23:01 -08:00