aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/utils.py
Commit message (Collapse)AuthorAgeFilesLines
* [MINOR][DOCS] Remove consecutive duplicated words/typo in Spark RepoNiranjan Padmanabhan2017-01-041-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words. ## How was this patch tested? N/A since only docs or comments were updated. Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com> Closes #16455 from neurons/np.structure_streaming_doc.
* [SPARK-15953][WIP][STREAMING] Renamed ContinuousQuery to StreamingQueryTathagata Das2016-06-151-4/+4
| | | | | | | | | | 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-15686][SQL] Move user-facing streaming classes into sql.streamingReynold Xin2016-06-011-1/+1
| | | | | | | | | | | | ## 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-14603][SQL][FOLLOWUP] Verification of Metadata Operations by Session ↵gatorsmile2016-05-191-1/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Catalog #### What changes were proposed in this pull request? This follow-up PR is to address the remaining comments in https://github.com/apache/spark/pull/12385 The major change in this PR is to issue better error messages in PySpark by using the mechanism that was proposed by davies in https://github.com/apache/spark/pull/7135 For example, in PySpark, if we input the following statement: ```python >>> l = [('Alice', 1)] >>> df = sqlContext.createDataFrame(l) >>> df.createTempView("people") >>> df.createTempView("people") ``` Before this PR, the exception we will get is like ``` Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView self._jdf.createTempView(name) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/Users/xiaoli/IdeaProjects/sparkDelivery/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 o35.createTempView. : org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException: Temporary table 'people' already exists; at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTempView(SessionCatalog.scala:324) at org.apache.spark.sql.SparkSession.createTempView(SparkSession.scala:523) at org.apache.spark.sql.Dataset.createTempView(Dataset.scala:2328) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) 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) ``` After this PR, the exception we will get become cleaner: ``` Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView self._jdf.createTempView(name) File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__ File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 75, in deco raise AnalysisException(s.split(': ', 1)[1], stackTrace) pyspark.sql.utils.AnalysisException: u"Temporary table 'people' already exists;" ``` #### How was this patch tested? Fixed an existing PySpark test case Author: gatorsmile <gatorsmile@gmail.com> Closes #13126 from gatorsmile/followup-14684.
* [SPARK-14603][SQL] Verification of Metadata Operations by Session Cataloggatorsmile2016-05-101-0/+2
| | | | | | | | | | | | | | | | | | Since we cannot really trust if the underlying external catalog can throw exceptions when there is an invalid metadata operation, let's do it in SessionCatalog. - [X] The first step is to unify the error messages issued in Hive-specific Session Catalog and general Session Catalog. - [X] The second step is to verify the inputs of metadata operations for partitioning-related operations. This is moved to a separate PR: https://github.com/apache/spark/pull/12801 - [X] The third step is to add database existence verification in `SessionCatalog` - [X] The fourth step is to add table existence verification in `SessionCatalog` - [X] The fifth step is to add function existence verification in `SessionCatalog` Add test cases and verify the error messages we issued Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #12385 from gatorsmile/verifySessionAPIs.
* [SPARK-14555] Second cut of Python API for Structured StreamingBurak Yavuz2016-04-281-0/+8
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds Python APIs for: - `ContinuousQueryManager` - `ContinuousQueryException` The `ContinuousQueryException` is a very basic wrapper, it doesn't provide the functionality that the Scala side provides, but it follows the same pattern for `AnalysisException`. For `ContinuousQueryManager`, all APIs are provided except for registering listeners. This PR also attempts to fix test flakiness by stopping all active streams just before tests. ## How was this patch tested? Python Doc tests and unit tests Author: Burak Yavuz <brkyvz@gmail.com> Closes #12673 from brkyvz/pyspark-cqm.
* [SPARK-14211][SQL] Remove ANTLR3 based parserHerman van Hovell2016-03-311-1/+1
| | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? This PR removes the ANTLR3 based parser, and moves the new ANTLR4 based parser into the `org.apache.spark.sql.catalyst.parser package`. ### How was this patch tested? Existing unit tests. cc rxin andrewor14 yhuai Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #12071 from hvanhovell/SPARK-14211.
* [SPARK-13713][SQL] Migrate parser from ANTLR3 to ANTLR4Herman van Hovell2016-03-281-0/+8
| | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? The current ANTLR3 parser is quite complex to maintain and suffers from code blow-ups. This PR introduces a new parser that is based on ANTLR4. This parser is based on the [Presto's SQL parser](https://github.com/facebook/presto/blob/master/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4). The current implementation can parse and create Catalyst and SQL plans. Large parts of the HiveQl DDL and some of the DML functionality is currently missing, the plan is to add this in follow-up PRs. This PR is a work in progress, and work needs to be done in the following area's: - [x] Error handling should be improved. - [x] Documentation should be improved. - [x] Multi-Insert needs to be tested. - [ ] Naming and package locations. ### How was this patch tested? Catalyst and SQL unit tests. Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #11557 from hvanhovell/ngParser.
* [SPARK-11804] [PYSPARK] Exception raise when using Jdbc predicates opt…Jeff Zhang2015-11-181-0/+13
| | | | | | | | …ion in PySpark Author: Jeff Zhang <zjffdu@apache.org> Closes #9791 from zjffdu/SPARK-11804.
* [SPARK-11322] [PYSPARK] Keep full stack trace in captured exceptionLiang-Chi Hsieh2015-10-281-4/+15
| | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-11322 As reported by JoshRosen in [databricks/spark-redshift/issues/89](https://github.com/databricks/spark-redshift/issues/89#issuecomment-149828308), the exception-masking behavior sometimes makes debugging harder. To deal with this issue, we should keep full stack trace in the captured exception. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #9283 from viirya/py-exception-stacktrace.
* [SPARK-9166][SQL][PYSPARK] Capture and hide IllegalArgumentException in ↵Liang-Chi Hsieh2015-07-191-0/+8
| | | | | | | | | | | | | | | | | Python API JIRA: https://issues.apache.org/jira/browse/SPARK-9166 Simply capture and hide `IllegalArgumentException` in Python API. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #7497 from viirya/hide_illegalargument and squashes the following commits: 8324dce [Liang-Chi Hsieh] Fix python style. 9ace67d [Liang-Chi Hsieh] Also check exception message. 8b2ce5c [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into hide_illegalargument 7be016a [Liang-Chi Hsieh] Capture and hide IllegalArgumentException in Python.
* [SPARK-8766] support non-ascii character in column namesDavies Liu2015-07-011-3/+3
| | | | | | | | | | | | | | Use UTF-8 to encode the name of column in Python 2, or it may failed to encode with default encoding ('ascii'). This PR also fix a bug when there is Java exception without error message. Author: Davies Liu <davies@databricks.com> Closes #7165 from davies/non_ascii and squashes the following commits: 02cb61a [Davies Liu] fix tests 3b09d31 [Davies Liu] add encoding in header 867754a [Davies Liu] support non-ascii character in column names
* [SPARK-8738] [SQL] [PYSPARK] capture SQL AnalysisException in Python APIDavies Liu2015-06-301-0/+54
Capture the AnalysisException in SQL, hide the long java stack trace, only show the error message. cc rxin Author: Davies Liu <davies@databricks.com> Closes #7135 from davies/ananylis and squashes the following commits: dad7ae7 [Davies Liu] add comment ec0c0e8 [Davies Liu] Update utils.py cdd7edd [Davies Liu] add doc 7b044c2 [Davies Liu] fix python 3 f84d3bd [Davies Liu] capture SQL AnalysisException in Python API