aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/tests.py
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19454][PYTHON][SQL] DataFrame.replace improvementszero3232017-04-051-0/+72
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Allows skipping `value` argument if `to_replace` is a `dict`: ```python df = sc.parallelize([("Alice", 1, 3.0)]).toDF() df.replace({"Alice": "Bob"}).show() ```` - Adds validation step to ensure homogeneous values / replacements. - Simplifies internal control flow. - Improves unit tests coverage. ## How was this patch tested? Existing unit tests, additional unit tests, manual testing. Author: zero323 <zero323@users.noreply.github.com> Closes #16793 from zero323/SPARK-19454.
* [MINOR][DOCS] Match several documentation changes in Scala to R/Pythonhyukjinkwon2017-03-261-0/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to match minor documentations changes in https://github.com/apache/spark/pull/17399 and https://github.com/apache/spark/pull/17380 to R/Python. ## How was this patch tested? Manual tests in Python , Python tests via `./python/run-tests.py --module=pyspark-sql` and lint-checks for Python/R. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17429 from HyukjinKwon/minor-match-doc.
* [SPARK-19876][SS][WIP] OneTime Trigger ExecutorTyson Condie2017-03-231-2/+15
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers. In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature. ## How was this patch tested? A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly. In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests: - The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop). - The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log. - A OneTime trigger execution that results in an exception being thrown. marmbrus tdas zsxwing Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Tyson Condie <tcondie@gmail.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #17219 from tcondie/stream-commit.
* [SPARK-18579][SQL] Use ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace ↵hyukjinkwon2017-03-231-0/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | options in CSV writing ## What changes were proposed in this pull request? This PR proposes to support _not_ trimming the white spaces when writing out. These are `false` by default in CSV reading path but these are `true` by default in CSV writing in univocity parser. Both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` options are not being used for writing and therefore, we are always trimming the white spaces. It seems we should provide a way to keep this white spaces easily. WIth the data below: ```scala val df = spark.read.csv(Seq("a , b , c").toDS) df.show() ``` ``` +---+----+---+ |_c0| _c1|_c2| +---+----+---+ | a | b | c| +---+----+---+ ``` **Before** ```scala df.write.csv("/tmp/text.csv") spark.read.text("/tmp/text.csv").show() ``` ``` +-----+ |value| +-----+ |a,b,c| +-----+ ``` It seems this can't be worked around via `quoteAll` too. ```scala df.write.option("quoteAll", true).csv("/tmp/text.csv") spark.read.text("/tmp/text.csv").show() ``` ``` +-----------+ | value| +-----------+ |"a","b","c"| +-----------+ ``` **After** ```scala df.write.option("ignoreLeadingWhiteSpace", false).option("ignoreTrailingWhiteSpace", false).csv("/tmp/text.csv") spark.read.text("/tmp/text.csv").show() ``` ``` +----------+ | value| +----------+ |a , b , c| +----------+ ``` Note that this case is possible in R ```r > system("cat text.csv") f1,f2,f3 a , b , c > df <- read.csv(file="text.csv") > df f1 f2 f3 1 a b c > write.csv(df, file="text1.csv", quote=F, row.names=F) > system("cat text1.csv") f1,f2,f3 a , b , c ``` ## How was this patch tested? Unit tests in `CSVSuite` and manual tests for Python. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17310 from HyukjinKwon/SPARK-18579.
* [SPARK-12334][SQL][PYSPARK] Support read from multiple input paths for orc ↵Jeff Zhang2017-03-091-0/+5
| | | | | | | | | | | | file in DataFrameReader.orc Beside the issue in spark api, also fix 2 minor issues in pyspark - support read from multiple input paths for orc - support read from multiple input paths for text Author: Jeff Zhang <zjffdu@apache.org> Closes #10307 from zjffdu/SPARK-12334.
* [SPARK-19561][SQL] add int case handling for TimestampTypeJason White2017-03-091-0/+8
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add handling of input of type `Int` for dataType `TimestampType` to `EvaluatePython.scala`. Py4J serializes ints smaller than MIN_INT or larger than MAX_INT to Long, which are handled correctly already, but values between MIN_INT and MAX_INT are serialized to Int. These range limits correspond to roughly half an hour on either side of the epoch. As a result, PySpark doesn't allow TimestampType values to be created in this range. Alternatives attempted: patching the `TimestampType.toInternal` function to cast return values to `long`, so Py4J would always serialize them to Scala Long. Python3 does not have a `long` type, so this approach failed on Python3. ## How was this patch tested? Added a new PySpark-side test that fails without the change. The contribution is my original work and I license the work to the project under the project’s open source license. Resubmission of https://github.com/apache/spark/pull/16896. The original PR didn't go through Jenkins and broke the build. davies dongjoon-hyun cloud-fan Could you kick off a Jenkins run for me? It passed everything for me locally, but it's possible something has changed in the last few weeks. Author: Jason White <jason.white@shopify.com> Closes #17200 from JasonMWhite/SPARK-19561.
* Revert "[SPARK-19561] [PYTHON] cast TimestampType.toInternal output to long"Wenchen Fan2017-03-071-6/+0
| | | | This reverts commit 711addd46e98e42deca97c5b9c0e55fddebaa458.
* [SPARK-19561] [PYTHON] cast TimestampType.toInternal output to longJason White2017-03-071-0/+6
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Cast the output of `TimestampType.toInternal` to long to allow for proper Timestamp creation in DataFrames near the epoch. ## How was this patch tested? Added a new test that fails without the change. dongjoon-hyun davies Mind taking a look? The contribution is my original work and I license the work to the project under the project’s open source license. Author: Jason White <jason.white@shopify.com> Closes #16896 from JasonMWhite/SPARK-19561.
* [SPARK-19701][SQL][PYTHON] Throws a correct exception for 'in' operator ↵hyukjinkwon2017-03-051-0/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | against column ## What changes were proposed in this pull request? This PR proposes to remove incorrect implementation that has been not executed so far (at least from Spark 1.5.2) for `in` operator and throw a correct exception rather than saying it is a bool. I tested the codes above in 1.5.2, 1.6.3, 2.1.0 and in the master branch as below: **1.5.2** ```python >>> df = sqlContext.createDataFrame([[1]]) >>> 1 in df._1 Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark-1.5.2-bin-hadoop2.6/python/pyspark/sql/column.py", line 418, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **1.6.3** ```python >>> 1 in sqlContext.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark-1.6.3-bin-hadoop2.6/python/pyspark/sql/column.py", line 447, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **2.1.0** ```python >>> 1 in spark.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark-2.1.0-bin-hadoop2.7/python/pyspark/sql/column.py", line 426, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **Current Master** ```python >>> 1 in spark.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark/python/pyspark/sql/column.py", line 452, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` **After** ```python >>> 1 in spark.range(1).id Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark/python/pyspark/sql/column.py", line 184, in __contains__ raise ValueError("Cannot apply 'in' operator against a column: please use 'contains' " ValueError: Cannot apply 'in' operator against a column: please use 'contains' in a string column or 'array_contains' function for an array column. ``` In more details, It seems the implementation intended to support this ```python 1 in df.column ``` However, currently, it throws an exception as below: ```python Traceback (most recent call last): File "<stdin>", line 1, in <module> File ".../spark/python/pyspark/sql/column.py", line 426, in __nonzero__ raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', " ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions. ``` What happens here is as below: ```python class Column(object): def __contains__(self, item): print "I am contains" return Column() def __nonzero__(self): raise Exception("I am nonzero.") >>> 1 in Column() I am contains Traceback (most recent call last): File "<stdin>", line 1, in <module> File "<stdin>", line 6, in __nonzero__ Exception: I am nonzero. ``` It seems it calls `__contains__` first and then `__nonzero__` or `__bool__` is being called against `Column()` to make this a bool (or int to be specific). It seems `__nonzero__` (for Python 2), `__bool__` (for Python 3) and `__contains__` forcing the the return into a bool unlike other operators. There are few references about this as below: https://bugs.python.org/issue16011 http://stackoverflow.com/questions/12244074/python-source-code-for-built-in-in-operator/12244378#12244378 http://stackoverflow.com/questions/38542543/functionality-of-python-in-vs-contains/38542777 It seems we can't overwrite `__nonzero__` or `__bool__` as a workaround to make this working because these force the return type as a bool as below: ```python class Column(object): def __contains__(self, item): print "I am contains" return Column() def __nonzero__(self): return "a" >>> 1 in Column() I am contains Traceback (most recent call last): File "<stdin>", line 1, in <module> TypeError: __nonzero__ should return bool or int, returned str ``` ## How was this patch tested? Added unit tests in `tests.py`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17160 from HyukjinKwon/SPARK-19701.
* [SPARK-19610][SQL] Support parsing multiline CSV fileshyukjinkwon2017-02-281-1/+8
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes the support for multiple lines for CSV by resembling the multiline supports in JSON datasource (in case of JSON, per file). So, this PR introduces `wholeFile` option which makes the format not splittable and reads each whole file. Since Univocity parser can produces each row from a stream, it should be capable of parsing very large documents when the internal rows are fix in the memory. ## How was this patch tested? Unit tests in `CSVSuite` and `tests.py` Manual tests with a single 9GB CSV file in local file system, for example, ```scala spark.read.option("wholeFile", true).option("inferSchema", true).csv("tmp.csv").count() ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16976 from HyukjinKwon/SPARK-19610.
* [SPARK-19161][PYTHON][SQL] Improving UDF Docstringszero3232017-02-241-10/+15
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Replaces `UserDefinedFunction` object returned from `udf` with a function wrapper providing docstring and arguments information as proposed in [SPARK-19161](https://issues.apache.org/jira/browse/SPARK-19161). ### Backward incompatible changes: - `pyspark.sql.functions.udf` will return a `function` instead of `UserDefinedFunction`. To ensure backward compatible public API we use function attributes to mimic `UserDefinedFunction` API (`func` and `returnType` attributes). This should have a minimal impact on the user code. An alternative implementation could use dynamical sub-classing. This would ensure full backward compatibility but is more fragile in practice. ### Limitations: Full functionality (retained docstring and argument list) is achieved only in the recent Python version. Legacy Python version will preserve only docstrings, but not argument list. This should be an acceptable trade-off between achieved improvements and overall complexity. ### Possible impact on other tickets: This can affect [SPARK-18777](https://issues.apache.org/jira/browse/SPARK-18777). ## How was this patch tested? Existing unit tests to ensure backward compatibility, additional tests targeting proposed changes. Author: zero323 <zero323@users.noreply.github.com> Closes #16534 from zero323/SPARK-19161.
* [SPARK-19706][PYSPARK] add Column.contains in pysparkWenchen Fan2017-02-231-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? to be consistent with the scala API, we should also add `contains` to `Column` in pyspark. ## How was this patch tested? updated unit test Author: Wenchen Fan <wenchen@databricks.com> Closes #17036 from cloud-fan/pyspark.
* [SPARK-18352][SQL] Support parsing multiline json filesNathan Howell2017-02-161-0/+7
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory. Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired. These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing. I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one. ## How was this patch tested? New and existing unit tests. No performance or load tests have been run. Author: Nathan Howell <nhowell@godaddy.com> Closes #16386 from NathanHowell/SPARK-18352.
* [SPARK-19160][PYTHON][SQL] Add udf decoratorzero3232017-02-151-0/+57
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds `udf` decorator syntax as proposed in [SPARK-19160](https://issues.apache.org/jira/browse/SPARK-19160). This allows users to define UDF using simplified syntax: ```python from pyspark.sql.decorators import udf udf(IntegerType()) def add_one(x): """Adds one""" if x is not None: return x + 1 ``` without need to define a separate function and udf. ## How was this patch tested? Existing unit tests to ensure backward compatibility and additional unit tests covering new functionality. Author: zero323 <zero323@users.noreply.github.com> Closes #16533 from zero323/SPARK-19160.
* [SPARK-18541][PYTHON] Add metadata parameter to pyspark.sql.Column.alias()Sheamus K. Parkes2017-02-141-0/+10
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add a `metadata` keyword parameter to `pyspark.sql.Column.alias()` to allow users to mix-in metadata while manipulating `DataFrame`s in `pyspark`. Without this, I believe it was necessary to pass back through `SparkSession.createDataFrame` each time a user wanted to manipulate `StructField.metadata` in `pyspark`. This pull request also improves consistency between the Scala and Python APIs (i.e. I did not add any functionality that was not already in the Scala API). Discussed ahead of time on JIRA with marmbrus ## How was this patch tested? Added unit tests (and doc tests). Ran the pertinent tests manually. Author: Sheamus K. Parkes <shea.parkes@milliman.com> Closes #16094 from shea-parkes/pyspark-column-alias-metadata.
* [SPARK-19162][PYTHON][SQL] UserDefinedFunction should validate that func is ↵zero3232017-02-141-0/+7
| | | | | | | | | | | | | | | | callable ## What changes were proposed in this pull request? UDF constructor checks if `func` argument is callable and if it is not, fails fast instead of waiting for an action. ## How was this patch tested? Unit tests. Author: zero323 <zero323@users.noreply.github.com> Closes #16535 from zero323/SPARK-19162.
* [SPARK-19429][PYTHON][SQL] Support slice arguments in Column.__getitem__zero3232017-02-131-0/+8
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - Add support for `slice` arguments in `Column.__getitem__`. - Remove obsolete `__getslice__` bindings. ## How was this patch tested? Existing unit tests, additional tests covering `[]` with `slice`. Author: zero323 <zero323@users.noreply.github.com> Closes #16771 from zero323/SPARK-19429.
* [SPARK-19427][PYTHON][SQL] Support data type string as a returnType argument ↵zero3232017-02-131-0/+15
| | | | | | | | | | | | | | | | | | | | | | | of UDF ## What changes were proposed in this pull request? Add support for data type string as a return type argument of `UserDefinedFunction`: ```python f = udf(lambda x: x, "integer") f.returnType ## IntegerType ``` ## How was this patch tested? Existing unit tests, additional unit tests covering new feature. Author: zero323 <zero323@users.noreply.github.com> Closes #16769 from zero323/SPARK-19427.
* [SPARK-16609] Add to_date/to_timestamp with format functionsanabranch2017-02-071-0/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request adds two new user facing functions: - `to_date` which accepts an expression and a format and returns a date. - `to_timestamp` which accepts an expression and a format and returns a timestamp. For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM) ### Date Function *Previously* ``` to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")) ``` *Current* ``` to_date(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Timestamp Function *Previously* ``` unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp") ``` *Current* ``` to_timestamp(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Tasks - [X] Add `to_date` to Scala Functions - [x] Add `to_date` to Python Functions - [x] Add `to_date` to SQL Functions - [X] Add `to_timestamp` to Scala Functions - [x] Add `to_timestamp` to Python Functions - [x] Add `to_timestamp` to SQL Functions - [x] Add function to R ## How was this patch tested? - [x] Add Functions to `DateFunctionsSuite` - Test new `ParseToTimestamp` Expression (*not necessary*) - Test new `ParseToDate` Expression (*not necessary*) - [x] Add test for R - [x] Add test for Python in test.py Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Author: Bill Chambers <bill@databricks.com> Author: anabranch <bill@databricks.com> Closes #16138 from anabranch/SPARK-16609.
* [SPARK-14352][SQL] approxQuantile should support multi columnsZheng RuiFeng2017-02-011-1/+22
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1, add the multi-cols support based on current private api 2, add the multi-cols support to pyspark ## How was this patch tested? unit tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Author: Ruifeng Zheng <ruifengz@foxmail.com> Closes #12135 from zhengruifeng/quantile4multicols.
* [SPARK-19163][PYTHON][SQL] Delay _judf initialization to the __call__zero3232017-01-311-0/+44
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Defer `UserDefinedFunction._judf` initialization to the first call. This prevents unintended `SparkSession` initialization. This allows users to define and import UDF without creating a context / session as a side effect. [SPARK-19163](https://issues.apache.org/jira/browse/SPARK-19163) ## How was this patch tested? Unit tests. Author: zero323 <zero323@users.noreply.github.com> Closes #16536 from zero323/SPARK-19163.
* [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is ↵gatorsmile2017-01-221-4/+4
| | | | | | | | | | | | | | Not Enabled ### What changes were proposed in this pull request? It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables. ### How was this patch tested? Fixed the test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #16587 from gatorsmile/blockHiveTable.
* [SPARK-18589][SQL] Fix Python UDF accessing attributes from both side of joinDavies Liu2017-01-201-0/+9
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? PythonUDF is unevaluable, which can not be used inside a join condition, currently the optimizer will push a PythonUDF which accessing both side of join into the join condition, then the query will fail to plan. This PR fix this issue by checking the expression is evaluable or not before pushing it into Join. ## How was this patch tested? Add a regression test. Author: Davies Liu <davies@databricks.com> Closes #16581 from davies/pyudf_join.
* [SPARK-19223][SQL][PYSPARK] Fix InputFileBlockHolder for datasources which ↵Liang-Chi Hsieh2017-01-181-0/+24
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | are based on HadoopRDD or NewHadoopRDD ## What changes were proposed in this pull request? For some datasources which are based on HadoopRDD or NewHadoopRDD, such as spark-xml, InputFileBlockHolder doesn't work with Python UDF. The method to reproduce it is, running the following codes with `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`: from pyspark.sql.functions import udf,input_file_name from pyspark.sql.types import StringType from pyspark.sql import SparkSession def filename(path): return path session = SparkSession.builder.appName('APP').getOrCreate() session.udf.register('sameText', filename) sameText = udf(filename, StringType()) df = session.read.format('xml').load('a.xml', rowTag='root').select('*', input_file_name().alias('file')) df.select('file').show() # works df.select(sameText(df['file'])).show() # returns empty content The issue is because in `HadoopRDD` and `NewHadoopRDD` we set the file block's info in `InputFileBlockHolder` before the returned iterator begins consuming. `InputFileBlockHolder` will record this info into thread local variable. When running Python UDF in batch, we set up another thread to consume the iterator from child plan's output rdd, so we can't read the info back in another thread. To fix this, we have to set the info in `InputFileBlockHolder` after the iterator begins consuming. So the info can be read in correct thread. ## How was this patch tested? Manual test with above example codes for spark-xml package on pyspark: `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`. Added pyspark test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16585 from viirya/fix-inputfileblock-hadooprdd.
* [SPARK-18687][PYSPARK][SQL] Backward compatibility - creating a Dataframe on ↵Vinayak2017-01-131-1/+6
| | | | | | | | | | | | | | | | | a new SQLContext object fails with a Derby error Change is for SQLContext to reuse the active SparkSession during construction if the sparkContext supplied is the same as the currently active SparkContext. Without this change, a new SparkSession is instantiated that results in a Derby error when attempting to create a dataframe using a new SQLContext object even though the SparkContext supplied to the new SQLContext is same as the currently active one. Refer https://issues.apache.org/jira/browse/SPARK-18687 for details on the error and a repro. Existing unit tests and a new unit test added to pyspark-sql: /python/run-tests --python-executables=python --modules=pyspark-sql Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Vinayak <vijoshi5@in.ibm.com> Author: Vinayak Joshi <vijoshi@users.noreply.github.com> Closes #16119 from vijoshi/SPARK-18687_master.
* [SPARK-19055][SQL][PYSPARK] Fix SparkSession initialization when ↵Liang-Chi Hsieh2017-01-121-0/+23
| | | | | | | | | | | | | | | | | | | | | | SparkContext is stopped ## What changes were proposed in this pull request? In SparkSession initialization, we store created the instance of SparkSession into a class variable _instantiatedContext. Next time we can use SparkSession.builder.getOrCreate() to retrieve the existing SparkSession instance. However, when the active SparkContext is stopped and we create another new SparkContext to use, the existing SparkSession is still associated with the stopped SparkContext. So the operations with this existing SparkSession will be failed. We need to detect such case in SparkSession and renew the class variable _instantiatedContext if needed. ## How was this patch tested? New test added in PySpark. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16454 from viirya/fix-pyspark-sparksession.
* [SPARK-18888] partitionBy in DataStreamWriter in Python throws _to_seq not ↵Burak Yavuz2016-12-151-3/+4
| | | | | | | | | | | | | | | | defined ## What changes were proposed in this pull request? `_to_seq` wasn't imported. ## How was this patch tested? Added partitionBy to existing write path unit test Author: Burak Yavuz <brkyvz@gmail.com> Closes #16297 from brkyvz/SPARK-18888.
* [SPARK-18852][SS] StreamingQuery.lastProgress should be null when ↵Shixiong Zhu2016-12-141-1/+17
| | | | | | | | | | | | | | | | | | recentProgress is empty ## What changes were proposed in this pull request? Right now `StreamingQuery.lastProgress` throws NoSuchElementException and it's hard to be used in Python since Python user will just see Py4jError. This PR just makes it return null instead. ## How was this patch tested? `test("lastProgress should be null when recentProgress is empty")` Author: Shixiong Zhu <shixiong@databricks.com> Closes #16273 from zsxwing/SPARK-18852.
* [SPARK-18766][SQL] Push Down Filter Through BatchEvalPython (Python UDF)gatorsmile2016-12-101-0/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? Currently, when users use Python UDF in Filter, BatchEvalPython is always generated below FilterExec. However, not all the predicates need to be evaluated after Python UDF execution. Thus, this PR is to push down the determinisitc predicates through `BatchEvalPython`. ```Python >>> df = spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"]) >>> from pyspark.sql.functions import udf, col >>> from pyspark.sql.types import BooleanType >>> my_filter = udf(lambda a: a < 2, BooleanType()) >>> sel = df.select(col("key"), col("value")).filter((my_filter(col("key"))) & (df.value < "2")) >>> sel.explain(True) ``` Before the fix, the plan looks like ``` == Optimized Logical Plan == Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2)) +- LogicalRDD [key#0L, value#1] == Physical Plan == *Project [key#0L, value#1] +- *Filter ((isnotnull(value#1) && pythonUDF0#9) && (value#1 < 2)) +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9] +- Scan ExistingRDD[key#0L,value#1] ``` After the fix, the plan looks like ``` == Optimized Logical Plan == Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2)) +- LogicalRDD [key#0L, value#1] == Physical Plan == *Project [key#0L, value#1] +- *Filter pythonUDF0#9: boolean +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9] +- *Filter (isnotnull(value#1) && (value#1 < 2)) +- Scan ExistingRDD[key#0L,value#1] ``` ### How was this patch tested? Added both unit test cases for `BatchEvalPythonExec` and also add an end-to-end test case in Python test suite. Author: gatorsmile <gatorsmile@gmail.com> Closes #16193 from gatorsmile/pythonUDFPredicatePushDown.
* [SPARK-18667][PYSPARK][SQL] Change the way to group row in ↵Liang-Chi Hsieh2016-12-081-0/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | BatchEvalPythonExec so input_file_name function can work with UDF in pyspark ## What changes were proposed in this pull request? `input_file_name` doesn't return filename when working with UDF in PySpark. An example shows the problem: from pyspark.sql.functions import * from pyspark.sql.types import * def filename(path): return path sourceFile = udf(filename, StringType()) spark.read.json("tmp.json").select(sourceFile(input_file_name())).show() +---------------------------+ |filename(input_file_name())| +---------------------------+ | | +---------------------------+ The cause of this issue is, we group rows in `BatchEvalPythonExec` for batching processing of PythonUDF. Currently we group rows first and then evaluate expressions on the rows. If the data is less than the required number of rows for a group, the iterator will be consumed to the end before the evaluation. However, once the iterator reaches the end, we will unset input filename. So the input_file_name expression can't return correct filename. This patch fixes the approach to group the batch of rows. We evaluate the expression first and then group evaluated results to batch. ## How was this patch tested? Added unit test to PySpark. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16115 from viirya/fix-py-udf-input-filename.
* [SPARK-18754][SS] Rename recentProgresses to recentProgressMichael Armbrust2016-12-071-2/+2
| | | | | | | | Based on an informal survey, users find this option easier to understand / remember. Author: Michael Armbrust <michael@databricks.com> Closes #16182 from marmbrus/renameRecentProgress.
* [SPARK-18634][PYSPARK][SQL] Corruption and Correctness issues with exploding ↵Liang-Chi Hsieh2016-12-051-0/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Python UDFs ## What changes were proposed in this pull request? As reported in the Jira, there are some weird issues with exploding Python UDFs in SparkSQL. The following test code can reproduce it. Notice: the following test code is reported to return wrong results in the Jira. However, as I tested on master branch, it causes exception and so can't return any result. >>> from pyspark.sql.functions import * >>> from pyspark.sql.types import * >>> >>> df = spark.range(10) >>> >>> def return_range(value): ... return [(i, str(i)) for i in range(value - 1, value + 1)] ... >>> range_udf = udf(return_range, ArrayType(StructType([StructField("integer_val", IntegerType()), ... StructField("string_val", StringType())]))) >>> >>> df.select("id", explode(range_udf(df.id))).show() Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/spark/python/pyspark/sql/dataframe.py", line 318, in show print(self._jdf.showString(n, 20)) File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 1133, in __call__ File "/spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py", line 319, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o126.showString.: java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:156) at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:120) at org.apache.spark.sql.execution.GenerateExec.consume(GenerateExec.scala:57) The cause of this issue is, in `ExtractPythonUDFs` we insert `BatchEvalPythonExec` to run PythonUDFs in batch. `BatchEvalPythonExec` will add extra outputs (e.g., `pythonUDF0`) to original plan. In above case, the original `Range` only has one output `id`. After `ExtractPythonUDFs`, the added `BatchEvalPythonExec` has two outputs `id` and `pythonUDF0`. Because the output of `GenerateExec` is given after analysis phase, in above case, it is the combination of `id`, i.e., the output of `Range`, and `col`. But in planning phase, we change `GenerateExec`'s child plan to `BatchEvalPythonExec` with additional output attributes. It will cause no problem in non wholestage codegen. Because when evaluating the additional attributes are projected out the final output of `GenerateExec`. However, as `GenerateExec` now supports wholestage codegen, the framework will input all the outputs of the child plan to `GenerateExec`. Then when consuming `GenerateExec`'s output data (i.e., calling `consume`), the number of output attributes is different to the output variables in wholestage codegen. To solve this issue, this patch only gives the generator's output to `GenerateExec` after analysis phase. `GenerateExec`'s output is the combination of its child plan's output and the generator's output. So when we change `GenerateExec`'s child, its output is still correct. ## How was this patch tested? Added test cases to PySpark. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16120 from viirya/fix-py-udf-with-generator.
* [SPARK-18694][SS] Add StreamingQuery.explain and exception to Python and fix ↵Shixiong Zhu2016-12-051-0/+29
| | | | | | | | | | | | | | | | | StreamingQueryException ## What changes were proposed in this pull request? - Add StreamingQuery.explain and exception to Python. - Fix StreamingQueryException to not expose `OffsetSeq`. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16125 from zsxwing/py-streaming-explain.
* [SPARK-18690][PYTHON][SQL] Backward compatibility of unbounded frameszero3232016-12-021-0/+35
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Makes `Window.unboundedPreceding` and `Window.unboundedFollowing` backward compatible. ## How was this patch tested? Pyspark SQL unittests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: zero323 <zero323@users.noreply.github.com> Closes #16123 from zero323/SPARK-17845-follow-up.
* [SPARK-18516][STRUCTURED STREAMING] Follow up PR to add ↵Tathagata Das2016-11-291-0/+5
| | | | | | | | | | | | | | | | | StreamingQuery.status to Python ## What changes were proposed in this pull request? - Add StreamingQueryStatus.json - Make it not case class (to avoid unnecessarily exposing implicit object StreamingQueryStatus, consistent with StreamingQueryProgress) - Add StreamingQuery.status to Python - Fix post-termination status ## How was this patch tested? New unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16075 from tdas/SPARK-18516-1.
* [SPARK-18516][SQL] Split state and progress in streamingTathagata Das2016-11-291-0/+22
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
* [SPARK-17946][PYSPARK] Python crossJoin API similar to ScalaSrinath Shankar2016-10-141-1/+14
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add a crossJoin function to the DataFrame API similar to that in Scala. Joins with no condition (cartesian products) must be specified with the crossJoin API ## How was this patch tested? Added python tests to ensure that an AnalysisException if a cartesian product is specified without crossJoin(), and that cartesian products can execute if specified via crossJoin() (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. Author: Srinath Shankar <srinath@databricks.com> Closes #15493 from srinathshankar/crosspython.
* [SPARK-17845] [SQL] More self-evident window function frame boundary APIReynold Xin2016-10-121-1/+24
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch improves the window function frame boundary API to make it more obvious to read and to use. The two high level changes are: 1. Create Window.currentRow, Window.unboundedPreceding, Window.unboundedFollowing to indicate the special values in frame boundaries. These methods map to the special integral values so we are not breaking backward compatibility here. This change makes the frame boundaries more self-evident (instead of Long.MinValue, it becomes Window.unboundedPreceding). 2. In Python, for any value less than or equal to JVM's Long.MinValue, treat it as Window.unboundedPreceding. For any value larger than or equal to JVM's Long.MaxValue, treat it as Window.unboundedFollowing. Before this change, if the user specifies any value that is less than Long.MinValue but not -sys.maxsize (e.g. -sys.maxsize + 1), the number we pass over to the JVM would overflow, resulting in a frame that does not make sense. Code example required to specify a frame before this patch: ``` Window.rowsBetween(-Long.MinValue, 0) ``` While the above code should still work, the new way is more obvious to read: ``` Window.rowsBetween(Window.unboundedPreceding, Window.currentRow) ``` ## How was this patch tested? - Updated DataFrameWindowSuite (for Scala/Java) - Updated test_window_functions_cumulative_sum (for Python) - Renamed DataFrameWindowSuite DataFrameWindowFunctionsSuite to better reflect its purpose Author: Reynold Xin <rxin@databricks.com> Closes #15438 from rxin/SPARK-17845.
* [SPARK-14761][SQL] Reject invalid join methods when join columns are not ↵Bijay Pathak2016-10-121-0/+6
| | | | | | | | | | | | | | | | | | | | specified in PySpark DataFrame join. ## What changes were proposed in this pull request? In PySpark, the invalid join type will not throw error for the following join: ```df1.join(df2, how='not-a-valid-join-type')``` The signature of the join is: ```def join(self, other, on=None, how=None):``` The existing code completely ignores the `how` parameter when `on` is `None`. This patch will process the arguments passed to join and pass in to JVM Spark SQL Analyzer, which will validate the join type passed. ## How was this patch tested? Used manual and existing test suites. Author: Bijay Pathak <bkpathak@mtu.edu> Closes #15409 from bkpathak/SPARK-14761.
* [SPARK-17808][PYSPARK] Upgraded version of Pyrolite to 4.13Bryan Cutler2016-10-111-0/+8
| | | | | | | | | | | | ## What changes were proposed in this pull request? Upgraded to a newer version of Pyrolite which supports serialization of a BinaryType StructField for PySpark.SQL ## How was this patch tested? Added a unit test which fails with a raised ValueError when using the previous version of Pyrolite 4.9 and Python3 Author: Bryan Cutler <cutlerb@gmail.com> Closes #15386 from BryanCutler/pyrolite-upgrade-SPARK-17808.
* [SPARK-17844] Simplify DataFrame API for defining frame boundaries in window ↵Reynold Xin2016-10-101-0/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | functions ## What changes were proposed in this pull request? When I was creating the example code for SPARK-10496, I realized it was pretty convoluted to define the frame boundaries for window functions when there is no partition column or ordering column. The reason is that we don't provide a way to create a WindowSpec directly with the frame boundaries. We can trivially improve this by adding rowsBetween and rangeBetween to Window object. As an example, to compute cumulative sum using the natural ordering, before this pr: ``` df.select('key, sum("value").over(Window.partitionBy(lit(1)).rowsBetween(Long.MinValue, 0))) ``` After this pr: ``` df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0))) ``` Note that you could argue there is no point specifying a window frame without partitionBy/orderBy -- but it is strange that only rowsBetween and rangeBetween are not the only two APIs not available. This also fixes https://issues.apache.org/jira/browse/SPARK-17656 (removing _root_.scala). ## How was this patch tested? Added test cases to compute cumulative sum in DataFrameWindowSuite for Scala/Java and tests.py for Python. Author: Reynold Xin <rxin@databricks.com> Closes #15412 from rxin/SPARK-17844.
* [SPARK-17805][PYSPARK] Fix in sqlContext.read.text when pass in list of pathsBryan Cutler2016-10-071-0/+6
| | | | | | | | | | | | ## What changes were proposed in this pull request? If given a list of paths, `pyspark.sql.readwriter.text` will attempt to use an undefined variable `paths`. This change checks if the param `paths` is a basestring and then converts it to a list, so that the same variable `paths` can be used for both cases ## How was this patch tested? Added unit test for reading list of files Author: Bryan Cutler <cutlerb@gmail.com> Closes #15379 from BryanCutler/sql-readtext-paths-SPARK-17805.
* [SPARK-17100] [SQL] fix Python udf in filter on top of outer joinDavies Liu2016-09-191-0/+8
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? In optimizer, we try to evaluate the condition to see whether it's nullable or not, but some expressions are not evaluable, we should check that before evaluate it. ## How was this patch tested? Added regression tests. Author: Davies Liu <davies@databricks.com> Closes #15103 from davies/udf_join.
* [SPARK-17514] df.take(1) and df.limit(1).collect() should perform the same ↵Josh Rosen2016-09-141-0/+18
| | | | | | | | | | | | | | | | | | | | in Python ## What changes were proposed in this pull request? In PySpark, `df.take(1)` runs a single-stage job which computes only one partition of the DataFrame, while `df.limit(1).collect()` computes all partitions and runs a two-stage job. This difference in performance is confusing. The reason why `limit(1).collect()` is so much slower is that `collect()` internally maps to `df.rdd.<some-pyspark-conversions>.toLocalIterator`, which causes Spark SQL to build a query where a global limit appears in the middle of the plan; this, in turn, ends up being executed inefficiently because limits in the middle of plans are now implemented by repartitioning to a single task rather than by running a `take()` job on the driver (this was done in #7334, a patch which was a prerequisite to allowing partition-local limits to be pushed beneath unions, etc.). In order to fix this performance problem I think that we should generalize the fix from SPARK-10731 / #8876 so that `DataFrame.collect()` also delegates to the Scala implementation and shares the same performance properties. This patch modifies `DataFrame.collect()` to first collect all results to the driver and then pass them to Python, allowing this query to be planned using Spark's `CollectLimit` optimizations. ## How was this patch tested? Added a regression test in `sql/tests.py` which asserts that the expected number of jobs, stages, and tasks are run for both queries. Author: Josh Rosen <joshrosen@databricks.com> Closes #15068 from JoshRosen/pyspark-collect-limit.
* [SPARK-17474] [SQL] fix python udf in TakeOrderedAndProjectExecDavies Liu2016-09-121-0/+8
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? When there is any Python UDF in the Project between Sort and Limit, it will be collected into TakeOrderedAndProjectExec, ExtractPythonUDFs failed to pull the Python UDFs out because QueryPlan.expressions does not include the expression inside Option[Seq[Expression]]. Ideally, we should fix the `QueryPlan.expressions`, but tried with no luck (it always run into infinite loop). In PR, I changed the TakeOrderedAndProjectExec to no use Option[Seq[Expression]] to workaround it. cc JoshRosen ## How was this patch tested? Added regression test. Author: Davies Liu <davies@databricks.com> Closes #15030 from davies/all_expr.
* [SPARK-17215][SQL] Method `SQLContext.parseDataType(dataTypeString: String)` ↵jiangxingbo2016-08-241-1/+1
| | | | | | | | | | | | | | | | | could be removed. ## What changes were proposed in this pull request? Method `SQLContext.parseDataType(dataTypeString: String)` could be removed, we should use `SparkSession.parseDataType(dataTypeString: String)` instead. This require updating PySpark. ## How was this patch tested? Existing test cases. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #14790 from jiangxb1987/parseDataType.
* [SPARK-17035] [SQL] [PYSPARK] Improve Timestamp not to lose precision for ↵Dongjoon Hyun2016-08-161-0/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | all cases ## What changes were proposed in this pull request? `PySpark` loses `microsecond` precision for some corner cases during converting `Timestamp` into `Long`. For example, for the following `datetime.max` value should be converted a value whose last 6 digits are '999999'. This PR improves the logic not to lose precision for all cases. **Corner case** ```python >>> datetime.datetime.max datetime.datetime(9999, 12, 31, 23, 59, 59, 999999) ``` **Before** ```python >>> from datetime import datetime >>> from pyspark.sql import Row >>> from pyspark.sql.types import StructType, StructField, TimestampType >>> schema = StructType([StructField("dt", TimestampType(), False)]) >>> [schema.toInternal(row) for row in [{"dt": datetime.max}]] [(253402329600000000,)] ``` **After** ```python >>> [schema.toInternal(row) for row in [{"dt": datetime.max}]] [(253402329599999999,)] ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14631 from dongjoon-hyun/SPARK-17035.
* [SPARK-16700][PYSPARK][SQL] create DataFrame from dict/Row with schemaDavies Liu2016-08-151-0/+16
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In 2.0, we verify the data type against schema for every row for safety, but with performance cost, this PR make it optional. When we verify the data type for StructType, it does not support all the types we support in infer schema (for example, dict), this PR fix that to make them consistent. For Row object which is created using named arguments, the order of fields are sorted by name, they may be not different than the order in provided schema, this PR fix that by ignore the order of fields in this case. ## How was this patch tested? Created regression tests for them. Author: Davies Liu <davies@databricks.com> Closes #14469 from davies/py_dict.
* [SPARK-16062] [SPARK-15989] [SQL] Fix two bugs of Python-only UDTsLiang-Chi Hsieh2016-08-021-0/+35
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? There are two related bugs of Python-only UDTs. Because the test case of second one needs the first fix too. I put them into one PR. If it is not appropriate, please let me know. ### First bug: When MapObjects works on Python-only UDTs `RowEncoder` will use `PythonUserDefinedType.sqlType` for its deserializer expression. If the sql type is `ArrayType`, we will have `MapObjects` working on it. But `MapObjects` doesn't consider `PythonUserDefinedType` as its input data type. It causes error like: import pyspark.sql.group from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT from pyspark.sql.types import * schema = StructType().add("key", LongType()).add("val", PythonOnlyUDT()) df = spark.createDataFrame([(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], schema=schema) df.show() File "/home/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 o36.showString. : java.lang.RuntimeException: Error while decoding: scala.MatchError: org.apache.spark.sql.types.PythonUserDefinedTypef4ceede8 (of class org.apache.spark.sql.types.PythonUserDefinedType) ... ### Second bug: When Python-only UDTs is the element type of ArrayType import pyspark.sql.group from pyspark.sql.tests import PythonOnlyPoint, PythonOnlyUDT from pyspark.sql.types import * schema = StructType().add("key", LongType()).add("val", ArrayType(PythonOnlyUDT())) df = spark.createDataFrame([(i % 3, [PythonOnlyPoint(float(i), float(i))]) for i in range(10)], schema=schema) df.show() ## How was this patch tested? PySpark's sql tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #13778 from viirya/fix-pyudt.
* [SPARK-16175] [PYSPARK] handle None for UDTDavies Liu2016-06-281-0/+11
| | | | | | | | | | | | | | ## 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.