aboutsummaryrefslogtreecommitdiff
path: root/python
Commit message (Collapse)AuthorAgeFilesLines
* Correct SparseVector.parse documentationMiles Yucht2016-02-161-1/+1
| | | | | | | | 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.
* [SPARK-13296][SQL] Move UserDefinedFunction into sql.expressions.Reynold Xin2016-02-132-4/+4
| | | | | | | | | | | | | | | | 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.
* [SPARK-12363][MLLIB] Remove setRun and fix PowerIterationClustering failed testLiang-Chi Hsieh2016-02-131-6/+19
| | | | | | | | | | | 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.
* [SPARK-12630][PYSPARK] [DOC] PySpark classification parameter desc to ↵vijaykiran2016-02-121-118/+143
| | | | | | | | | | | 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.
* [SPARK-12962] [SQL] [PySpark] PySpark support covar_samp and covar_popYanbo Liang2016-02-121-6/+35
| | | | | | | | | | PySpark support ```covar_samp``` and ```covar_pop```. cc rxin davies marmbrus Author: Yanbo Liang <ybliang8@gmail.com> Closes #10876 from yanboliang/spark-12962.
* [SPARK-13154][PYTHON] Add linting for pydocsHolden Karau2016-02-121-0/+3
| | | | | | | | | | 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.
* [SPARK-12974][ML][PYSPARK] Add Python API for spark.ml bisecting k-meansYanbo Liang2016-02-121-1/+124
| | | | | | | | Add Python API for spark.ml bisecting k-means. Author: Yanbo Liang <ybliang8@gmail.com> Closes #10889 from yanboliang/spark-12974.
* [SPARK-13153][PYSPARK] ML persistence failed when handle no default value ↵Tommy YU2016-02-111-2/+3
| | | | | | | | | | | | 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.
* [SPARK-13047][PYSPARK][ML] Pyspark Params.hasParam should not throw an errorsethah2016-02-112-4/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-13035][ML][PYSPARK] PySpark ml.clustering support export/importYanbo Liang2016-02-111-4/+25
| | | | | | | | PySpark ml.clustering support export/import. Author: Yanbo Liang <ybliang8@gmail.com> Closes #10999 from yanboliang/spark-13035.
* [MINOR][ML][PYSPARK] Cleanup test cases of clustering.pyYanbo Liang2016-02-112-15/+9
| | | | | | | | | 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.
* [SPARK-13037][ML][PYSPARK] PySpark ml.recommendation support export/importKai Jiang2016-02-111-4/+27
| | | | | | | | PySpark ml.recommendation support export/import. Author: Kai Jiang <jiangkai@gmail.com> Closes #11044 from vectorijk/spark-13037.
* [SPARK-12706] [SQL] grouping() and grouping_id()Davies Liu2016-02-102-11/+55
| | | | | | | | | | | | 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.
* [SPARK-12986][DOC] Fix pydoc warnings in mllib/regression.pyNam Pham2016-02-081-13/+21
| | | | | | | | 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.
* [SPARK-5865][API DOC] Add doc warnings for methods that return local data ↵Tommy YU2016-02-062-0/+23
| | | | | | | | | | | | | 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.
* [SPARK-7997][CORE] Add rpcEnv.awaitTermination() back to SparkEnvShixiong Zhu2016-02-021-2/+2
| | | | | | | | | | | | `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.
* [SPARK-12631][PYSPARK][DOC] PySpark clustering parameter desc to consistent ↵Bryan Cutler2016-02-021-74/+191
| | | | | | | | | | 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.
* [SPARK-13049] Add First/last with ignore nulls to functions.scalaHerman van Hovell2016-01-312-2/+34
| | | | | | | | | | | | | 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.
* [SPARK-13032][ML][PYSPARK] PySpark support model export/import and take ↵Yanbo Liang2016-01-295-29/+236
| | | | | | | | | | | | | | 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.
* [SPARK-12749][SQL] add json option to parse floating-point types as DecimalTypeBrandon Bradley2016-01-281-0/+2
| | | | | | | | | | 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.
* [SPARK-10847][SQL][PYSPARK] Pyspark - DataFrame - Optional Metadata with ↵Jason Lee2016-01-271-0/+7
| | | | | | | | | | `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.
* [SPARK-12780] Inconsistency returning value of ML python models' propertiesXusen Yin2016-01-261-2/+3
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-12780 Author: Xusen Yin <yinxusen@gmail.com> Closes #10724 from yinxusen/SPARK-12780.
* [SPARK-10509][PYSPARK] Reduce excessive param boiler plate codeHolden Karau2016-01-2612-317/+43
| | | | | | | | 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.
* [SPARK-12993][PYSPARK] Remove usage of ADD_FILES in pysparkJeff Zhang2016-01-261-10/+1
| | | | | | | | 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.
* [SPARK-11923][ML] Python API for ml.feature.ChiSqSelectorXusen Yin2016-01-261-1/+97
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-11923 Author: Xusen Yin <yinxusen@gmail.com> Closes #10186 from yinxusen/SPARK-11923.
* [SPARK-10086][MLLIB][STREAMING][PYSPARK] ignore StreamingKMeans test in ↵Xiangrui Meng2016-01-251-0/+1
| | | | | | | | | | | | | | 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.
* [SPARK-11922][PYSPARK][ML] Python api for ml.feature.quantile discretizerHolden Karau2016-01-251-4/+85
| | | | | | | | | | | 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.
* [SPARK-12905][ML][PYSPARK] PCAModel return eigenvalues for PySparkYanbo Liang2016-01-251-0/+11
| | | | | | | | | | ```PCAModel``` can output ```explainedVariance``` at Python side. cc mengxr srowen Author: Yanbo Liang <ybliang8@gmail.com> Closes #10830 from yanboliang/spark-12905.
* [SPARK-12624][PYSPARK] Checks row length when converting Java arrays to ↵Cheng Lian2016-01-241-0/+9
| | | | | | | | | | 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.
* [SPARK-12120][PYSPARK] Improve exception message when failing to init…Jeff Zhang2016-01-241-3/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | …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.
* [SPARK-11295][PYSPARK] Add packages to JUnit output for Python testsGábor Lipták2016-01-205-11/+19
| | | | | | | | | 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.
* Revert "[SPARK-11295] Add packages to JUnit output for Python tests"Xiangrui Meng2016-01-195-18/+10
| | | | This reverts commit c6f971b4aeca7265ab374fa46c5c452461d9b6a7.
* [SPARK-9716][ML] BinaryClassificationEvaluator should accept Double ↵BenFradet2016-01-191-2/+3
| | | | | | | | | | 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.
* [SPARK-11295] Add packages to JUnit output for Python testsGábor Lipták2016-01-195-10/+18
| | | | | | | | | | 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.
* [SPARK-11944][PYSPARK][MLLIB] python mllib.clustering.bisecting k meansHolden Karau2016-01-192-5/+142
| | | | | | | | 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.
* [SPARK-7683][PYSPARK] Confusing behavior of fold function of RDD in pysparkSean Owen2016-01-191-1/+1
| | | | | | | | | | | | 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.
* [SPARK-11925][ML][PYSPARK] Add PySpark missing methods for ml.feature during ↵Yanbo Liang2016-01-151-10/+62
| | | | | | | | | | | | | 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.
* [SPARK-12575][SQL] Grammar parity with existing SQL parserHerman van Hovell2016-01-151-2/+1
| | | | | | | | | | | | | | | | 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.
* [SPARK-12756][SQL] use hash expression in ExchangeWenchen Fan2016-01-132-16/+16
| | | | | | | | | | 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.
* [SPARK-12791][SQL] Simplify CaseWhen by breaking "branches" into ↵Reynold Xin2016-01-131-12/+12
| | | | | | | | | | | | "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.
* [SPARK-12642][SQL] improve the hash expression to be decoupled from unsafe rowWenchen Fan2016-01-131-1/+1
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-12642 Author: Wenchen Fan <wenchen@databricks.com> Closes #10694 from cloud-fan/hash-expr.
* [SPARK-12268][PYSPARK] Make pyspark shell pythonstartup work under python3Erik Selin2016-01-131-1/+3
| | | | | | | | | | 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.
* [SPARK-12652][PYSPARK] Upgrade Py4J to 0.9.1Shixiong Zhu2016-01-125-88/+6
| | | | | | | | | | | | - [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. https://github.com/zsxwing/spark/commit/bfd4b5c040eb29394c3132af3c670b1a7272457c - [x] Verify no leak any more after reverting our workarounds Author: Shixiong Zhu <shixiong@databricks.com> Closes #10692 from zsxwing/py4j-0.9.1.
* [SPARK-12603][MLLIB] PySpark MLlib GaussianMixtureModel should support ↵Yanbo Liang2016-01-111-13/+22
| | | | | | | | | | 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.
* [SPARK-12618][CORE][STREAMING][SQL] Clean up build warnings: 2.0.0 editionSean Owen2016-01-081-1/+1
| | | | | | | | 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.
* [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not Nonezero3232016-01-072-1/+13
| | | | | | | | 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.
* Revert "[SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not None"Yin Huai2016-01-062-13/+1
| | | | | | | | This reverts commit fcd013cf70e7890aa25a8fe3cb6c8b36bf0e1f04. Author: Yin Huai <yhuai@databricks.com> Closes #10632 from yhuai/pythonStyle.
* [SPARK-12617][PYSPARK] Move Py4jCallbackConnectionCleaner to StreamingShixiong Zhu2016-01-062-61/+63
| | | | | | | | 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.
* [SPARK-12006][ML][PYTHON] Fix GMM failure if initialModel is not Nonezero3232016-01-062-1/+13
| | | | | | | | 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.
* [SPARK-11815][ML][PYSPARK] PySpark DecisionTreeClassifier & ↵Yanbo Liang2016-01-062-10/+17
| | | | | | | | | | 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.