aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-16772][PYTHON][DOCS] Restore "datatype string" to Python API docstringsNicholas Chammas2016-07-292-12/+8
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR corrects [an error made in an earlier PR](https://github.com/apache/spark/pull/14393/files#r72843069). ## How was this patch tested? ```sh $ ./dev/lint-python PEP8 checks passed. rm -rf _build/* pydoc checks passed. ``` I also built the docs and confirmed that they looked good in my browser. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #14408 from nchammas/SPARK-16772.
* [SPARK-16772] Correct API doc references to PySpark classes + formatting fixesNicholas Chammas2016-07-288-58/+75
| | | | | | | | | | | | | | | | | | ## What's Been Changed The PR corrects several broken or missing class references in the Python API docs. It also correct formatting problems. For example, you can see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.registerFunction) how Sphinx is not picking up the reference to `DataType`. That's because the reference is relative to the current module, whereas `DataType` is in a different module. You can also see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.createDataFrame) how the formatting for byte, tinyint, and so on is italic instead of monospace. That's because in ReST single backticks just make things italic, unlike in Markdown. ## Testing I tested this PR by [building the Python docs](https://github.com/apache/spark/tree/master/docs#generating-the-documentation-html) and reviewing the results locally in my browser. I confirmed that the broken or missing class references were resolved, and that the formatting was corrected. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #14393 from nchammas/python-docstring-fixes.
* [SPARK-16662][PYSPARK][SQL] fix HiveContext warning bugWeichenXu2016-07-231-5/+4
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? move the `HiveContext` deprecate warning printing statement into `HiveContext` constructor. so that this warning will appear only when we use `HiveContext` otherwise this warning will always appear if we reference the pyspark.ml.context code file. ## How was this patch tested? Manual. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14301 from WeichenXu123/hiveContext_python_warning_update.
* [SPARK-16651][PYSPARK][DOC] Make `withColumnRenamed/drop` description more ↵Dongjoon Hyun2016-07-221-0/+2
| | | | | | | | | | | | | | | | consistent with Scala API ## What changes were proposed in this pull request? `withColumnRenamed` and `drop` is a no-op if the given column name does not exists. Python documentation also describe that, but this PR adds more explicit line consistently with Scala to reduce the ambiguity. ## How was this patch tested? It's about docs. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14288 from dongjoon-hyun/SPARK-16651.
* [DOC] improve python doc for rdd.histogram and dataframe.joinMortada Mehyar2016-07-181-5/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? doc change only ## How was this patch tested? doc change only Author: Mortada Mehyar <mortada.mehyar@gmail.com> Closes #14253 from mortada/histogram_typos.
* [SPARK-16546][SQL][PYSPARK] update python dataframe.dropWeichenXu2016-07-141-8/+19
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Make `dataframe.drop` API in python support multi-columns parameters, so that it is the same with scala API. ## How was this patch tested? The doc test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14203 from WeichenXu123/drop_python_api.
* [SPARK-16503] SparkSession should provide Spark versionLiwei Lin2016-07-131-0/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch enables SparkSession to provide spark version. ## How was this patch tested? Manual test: ``` scala> sc.version res0: String = 2.1.0-SNAPSHOT scala> spark.version res1: String = 2.1.0-SNAPSHOT ``` ``` >>> sc.version u'2.1.0-SNAPSHOT' >>> spark.version u'2.1.0-SNAPSHOT' ``` Author: Liwei Lin <lwlin7@gmail.com> Closes #14165 from lw-lin/add-version.
* [SPARK-16429][SQL] Include `StringType` columns in `describe()`Dongjoon Hyun2016-07-081-4/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, Spark `describe` supports `StringType`. However, `describe()` returns a dataset for only all numeric columns. This PR aims to include `StringType` columns in `describe()`, `describe` without argument. **Background** ```scala scala> spark.read.json("examples/src/main/resources/people.json").describe("age", "name").show() +-------+------------------+-------+ |summary| age| name| +-------+------------------+-------+ | count| 2| 3| | mean| 24.5| null| | stddev|7.7781745930520225| null| | min| 19| Andy| | max| 30|Michael| +-------+------------------+-------+ ``` **Before** ```scala scala> spark.read.json("examples/src/main/resources/people.json").describe().show() +-------+------------------+ |summary| age| +-------+------------------+ | count| 2| | mean| 24.5| | stddev|7.7781745930520225| | min| 19| | max| 30| +-------+------------------+ ``` **After** ```scala scala> spark.read.json("examples/src/main/resources/people.json").describe().show() +-------+------------------+-------+ |summary| age| name| +-------+------------------+-------+ | count| 2| 3| | mean| 24.5| null| | stddev|7.7781745930520225| null| | min| 19| Andy| | max| 30|Michael| +-------+------------------+-------+ ``` ## How was this patch tested? Pass the Jenkins with a update testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14095 from dongjoon-hyun/SPARK-16429.
* [SPARK-13638][SQL] Add quoteAll option to CSV DataFrameWriterJurriaan Pruis2016-07-081-2/+5
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Adds an quoteAll option for writing CSV which will quote all fields. See https://issues.apache.org/jira/browse/SPARK-13638 ## How was this patch tested? Added a test to verify the output columns are quoted for all fields in the Dataframe Author: Jurriaan Pruis <email@jurriaanpruis.nl> Closes #13374 from jurriaan/csv-quote-all.
* [SPARK-16052][SQL] Improve `CollapseRepartition` optimizer for ↵Dongjoon Hyun2016-07-081-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Repartition/RepartitionBy ## What changes were proposed in this pull request? This PR improves `CollapseRepartition` to optimize the adjacent combinations of **Repartition** and **RepartitionBy**. Also, this PR adds a testsuite for this optimizer. **Target Scenario** ```scala scala> val dsView1 = spark.range(8).repartition(8, $"id") scala> dsView1.createOrReplaceTempView("dsView1") scala> sql("select id from dsView1 distribute by id").explain(true) ``` **Before** ```scala scala> sql("select id from dsView1 distribute by id").explain(true) == Parsed Logical Plan == 'RepartitionByExpression ['id] +- 'Project ['id] +- 'UnresolvedRelation `dsView1` == Analyzed Logical Plan == id: bigint RepartitionByExpression [id#0L] +- Project [id#0L] +- SubqueryAlias dsview1 +- RepartitionByExpression [id#0L], 8 +- Range (0, 8, splits=8) == Optimized Logical Plan == RepartitionByExpression [id#0L] +- RepartitionByExpression [id#0L], 8 +- Range (0, 8, splits=8) == Physical Plan == Exchange hashpartitioning(id#0L, 200) +- Exchange hashpartitioning(id#0L, 8) +- *Range (0, 8, splits=8) ``` **After** ```scala scala> sql("select id from dsView1 distribute by id").explain(true) == Parsed Logical Plan == 'RepartitionByExpression ['id] +- 'Project ['id] +- 'UnresolvedRelation `dsView1` == Analyzed Logical Plan == id: bigint RepartitionByExpression [id#0L] +- Project [id#0L] +- SubqueryAlias dsview1 +- RepartitionByExpression [id#0L], 8 +- Range (0, 8, splits=8) == Optimized Logical Plan == RepartitionByExpression [id#0L] +- Range (0, 8, splits=8) == Physical Plan == Exchange hashpartitioning(id#0L, 200) +- *Range (0, 8, splits=8) ``` ## How was this patch tested? Pass the Jenkins tests (including a new testsuite). Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13765 from dongjoon-hyun/SPARK-16052.
* [MINOR][PYSPARK][DOC] Fix wrongly formatted examples in PySpark documentationhyukjinkwon2016-07-065-16/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes wrongly formatted examples in PySpark documentation as below: - **`SparkSession`** - **Before** ![2016-07-06 11 34 41](https://cloud.githubusercontent.com/assets/6477701/16605847/ae939526-436d-11e6-8ab8-6ad578362425.png) - **After** ![2016-07-06 11 33 56](https://cloud.githubusercontent.com/assets/6477701/16605845/ace9ee78-436d-11e6-8923-b76d4fc3e7c3.png) - **`Builder`** - **Before** ![2016-07-06 11 34 44](https://cloud.githubusercontent.com/assets/6477701/16605844/aba60dbc-436d-11e6-990a-c87bc0281c6b.png) - **After** ![2016-07-06 1 26 37](https://cloud.githubusercontent.com/assets/6477701/16607562/586704c0-437d-11e6-9483-e0af93d8f74e.png) This PR also fixes several similar instances across the documentation in `sql` PySpark module. ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #14063 from HyukjinKwon/minor-pyspark-builder.
* [SPARK-16335][SQL] Structured streaming should fail if source directory does ↵Reynold Xin2016-07-011-7/+4
| | | | | | | | | | | | | | not exist ## What changes were proposed in this pull request? In structured streaming, Spark does not report errors when the specified directory does not exist. This is a behavior different from the batch mode. This patch changes the behavior to fail if the directory does not exist (when the path is not a glob pattern). ## How was this patch tested? Updated unit tests to reflect the new behavior. Author: Reynold Xin <rxin@databricks.com> Closes #14002 from rxin/SPARK-16335.
* [SPARK-15954][SQL] Disable loading test tables in Python testsReynold Xin2016-06-301-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch introduces a flag to disable loading test tables in TestHiveSparkSession and disables that in Python. This fixes an issue in which python/run-tests would fail due to failure to load test tables. Note that these test tables are not used outside of HiveCompatibilitySuite. In the long run we should probably decouple the loading of test tables from the test Hive setup. ## How was this patch tested? This is a test only change. Author: Reynold Xin <rxin@databricks.com> Closes #14005 from rxin/SPARK-15954.
* [SPARK-16313][SQL] Spark should not silently drop exceptions in file listingReynold Xin2016-06-302-2/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? Spark silently drops exceptions during file listing. This is a very bad behavior because it can mask legitimate errors and the resulting plan will silently have 0 rows. This patch changes it to not silently drop the errors. ## How was this patch tested? Manually verified. Author: Reynold Xin <rxin@databricks.com> Closes #13987 from rxin/SPARK-16313.
* [SPARK-16289][SQL] Implement posexplode table generating functionDongjoon Hyun2016-06-301-0/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive. **Before** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7 ``` **After** ```scala scala> sql("select posexplode(map('a', 1, 'b', 2))").show +---+---+-----+ |pos|key|value| +---+---+-----+ | 0| a| 1| | 1| b| 2| +---+---+-----+ ``` For `array` argument, `after` is the same with `before`. ``` scala> sql("select posexplode(array(1, 2, 3))").show +---+---+ |pos|col| +---+---+ | 0| 1| | 1| 2| | 2| 3| +---+---+ ``` ## How was this patch tested? Pass the Jenkins tests with newly added testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #13971 from dongjoon-hyun/SPARK-16289.
* [SPARK-15820][PYSPARK][SQL] Add Catalog.refreshTable into python APIWeichenXu2016-06-301-0/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add Catalog.refreshTable API into python interface for Spark-SQL. ## How was this patch tested? Existing test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #13558 from WeichenXu123/update_python_sql_interface_refreshTable.
* [TRIVIAL] [PYSPARK] Clean up orc compression option as wellhyukjinkwon2016-06-291-2/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR corrects ORC compression option for PySpark as well. I think this was missed mistakenly in https://github.com/apache/spark/pull/13948. ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #13963 from HyukjinKwon/minor-orc-compress.
* [SPARK-16236][SQL][FOLLOWUP] Add Path Option back to Load API in DataFrameReadergatorsmile2016-06-291-1/+3
| | | | | | | | | | | | | | #### What changes were proposed in this pull request? In Python API, we have the same issue. Thanks for identifying this issue, zsxwing ! Below is an example: ```Python spark.read.format('json').load('python/test_support/sql/people.json') ``` #### How was this patch tested? Existing test cases cover the changes by this PR Author: gatorsmile <gatorsmile@gmail.com> Closes #13965 from gatorsmile/optionPaths.
* [SPARK-16266][SQL][STREAING] Moved DataStreamReader/Writer from pyspark.sql ↵Tathagata Das2016-06-285-499/+505
| | | | | | | | | | | | | | | | | | to pyspark.sql.streaming ## What changes were proposed in this pull request? - Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming to make them consistent with scala packaging - Exposed the necessary classes in sql.streaming package so that they appear in the docs - Added pyspark.sql.streaming module to the docs ## How was this patch tested? - updated unit tests. - generated docs for testing visibility of pyspark.sql.streaming classes. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13955 from tdas/SPARK-16266.
* [SPARK-16268][PYSPARK] SQLContext should import DataStreamReaderShixiong Zhu2016-06-281-2/+9
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fixed the following error: ``` >>> sqlContext.readStream Traceback (most recent call last): File "<stdin>", line 1, in <module> File "...", line 442, in readStream return DataStreamReader(self._wrapped) NameError: global name 'DataStreamReader' is not defined ``` ## How was this patch tested? The added test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13958 from zsxwing/fix-import.
* [MINOR][DOCS][STRUCTURED STREAMING] Minor doc fixes around `DataFrameWriter` ↵Burak Yavuz2016-06-281-2/+2
| | | | | | | | | | | | and `DataStreamWriter` ## What changes were proposed in this pull request? Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start Author: Burak Yavuz <brkyvz@gmail.com> Closes #13952 from brkyvz/minor-doc-fix.
* [SPARK-16175] [PYSPARK] handle None for UDTDavies Liu2016-06-282-2/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Scala UDT will bypass all the null and will not pass them into serialize() and deserialize() of UDT, this PR update the Python UDT to do this as well. ## How was this patch tested? Added tests. Author: Davies Liu <davies@databricks.com> Closes #13878 from davies/udt_null.
* [SPARK-16259][PYSPARK] cleanup options in DataFrame read/write APIDavies Liu2016-06-281-99/+20
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? There are some duplicated code for options in DataFrame reader/writer API, this PR clean them up, it also fix a bug for `escapeQuotes` of csv(). ## How was this patch tested? Existing tests. Author: Davies Liu <davies@databricks.com> Closes #13948 from davies/csv_options.
* [SPARK-16224] [SQL] [PYSPARK] SparkSession builder's configs need to be set ↵Yin Huai2016-06-282-1/+49
| | | | | | | | | | | | | | to the existing Scala SparkContext's SparkConf ## What changes were proposed in this pull request? When we create a SparkSession at the Python side, it is possible that a SparkContext has been created. For this case, we need to set configs of the SparkSession builder to the Scala SparkContext's SparkConf (we need to do so because conf changes on a active Python SparkContext will not be propagated to the JVM side). Otherwise, we may create a wrong SparkSession (e.g. Hive support is not enabled even if enableHiveSupport is called). ## How was this patch tested? New tests and manual tests. Author: Yin Huai <yhuai@databricks.com> Closes #13931 from yhuai/SPARK-16224.
* [SPARK-16128][SQL] Allow setting length of characters to be truncated to, in ↵Prashant Sharma2016-06-281-3/+15
| | | | | | | | | | | | | | | | | | Dataset.show function. ## What changes were proposed in this pull request? Allowing truncate to a specific number of character is convenient at times, especially while operating from the REPL. Sometimes those last few characters make all the difference, and showing everything brings in whole lot of noise. ## How was this patch tested? Existing tests. + 1 new test in DataFrameSuite. For SparkR and pyspark, existing tests and manual testing. Author: Prashant Sharma <prashsh1@in.ibm.com> Author: Prashant Sharma <prashant@apache.org> Closes #13839 from ScrapCodes/add_truncateTo_DF.show.
* [SPARK-16220][SQL] Revert Change to Bring Back SHOW FUNCTIONS FunctionalityBill Chambers2016-06-271-1/+11
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Fix tests regarding show functions functionality - Revert `catalog.ListFunctions` and `SHOW FUNCTIONS` to return to `Spark 1.X` functionality. Cherry picked changes from this PR: https://github.com/apache/spark/pull/13413/files ## How was this patch tested? Unit tests. Author: Bill Chambers <bill@databricks.com> Author: Bill Chambers <wchambers@ischool.berkeley.edu> Closes #13916 from anabranch/master.
* [SPARK-16179][PYSPARK] fix bugs for Python udf in generateDavies Liu2016-06-241-0/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fix the bug when Python UDF is used in explode (generator), GenerateExec requires that all the attributes in expressions should be resolvable from children when creating, we should replace the children first, then replace it's expressions. ``` >>> df.select(explode(f(*df))).show() Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/home/vlad/dev/spark/python/pyspark/sql/dataframe.py", line 286, in show print(self._jdf.showString(n, truncate)) File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/home/vlad/dev/spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/home/vlad/dev/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o52.showString. : org.apache.spark.sql.catalyst.errors.package$TreeNodeException: makeCopy, tree: Generate explode(<lambda>(_1#0L)), false, false, [col#15L] +- Scan ExistingRDD[_1#0L] at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy(TreeNode.scala:387) at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:69) at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:45) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressionsDown(QueryPlan.scala:177) at org.apache.spark.sql.catalyst.plans.QueryPlan.transformExpressions(QueryPlan.scala:144) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.org$apache$spark$sql$execution$python$ExtractPythonUDFs$$extract(ExtractPythonUDFs.scala:153) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:114) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$$anonfun$apply$2.applyOrElse(ExtractPythonUDFs.scala:113) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:301) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:300) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:298) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319) at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:298) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:113) at org.apache.spark.sql.execution.python.ExtractPythonUDFs$.apply(ExtractPythonUDFs.scala:93) at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution$$anonfun$prepareForExecution$1.apply(QueryExecution.scala:95) at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124) at scala.collection.immutable.List.foldLeft(List.scala:84) at org.apache.spark.sql.execution.QueryExecution.prepareForExecution(QueryExecution.scala:95) at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:85) at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:85) at org.apache.spark.sql.Dataset.withTypedCallback(Dataset.scala:2557) at org.apache.spark.sql.Dataset.head(Dataset.scala:1923) at org.apache.spark.sql.Dataset.take(Dataset.scala:2138) at org.apache.spark.sql.Dataset.showString(Dataset.scala:239) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) at py4j.Gateway.invoke(Gateway.java:280) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:211) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.reflect.InvocationTargetException at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1$$anonfun$apply$13.apply(TreeNode.scala:413) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:412) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$makeCopy$1.apply(TreeNode.scala:387) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) ... 42 more Caused by: org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: pythonUDF0#20 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:279) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:278) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$5.apply(TreeNode.scala:321) at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:179) at org.apache.spark.sql.catalyst.trees.TreeNode.transformChildren(TreeNode.scala:319) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:284) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:268) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:87) at org.apache.spark.sql.execution.GenerateExec.<init>(GenerateExec.scala:63) ... 52 more Caused by: java.lang.RuntimeException: Couldn't find pythonUDF0#20 in [_1#0L] at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:94) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:88) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) ... 67 more ``` ## How was this patch tested? Added regression tests. Author: Davies Liu <davies@databricks.com> Closes #13883 from davies/udf_in_generate.
* [SPARK-16086] [SQL] [PYSPARK] create Row without any fieldsDavies Liu2016-06-212-6/+12
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR allows us to create a Row without any fields. ## How was this patch tested? Added a test for empty row and udf without arguments. Author: Davies Liu <davies@databricks.com> Closes #13812 from davies/no_argus.
* [SPARK-13792][SQL] Addendum: Fix Python APIReynold Xin2016-06-211-21/+33
| | | | | | | | | | | | ## What changes were proposed in this pull request? This is a follow-up to https://github.com/apache/spark/pull/13795 to properly set CSV options in Python API. As part of this, I also make the Python option setting for both CSV and JSON more robust against positional errors. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #13800 from rxin/SPARK-13792-2.
* Revert "[SPARK-16086] [SQL] fix Python UDF without arguments (for 1.6)"Xiangrui Meng2016-06-212-8/+6
| | | | This reverts commit a46553cbacf0e4012df89fe55385dec5beaa680a.
* [SPARK-13792][SQL] Limit logging of bad records in CSV data sourceReynold Xin2016-06-201-0/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request adds a new option (maxMalformedLogPerPartition) in CSV reader to limit the maximum of logging message Spark generates per partition for malformed records. The error log looks something like ``` 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: Dropping malformed line: adsf,1,4 16/06/20 18:50:14 WARN CSVRelation: More than 10 malformed records have been found on this partition. Malformed records from now on will not be logged. ``` Closes #12173 ## How was this patch tested? Manually tested. Author: Reynold Xin <rxin@databricks.com> Closes #13795 from rxin/SPARK-13792.
* [SPARK-16086] [SQL] fix Python UDF without arguments (for 1.6)Davies Liu2016-06-202-6/+8
| | | | | | | | | | | | | Fix the bug for Python UDF that does not have any arguments. Added regression tests. Author: Davies Liu <davies.liu@gmail.com> Closes #13793 from davies/fix_no_arguments. (cherry picked from commit abe36c53d126bb580e408a45245fd8e81806869c) Signed-off-by: Davies Liu <davies.liu@gmail.com>
* [SPARK-15973][PYSPARK] Fix GroupedData DocumentationJosh Howes2016-06-171-11/+11
| | | | | | | | | | | | | | | | | *This contribution is my original work and that I license the work to the project under the project's open source license.* ## What changes were proposed in this pull request? Documentation updates to PySpark's GroupedData ## How was this patch tested? Manual Tests Author: Josh Howes <josh.howes@gmail.com> Author: Josh Howes <josh.howes@maxpoint.com> Closes #13724 from josh-howes/bugfix/SPARK-15973.
* [SPARK-15803] [PYSPARK] Support with statement syntax for SparkSessionJeff Zhang2016-06-171-0/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Support with statement syntax for SparkSession in pyspark ## How was this patch tested? Manually verify it. Although I can add unit test for it, it would affect other unit test because the SparkContext is stopped after the with statement. Author: Jeff Zhang <zjffdu@apache.org> Closes #13541 from zjffdu/SPARK-15803.
* [SPARK-15981][SQL][STREAMING] Fixed bug and added tests in DataStreamReader ↵Tathagata Das2016-06-161-122/+136
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Python API ## What changes were proposed in this pull request? - Fixed bug in Python API of DataStreamReader. Because a single path was being converted to a array before calling Java DataStreamReader method (which takes a string only), it gave the following error. ``` File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 947, in pyspark.sql.readwriter.DataStreamReader.json Failed example: json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema) Exception raised: Traceback (most recent call last): File "/System/Library/Frameworks/Python.framework/Versions/2.6/lib/python2.6/doctest.py", line 1253, in __run compileflags, 1) in test.globs File "<doctest pyspark.sql.readwriter.DataStreamReader.json[0]>", line 1, in <module> json_sdf = spark.readStream.json(os.path.join(tempfile.mkdtemp(), 'data'), schema = sdf_schema) File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/readwriter.py", line 963, in json return self._df(self._jreader.json(path)) File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ answer, self.gateway_client, self.target_id, self.name) File "/Users/tdas/Projects/Spark/spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/Users/tdas/Projects/Spark/spark/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 316, in get_return_value format(target_id, ".", name, value)) Py4JError: An error occurred while calling o121.json. Trace: py4j.Py4JException: Method json([class java.util.ArrayList]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326) at py4j.Gateway.invoke(Gateway.java:272) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:211) at java.lang.Thread.run(Thread.java:744) ``` - Reduced code duplication between DataStreamReader and DataFrameWriter - Added missing Python doctests ## How was this patch tested? New tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13703 from tdas/SPARK-15981.
* [SPARK-15888] [SQL] fix Python UDF with aggregateDavies Liu2016-06-151-1/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? After we move the ExtractPythonUDF rule into physical plan, Python UDF can't work on top of aggregate anymore, because they can't be evaluated before aggregate, should be evaluated after aggregate. This PR add another rule to extract these kind of Python UDF from logical aggregate, create a Project on top of Aggregate. ## How was this patch tested? Added regression tests. The plan of added test query looks like this: ``` == Parsed Logical Plan == 'Project [<lambda>('k, 's) AS t#26] +- Aggregate [<lambda>(key#5L)], [<lambda>(key#5L) AS k#17, sum(cast(<lambda>(value#6) as bigint)) AS s#22L] +- LogicalRDD [key#5L, value#6] == Analyzed Logical Plan == t: int Project [<lambda>(k#17, s#22L) AS t#26] +- Aggregate [<lambda>(key#5L)], [<lambda>(key#5L) AS k#17, sum(cast(<lambda>(value#6) as bigint)) AS s#22L] +- LogicalRDD [key#5L, value#6] == Optimized Logical Plan == Project [<lambda>(agg#29, agg#30L) AS t#26] +- Aggregate [<lambda>(key#5L)], [<lambda>(key#5L) AS agg#29, sum(cast(<lambda>(value#6) as bigint)) AS agg#30L] +- LogicalRDD [key#5L, value#6] == Physical Plan == *Project [pythonUDF0#37 AS t#26] +- BatchEvalPython [<lambda>(agg#29, agg#30L)], [agg#29, agg#30L, pythonUDF0#37] +- *HashAggregate(key=[<lambda>(key#5L)#31], functions=[sum(cast(<lambda>(value#6) as bigint))], output=[agg#29,agg#30L]) +- Exchange hashpartitioning(<lambda>(key#5L)#31, 200) +- *HashAggregate(key=[pythonUDF0#34 AS <lambda>(key#5L)#31], functions=[partial_sum(cast(pythonUDF1#35 as bigint))], output=[<lambda>(key#5L)#31,sum#33L]) +- BatchEvalPython [<lambda>(key#5L), <lambda>(value#6)], [key#5L, value#6, pythonUDF0#34, pythonUDF1#35] +- Scan ExistingRDD[key#5L,value#6] ``` Author: Davies Liu <davies@databricks.com> Closes #13682 from davies/fix_py_udf.
* [SPARK-15953][WIP][STREAMING] Renamed ContinuousQuery to StreamingQueryTathagata Das2016-06-157-100/+99
| | | | | | | | | | Renamed for simplicity, so that its obvious that its related to streaming. Existing unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13673 from tdas/SPARK-15953.
* [SPARK-15935][PYSPARK] Fix a wrong format tag in the error messageShixiong Zhu2016-06-141-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? A follow up PR for #13655 to fix a wrong format tag. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13665 from zsxwing/fix.
* [SPARK-15933][SQL][STREAMING] Refactored DF reader-writer to use readStream ↵Tathagata Das2016-06-146-168/+570
| | | | | | | | | | | | | | | | and writeStream for streaming DFs ## What changes were proposed in this pull request? Currently, the DataFrameReader/Writer has method that are needed for streaming and non-streaming DFs. This is quite awkward because each method in them through runtime exception for one case or the other. So rather having half the methods throw runtime exceptions, its just better to have a different reader/writer API for streams. - [x] Python API!! ## How was this patch tested? Existing unit tests + two sets of unit tests for DataFrameReader/Writer and DataStreamReader/Writer. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13653 from tdas/SPARK-15933.
* [SPARK-15935][PYSPARK] Enable test for sql/streaming.py and fix these testsShixiong Zhu2016-06-143-20/+54
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR just enables tests for sql/streaming.py and also fixes the failures. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13655 from zsxwing/python-streaming-test.
* [SPARK-15663][SQL] SparkSession.catalog.listFunctions shouldn't include the ↵Sandeep Singh2016-06-131-11/+1
| | | | | | | | | | | | | | list of built-in functions ## What changes were proposed in this pull request? SparkSession.catalog.listFunctions currently returns all functions, including the list of built-in functions. This makes the method not as useful because anytime it is run the result set contains over 100 built-in functions. ## How was this patch tested? CatalogSuite Author: Sandeep Singh <sandeep@techaddict.me> Closes #13413 from techaddict/SPARK-15663.
* [SPARK-15898][SQL] DataFrameReader.text should return DataFrameWenchen Fan2016-06-121-4/+4
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? We want to maintain API compatibility for DataFrameReader.text, and will introduce a new API called DataFrameReader.textFile which returns Dataset[String]. affected PRs: https://github.com/apache/spark/pull/11731 https://github.com/apache/spark/pull/13104 https://github.com/apache/spark/pull/13184 ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #13604 from cloud-fan/revert.
* [SPARK-15840][SQL] Add two missing options in documentation and some option ↵hyukjinkwon2016-06-111-13/+27
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | related changes ## What changes were proposed in this pull request? This PR 1. Adds the documentations for some missing options, `inferSchema` and `mergeSchema` for Python and Scala. 2. Fiixes `[[DataFrame]]` to ```:class:`DataFrame` ``` so that this can be shown - from ![2016-06-09 9 31 16](https://cloud.githubusercontent.com/assets/6477701/15929721/8b864734-2e89-11e6-83f6-207527de4ac9.png) - to (with class link) ![2016-06-09 9 31 00](https://cloud.githubusercontent.com/assets/6477701/15929717/8a03d728-2e89-11e6-8a3f-08294964db22.png) (Please refer [the latest documentation](https://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/api/python/pyspark.sql.html)) 3. Moves `mergeSchema` option to `ParquetOptions` with removing unused options, `metastoreSchema` and `metastoreTableName`. They are not used anymore. They were removed in https://github.com/apache/spark/commit/e720dda42e806229ccfd970055c7b8a93eb447bf and there are no use cases as below: ```bash grep -r -e METASTORE_SCHEMA -e \"metastoreSchema\" -e \"metastoreTableName\" -e METASTORE_TABLE_NAME . ``` ``` ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala: private[sql] val METASTORE_SCHEMA = "metastoreSchema" ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala: private[sql] val METASTORE_TABLE_NAME = "metastoreTableName" ./sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala: ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier( ``` It only sets `metastoreTableName` in the last case but does not use the table name. 4. Sets the correct default values (in the documentation) for `compression` option for ORC(`snappy`, see [OrcOptions.scala#L33-L42](https://github.com/apache/spark/blob/3ded5bc4db2badc9ff49554e73421021d854306b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala#L33-L42)) and Parquet(`the value specified in SQLConf`, see [ParquetOptions.scala#L38-L47](https://github.com/apache/spark/blob/3ded5bc4db2badc9ff49554e73421021d854306b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala#L38-L47)) and `columnNameOfCorruptRecord` for JSON(`the value specified in SQLConf`, see [JsonFileFormat.scala#L53-L55](https://github.com/apache/spark/blob/4538443e276597530a27c6922e48503677b13956/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala#L53-L55) and [JsonFileFormat.scala#L105-L106](https://github.com/apache/spark/blob/4538443e276597530a27c6922e48503677b13956/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala#L105-L106)). ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Author: Hyukjin Kwon <gurwls223@gmail.com> Closes #13576 from HyukjinKwon/SPARK-15840.
* [SPARK-15585][SQL] Add doc for turning off quotationsTakeshi YAMAMURO2016-06-111-2/+4
| | | | | | | | | | | | ## What changes were proposed in this pull request? This pr is to add doc for turning off quotations because this behavior is different from `com.databricks.spark.csv`. ## How was this patch tested? Check behavior to put an empty string in csv options. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #13616 from maropu/SPARK-15585-2.
* [MINOR] Fix Typos 'an -> a'Zheng RuiFeng2016-06-063-3/+3
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `an -> a` Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one. ## How was this patch tested? manual tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #13515 from zhengruifeng/an_a.
* Revert "[SPARK-15585][SQL] Fix NULL handling along with a spark-csv behaivour"Reynold Xin2016-06-051-39/+42
| | | | This reverts commit b7e8d1cb3ce932ba4a784be59744af8a8ef027ce.
* [SPARK-15585][SQL] Fix NULL handling along with a spark-csv behaivourTakeshi YAMAMURO2016-06-051-42/+39
| | | | | | | | | | | | | ## What changes were proposed in this pull request? This pr fixes the behaviour of `format("csv").option("quote", null)` along with one of spark-csv. Also, it explicitly sets default values for CSV options in python. ## How was this patch tested? Added tests in CSVSuite. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #13372 from maropu/SPARK-15585.
* [SPARK-15686][SQL] Move user-facing streaming classes into sql.streamingReynold Xin2016-06-012-2/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch moves all user-facing structured streaming classes into sql.streaming. As part of this, I also added some since version annotation to methods and classes that don't have them. ## How was this patch tested? Updated tests to reflect the moves. Author: Reynold Xin <rxin@databricks.com> Closes #13429 from rxin/SPARK-15686.
* [SPARK-15517][SQL][STREAMING] Add support for complete output mode in ↵Tathagata Das2016-05-312-3/+24
| | | | | | | | | | | | | | | | | | | | | | | | | | | Structure Streaming ## What changes were proposed in this pull request? Currently structured streaming only supports append output mode. This PR adds the following. - Added support for Complete output mode in the internal state store, analyzer and planner. - Added public API in Scala and Python for users to specify output mode - Added checks for unsupported combinations of output mode and DF operations - Plans with no aggregation should support only Append mode - Plans with aggregation should support only Update and Complete modes - Default output mode is Append mode (**Question: should we change this to automatically set to Complete mode when there is aggregation?**) - Added support for Complete output mode in Memory Sink. So Memory Sink internally supports append and complete, update. But from public API only Complete and Append output modes are supported. ## How was this patch tested? Unit tests in various test suites - StreamingAggregationSuite: tests for complete mode - MemorySinkSuite: tests for checking behavior in Append and Complete modes. - UnsupportedOperationSuite: tests for checking unsupported combinations of DF ops and output modes - DataFrameReaderWriterSuite: tests for checking that output mode cannot be called on static DFs - Python doc test and existing unit tests modified to call write.outputMode. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13286 from tdas/complete-mode.
* Revert "[SPARK-11753][SQL][TEST-HADOOP2.2] Make allowNonNumericNumbers ↵Shixiong Zhu2016-05-311-3/+0
| | | | | | | | | | | | | | | | option work ## What changes were proposed in this pull request? This reverts commit c24b6b679c3efa053f7de19be73eb36dc70d9930. Sent a PR to run Jenkins tests due to the revert conflicts of `dev/deps/spark-deps-hadoop*`. ## How was this patch tested? Jenkins unit tests, integration tests, manual tests) Author: Shixiong Zhu <shixiong@databricks.com> Closes #13417 from zsxwing/revert-SPARK-11753.