aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark
Commit message (Collapse)AuthorAgeFilesLines
* Revert "[SPARK-14014][SQL] Replace existing catalog with SessionCatalog"Andrew Or2016-03-231-1/+1
| | | | This reverts commit 5dfc01976bb0d72489620b4f32cc12d620bb6260.
* [SPARK-12183][ML][MLLIB] Remove mllib tree implementation, and wrap spark.ml oneJoseph K. Bradley2016-03-232-2/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-14014][SQL] Replace existing catalog with SessionCatalogAndrew Or2016-03-231-1/+1
| | | | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-13068][PYSPARK][ML] Type conversion for Pyspark paramssethah2016-03-2310-176/+421
| | | | | | | | | | | | | | | | ## 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.
* [SPARK-14088][SQL] Some Dataset API touch-upReynold Xin2016-03-222-2/+14
| | | | | | | | | | | | | | | ## 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.
* [SPARK-13951][ML][PYTHON] Nested Pipeline persistenceJoseph K. Bradley2016-03-229-175/+300
| | | | | | | | | | | | | | | 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
* [SPARK-13953][SQL] Specifying the field name for corrupted record via option ↵hyukjinkwon2016-03-221-1/+4
| | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-14058][PYTHON] Incorrect docstring in Window.orderzero3232016-03-211-1/+1
| | | | | | | | | | | | | | ## 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.
* [SPARK-13764][SQL] Parse modes in JSON data sourcehyukjinkwon2016-03-211-0/+8
| | | | | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-13993][PYSPARK] Add pyspark Rformula/RforumlaModel save/loadXusen Yin2016-03-201-3/+27
| | | | | | | | | | | | | | ## 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.
* [SPARK-13937][PYSPARK][ML] Change JavaWrapper _java_obj from static to ↵Bryan Cutler2016-03-171-4/+9
| | | | | | | | | | | | | | 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.
* [SPARK-13034] PySpark ml.classification support export/importGayathriMurali2016-03-162-9/+61
| | | | | | | | | | | | | | | | ## 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.
* [SPARK-13038][PYSPARK] Add load/save to pipelineXusen Yin2016-03-166-89/+317
| | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-10380][SQL] Fix confusing documentation examples for ↵Reynold Xin2016-03-143-7/+37
| | | | | | | | | | | | | | | | 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.
* [SPARK-13843][STREAMING] Remove streaming-flume, streaming-mqtt, ↵Shixiong Zhu2016-03-143-473/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-13848][SPARK-5185] Update to Py4J 0.9.2 in order to fix classloading ↵Josh Rosen2016-03-145-49/+22
| | | | | | | | | | | | | | | | 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.
* [SPARK-13671] [SPARK-13311] [SQL] Use different physical plans for RDD and ↵Davies Liu2016-03-121-2/+1
| | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-13807] De-duplicate `Python*Helper` instantiation code in PySpark ↵Josh Rosen2016-03-114-84/+60
| | | | | | | | | | | | 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.
* [SPARK-13787][ML][PYSPARK] Pyspark feature importances for decision tree and ↵sethah2016-03-112-0/+88
| | | | | | | | | | | | | | | | 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.
* [SPARK-13672][ML] Add python examples of BisectingKMeans in ML and MLLIBZheng RuiFeng2016-03-111-0/+1
| | | | | | | | | | | | | | | | 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.
* [SPARK-13244][SQL] Migrates DataFrame to DatasetCheng Lian2016-03-101-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [MINOR] Fix typo in 'hypot' docstringTristan Reid2016-03-091-1/+1
| | | | | | | | | | Minor typo: docstring for pyspark.sql.functions: hypot has extra characters N/A Author: Tristan Reid <treid@netflix.com> Closes #11616 from tristanreid/master.
* [SPARK-13595][BUILD] Move docker, extras modules into externalSean Owen2016-03-091-1/+1
| | | | | | | | | | | | | | ## 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.
* [SPARK-13625][PYSPARK][ML] Added a check to see if an attribute is a ↵Bryan Cutler2016-03-082-1/+19
| | | | | | | | | | | | | | | | 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.
* [SPARK-13593] [SQL] improve the `createDataFrame` to accept data type string ↵Wenchen Fan2016-03-083-23/+214
| | | | | | | | | | | | | | | | | | 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.
* [SPARK-13740][SQL] add null check for _verify_type in types.pyWenchen Fan2016-03-081-7/+26
| | | | | | | | | | | | | | ## 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.
* [SPARK-13697] [PYSPARK] Fix the missing module name of ↵Shixiong Zhu2016-03-062-1/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-12720][SQL] SQL Generation Support for Cube, Rollup, and Grouping Setsgatorsmile2016-03-051-7/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | #### 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.
* [SPARK-13036][SPARK-13318][SPARK-13319] Add save/load for feature.pyXusen Yin2016-03-041-45/+296
| | | | | | | | | | 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.
* [SPARK-13676] Fix mismatched default values for regParam in LogisticRegressionDongjoon Hyun2016-03-041-5/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-13647] [SQL] also check if numeric value is within allowed range in ↵Wenchen Fan2016-03-031-3/+24
| | | | | | | | | | | | | | | | _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.
* [MINOR] Fix typos in comments and testcase name of codeDongjoon Hyun2016-03-031-1/+1
| | | | | | | | | | | | | | ## 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.
* [SPARK-13543][SQL] Support for specifying compression codec for Parquet/ORC ↵hyukjinkwon2016-03-031-21/+34
| | | | | | | | | | | | | | | | 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.
* [SPARK-12877][ML] Add train-validation-split to pysparkJeremyNixon2016-03-032-2/+244
| | | | | | | | | | | | | | ## 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.
* [SPARK-13594][SQL] remove typed operations(e.g. map, flatMap) from python ↵Wenchen Fan2016-03-026-60/+22
| | | | | | | | | | | | | | | | 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.
* [SPARK-13008][ML][PYTHON] Put one alg per line in pyspark.ml all listsJoseph K. Bradley2016-03-013-15/+36
| | | | | | | | | | | | 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.
* [SPARK-13509][SPARK-13507][SQL] Support for writing CSV with a single ↵hyukjinkwon2016-02-291-0/+50
| | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-12633][PYSPARK] [DOC] PySpark regression parameter desc to consistent ↵vijaykiran2016-02-292-164/+166
| | | | | | | | | | | | | 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.
* [SPARK-13545][MLLIB][PYSPARK] Make MLlib LogisticRegressionWithLBFGS's ↵Yanbo Liang2016-02-291-3/+5
| | | | | | | | | | | | | | | | | 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.
* [SPARK-13505][ML] add python api for MaxAbsScalerzlpmichelle2016-02-261-7/+68
| | | | | | | | | | | | ## 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.
* [SPARK-12634][PYSPARK][DOC] PySpark tree parameter desc to consistent formatBryan Cutler2016-02-261-142/+197
| | | | | | | | | | | 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.
* [SPARK-13033] [ML] [PYSPARK] Add import/export for ml.regressionTommy YU2016-02-251-4/+30
| | | | | | | | | | | | | | | | | 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
* [SPARK-13292] [ML] [PYTHON] QuantileDiscretizer should take random seed in ↵Yu ISHIKAWA2016-02-251-6/+8
| | | | | | | | | | | | | | | | 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
* [SPARK-7106][MLLIB][PYSPARK] Support model save/load in Python's FPGrowthKai Jiang2016-02-241-4/+31
| | | | | | | | | | | | | | ## 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.
* [SPARK-13479][SQL][PYTHON] Added Python API for approxQuantileJoseph K. Bradley2016-02-242-0/+61
| | | | | | | | | | | | | | | | | ## 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.
* [SPARK-13250] [SQL] Update PhysicallRDD to convert to UnsafeRow if using the ↵Nong Li2016-02-241-1/+2
| | | | | | | | | | | | | | | | | 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.
* [SPARK-13467] [PYSPARK] abstract python function to simplify pyspark codeWenchen Fan2016-02-243-15/+18
| | | | | | | | | | | | | | ## 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.
* [SPARK-13329] [SQL] considering output for statistics of logical planDavies Liu2016-02-231-2/+2
| | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-13429][MLLIB] Unify Logistic Regression convergence tolerance of ML & ↵Yanbo Liang2016-02-221-2/+2
| | | | | | | | | | | | | | 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.
* [SPARK-12632][PYSPARK][DOC] PySpark fpm and als parameter desc to consistent ↵Bryan Cutler2016-02-222-34/+102
| | | | | | | | | | | | | | 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.