Commit graph

15041 commits

Author SHA1 Message Date
Kousuke Saruta 6eba655259 [SPARK-12404][SQL] Ensure objects passed to StaticInvoke is Serializable
Now `StaticInvoke` receives `Any` as a object and `StaticInvoke` can be serialized but sometimes the object passed is not serializable.

For example, following code raises Exception because `RowEncoder#extractorsFor` invoked indirectly makes `StaticInvoke`.

```
case class TimestampContainer(timestamp: java.sql.Timestamp)
val rdd = sc.parallelize(1 to 2).map(_ => TimestampContainer(System.currentTimeMillis))
val df = rdd.toDF
val ds = df.as[TimestampContainer]
val rdd2 = ds.rdd                                 <----------------- invokes extractorsFor indirectory
```

I'll add test cases.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Author: Michael Armbrust <michael@databricks.com>

Closes #10357 from sarutak/SPARK-12404.
2015-12-18 14:05:06 -08:00
Yin Huai 41ee7c57ab [SPARK-12218][SQL] Invalid splitting of nested AND expressions in Data Source filter API
JIRA: https://issues.apache.org/jira/browse/SPARK-12218

When creating filters for Parquet/ORC, we should not push nested AND expressions partially.

Author: Yin Huai <yhuai@databricks.com>

Closes #10362 from yhuai/SPARK-12218.
2015-12-18 10:53:13 -08:00
Davies Liu 4af647c77d [SPARK-12054] [SQL] Consider nullability of expression in codegen
This could simplify the generated code for expressions that is not nullable.

This PR fix lots of bugs about nullability.

Author: Davies Liu <davies@databricks.com>

Closes #10333 from davies/skip_nullable.
2015-12-18 10:09:17 -08:00
Dilip Biswal ee444fe4b8 [SPARK-11619][SQL] cannot use UDTF in DataFrame.selectExpr
Description of the problem from cloud-fan

Actually this line: https://github.com/apache/spark/blob/branch-1.5/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala#L689
When we use `selectExpr`, we pass in `UnresolvedFunction` to `DataFrame.select` and fall in the last case. A workaround is to do special handling for UDTF like we did for `explode`(and `json_tuple` in 1.6), wrap it with `MultiAlias`.
Another workaround is using `expr`, for example, `df.select(expr("explode(a)").as(Nil))`, I think `selectExpr` is no longer needed after we have the `expr` function....

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #9981 from dilipbiswal/spark-11619.
2015-12-18 09:54:30 -08:00
Marcelo Vanzin 2782818287 [SPARK-12350][CORE] Don't log errors when requested stream is not found.
If a client requests a non-existent stream, just send a failure message
back, without logging any error on the server side (since it's not a
server error).

On the executor side, avoid error logs by translating any errors during
transfer to a `ClassNotFoundException`, so that loading the class is
retried on a the parent class loader. This can mask IO errors during
transmission, but the most common cause is that the class is not
served by the remote end.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10337 from vanzin/SPARK-12350.
2015-12-18 09:49:08 -08:00
Jeff L ea59b0f3a6 [SPARK-9057][STREAMING] Twitter example joining to static RDD of word sentiment values
Example of joining a static RDD of word sentiments to a streaming RDD of Tweets in order to demo the usage of the transform() method.

Author: Jeff L <sha0lin@alumni.carnegiemellon.edu>

Closes #8431 from Agent007/SPARK-9057.
2015-12-18 15:06:54 +00:00
Michael Gummelt 2bebaa39d9 [SPARK-12413] Fix Mesos ZK persistence
I believe this fixes SPARK-12413.  I'm currently running an integration test to verify.

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes #10366 from mgummelt/fix-zk-mesos.
2015-12-18 20:18:00 +09:00
Jeff Zhang 40e52a27c7 [CORE][TESTS] minor fix of JavaSerializerSuite
Not jira is created.
The original test is passed because the class cast is lazy (only when the object's method is invoked).

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10371 from zjffdu/minor_fix.
2015-12-18 00:49:56 -08:00
Shixiong Zhu 0370abdfd6 [MINOR] Hide the error logs for 'SQLListenerMemoryLeakSuite'
Hide the error logs for 'SQLListenerMemoryLeakSuite' to avoid noises. Most of changes are space changes.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10363 from zsxwing/hide-log.
2015-12-17 18:18:12 -08:00
jhu-chang f4346f612b [SPARK-11749][STREAMING] Duplicate creating the RDD in file stream when recovering from checkpoint data
Add a transient flag `DStream.restoredFromCheckpointData` to control the restore processing in DStream to avoid duplicate works:  check this flag first in `DStream.restoreCheckpointData`, only when `false`, the restore process will be executed.

Author: jhu-chang <gt.hu.chang@gmail.com>

Closes #9765 from jhu-chang/SPARK-11749.
2015-12-17 17:53:15 -08:00
Herman van Hovell 658f66e620 [SPARK-8641][SQL] Native Spark Window functions
This PR removes Hive windows functions from Spark and replaces them with (native) Spark ones. The PR is on par with Hive in terms of features.

This has the following advantages:
* Better memory management.
* The ability to use spark UDAFs in Window functions.

cc rxin / yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #9819 from hvanhovell/SPARK-8641-2.
2015-12-17 15:16:35 -08:00
Evan Chen ed6ebda5c8 [SPARK-12376][TESTS] Spark Streaming Java8APISuite fails in assertOrderInvariantEquals method
org.apache.spark.streaming.Java8APISuite.java is failing due to trying to sort immutable list in assertOrderInvariantEquals method.

Author: Evan Chen <chene@us.ibm.com>

Closes #10336 from evanyc15/SPARK-12376-StreamingJavaAPISuite.
2015-12-17 14:22:30 -08:00
Reynold Xin e096a652b9 [SPARK-12397][SQL] Improve error messages for data sources when they are not found
Point users to spark-packages.org to find them.

Author: Reynold Xin <rxin@databricks.com>

Closes #10351 from rxin/SPARK-12397.
2015-12-17 14:16:49 -08:00
Shixiong Zhu 540b5aeadc [SPARK-12410][STREAMING] Fix places that use '.' and '|' directly in split
String.split accepts a regular expression, so we should escape "." and "|".

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10361 from zsxwing/reg-bug.
2015-12-17 13:23:48 -08:00
Iulian Dragos 8184568810 [SPARK-12345][MESOS] Properly filter out SPARK_HOME in the Mesos REST server
Fix problem with #10332, this one should fix Cluster mode on Mesos

Author: Iulian Dragos <jaguarul@gmail.com>

Closes #10359 from dragos/issue/fix-spark-12345-one-more-time.
2015-12-18 03:19:31 +09:00
Shixiong Zhu 86e405f357 [SPARK-12220][CORE] Make Utils.fetchFile support files that contain special characters
This PR encodes and decodes the file name to fix the issue.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10208 from zsxwing/uri.
2015-12-17 09:55:37 -08:00
Yanbo Liang 6e0771665b [SQL] Update SQLContext.read.text doc
Since we rename the column name from ```text``` to ```value``` for DataFrame load by ```SQLContext.read.text```, we need to update doc.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10349 from yanboliang/text-value.
2015-12-17 09:19:46 -08:00
Davies Liu a170d34a1b [SPARK-12395] [SQL] fix resulting columns of outer join
For API DataFrame.join(right, usingColumns, joinType), if the joinType is right_outer or full_outer, the resulting join columns could be wrong (will be null).

The order of columns had been changed to match that with MySQL and PostgreSQL [1].

This PR also fix the nullability of output for outer join.

[1] http://www.postgresql.org/docs/9.2/static/queries-table-expressions.html

Author: Davies Liu <davies@databricks.com>

Closes #10353 from davies/fix_join.
2015-12-17 08:04:11 -08:00
Davies Liu cd3d937b0c Revert "Once driver register successfully, stop it to connect to master."
This reverts commit 5a514b61bb.
2015-12-17 08:01:27 -08:00
echo2mei 5a514b61bb Once driver register successfully, stop it to connect to master.
This commit is to resolve SPARK-12396.

Author: echo2mei <534384876@qq.com>

Closes #10354 from echoTomei/master.
2015-12-17 07:59:17 -08:00
Yin Huai 9d66c4216a [SPARK-12057][SQL] Prevent failure on corrupt JSON records
This PR makes JSON parser and schema inference handle more cases where we have unparsed records. It is based on #10043. The last commit fixes the failed test and updates the logic of schema inference.

Regarding the schema inference change, if we have something like
```
{"f1":1}
[1,2,3]
```
originally, we will get a DF without any column.
After this change, we will get a DF with columns `f1` and `_corrupt_record`. Basically, for the second row, `[1,2,3]` will be the value of `_corrupt_record`.

When merge this PR, please make sure that the author is simplyianm.

JIRA: https://issues.apache.org/jira/browse/SPARK-12057

Closes #10043

Author: Ian Macalinao <me@ian.pw>
Author: Yin Huai <yhuai@databricks.com>

Closes #10288 from yhuai/handleCorruptJson.
2015-12-16 23:18:53 -08:00
David Tolpin 437583f692 [SPARK-11904][PYSPARK] reduceByKeyAndWindow does not require checkpointing when invFunc is None
when invFunc is None, `reduceByKeyAndWindow(func, None, winsize, slidesize)` is equivalent to

     reduceByKey(func).window(winsize, slidesize).reduceByKey(winsize, slidesize)

and no checkpoint is necessary. The corresponding Scala code does exactly that, but Python code always creates a windowed stream with obligatory checkpointing. The patch fixes this.

I do not know how to unit-test this.

Author: David Tolpin <david.tolpin@gmail.com>

Closes #9888 from dtolpin/master.
2015-12-16 22:10:24 -08:00
Andrew Or 97678edeaa [SPARK-12390] Clean up unused serializer parameter in BlockManager
No change in functionality is intended. This only changes internal API.

Author: Andrew Or <andrew@databricks.com>

Closes #10343 from andrewor14/clean-bm-serializer.
2015-12-16 20:01:47 -08:00
Marcelo Vanzin d1508dd9b7 [SPARK-12386][CORE] Fix NPE when spark.executor.port is set.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10339 from vanzin/SPARK-12386.
2015-12-16 19:47:49 -08:00
Rohit Agarwal fdb3822756 [SPARK-12186][WEB UI] Send the complete request URI including the query string when redirecting.
Author: Rohit Agarwal <rohita@qubole.com>

Closes #10180 from mindprince/SPARK-12186.
2015-12-16 19:04:33 -08:00
tedyu f590178d7a [SPARK-12365][CORE] Use ShutdownHookManager where Runtime.getRuntime.addShutdownHook() is called
SPARK-9886 fixed ExternalBlockStore.scala

This PR fixes the remaining references to Runtime.getRuntime.addShutdownHook()

Author: tedyu <yuzhihong@gmail.com>

Closes #10325 from ted-yu/master.
2015-12-16 19:02:12 -08:00
Imran Rashid 38d9795a4f [SPARK-10248][CORE] track exceptions in dagscheduler event loop in tests
`DAGSchedulerEventLoop` normally only logs errors (so it can continue to process more events, from other jobs).  However, this is not desirable in the tests -- the tests should be able to easily detect any exception, and also shouldn't silently succeed if there is an exception.

This was suggested by mateiz on https://github.com/apache/spark/pull/7699.  It may have already turned up an issue in "zero split job".

Author: Imran Rashid <irashid@cloudera.com>

Closes #8466 from squito/SPARK-10248.
2015-12-16 19:01:05 -08:00
Andrew Or ce5fd4008e MAINTENANCE: Automated closing of pull requests.
This commit exists to close the following pull requests on Github:

Closes #1217 (requested by ankurdave, srowen)
Closes #4650 (requested by andrewor14)
Closes #5307 (requested by vanzin)
Closes #5664 (requested by andrewor14)
Closes #5713 (requested by marmbrus)
Closes #5722 (requested by andrewor14)
Closes #6685 (requested by srowen)
Closes #7074 (requested by srowen)
Closes #7119 (requested by andrewor14)
Closes #7997 (requested by jkbradley)
Closes #8292 (requested by srowen)
Closes #8975 (requested by andrewor14, vanzin)
Closes #8980 (requested by andrewor14, davies)
2015-12-16 17:05:57 -08:00
Andrew Or 861549acdb [MINOR] Add missing interpolation in NettyRPCEnv
```
Exception in thread "main" org.apache.spark.rpc.RpcTimeoutException:
Cannot receive any reply in ${timeout.duration}. This timeout is controlled by spark.rpc.askTimeout
	at org.apache.spark.rpc.RpcTimeout.org$apache$spark$rpc$RpcTimeout$$createRpcTimeoutException(RpcTimeout.scala:48)
	at org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:63)
	at org.apache.spark.rpc.RpcTimeout$$anonfun$addMessageIfTimeout$1.applyOrElse(RpcTimeout.scala:59)
	at scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:33)
```

Author: Andrew Or <andrew@databricks.com>

Closes #10334 from andrewor14/rpc-typo.
2015-12-16 16:13:48 -08:00
Davies Liu 27b98e99d2 [SPARK-12380] [PYSPARK] use SQLContext.getOrCreate in mllib
MLlib should use SQLContext.getOrCreate() instead of creating new SQLContext.

Author: Davies Liu <davies@databricks.com>

Closes #10338 from davies/create_context.
2015-12-16 15:48:11 -08:00
Martin Menestret 3a44aebd0c [SPARK-9690][ML][PYTHON] pyspark CrossValidator random seed
Extend CrossValidator with HasSeed in PySpark.

This PR replaces [https://github.com/apache/spark/pull/7997]

CC: yanboliang thunterdb mmenestret  Would one of you mind taking a look?  Thanks!

Author: Joseph K. Bradley <joseph@databricks.com>
Author: Martin MENESTRET <mmenestret@ippon.fr>

Closes #10268 from jkbradley/pyspark-cv-seed.
2015-12-16 14:05:35 -08:00
hyukjinkwon 9657ee8788 [SPARK-11677][SQL] ORC filter tests all pass if filters are actually not pushed down.
Currently ORC filters are not tested properly. All the tests pass even if the filters are not pushed down or disabled. In this PR, I add some logics for this.
Since ORC does not filter record by record fully, this checks the count of the result and if it contains the expected values.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9687 from HyukjinKwon/SPARK-11677.
2015-12-16 13:24:49 -08:00
gatorsmile edf65cd961 [SPARK-12164][SQL] Decode the encoded values and then display
Based on the suggestions from marmbrus cloud-fan in https://github.com/apache/spark/pull/10165 , this PR is to print the decoded values(user objects) in `Dataset.show`
```scala
    implicit val kryoEncoder = Encoders.kryo[KryoClassData]
    val ds = Seq(KryoClassData("a", 1), KryoClassData("b", 2), KryoClassData("c", 3)).toDS()
    ds.show(20, false);
```
The current output is like
```
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|value                                                                                                                                                                                 |
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
|[1, 0, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 115, 112, 97, 114, 107, 46, 115, 113, 108, 46, 75, 114, 121, 111, 67, 108, 97, 115, 115, 68, 97, 116, -31, 1, 1, -126, 97, 2]|
|[1, 0, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 115, 112, 97, 114, 107, 46, 115, 113, 108, 46, 75, 114, 121, 111, 67, 108, 97, 115, 115, 68, 97, 116, -31, 1, 1, -126, 98, 4]|
|[1, 0, 111, 114, 103, 46, 97, 112, 97, 99, 104, 101, 46, 115, 112, 97, 114, 107, 46, 115, 113, 108, 46, 75, 114, 121, 111, 67, 108, 97, 115, 115, 68, 97, 116, -31, 1, 1, -126, 99, 6]|
+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
```
After the fix, it will be like the below if and only if the users override the `toString` function in the class `KryoClassData`
```scala
override def toString: String = s"KryoClassData($a, $b)"
```
```
+-------------------+
|value              |
+-------------------+
|KryoClassData(a, 1)|
|KryoClassData(b, 2)|
|KryoClassData(c, 3)|
+-------------------+
```

If users do not override the `toString` function, the results will be like
```
+---------------------------------------+
|value                                  |
+---------------------------------------+
|org.apache.spark.sql.KryoClassData68ef|
|org.apache.spark.sql.KryoClassData6915|
|org.apache.spark.sql.KryoClassData693b|
+---------------------------------------+
```

Question: Should we add another optional parameter in the function `show`? It will decide if the function `show` will display the hex values or the object values?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10215 from gatorsmile/showDecodedValue.
2015-12-16 13:22:34 -08:00
Wenchen Fan a783a8ed49 [SPARK-12320][SQL] throw exception if the number of fields does not line up for Tuple encoder
Author: Wenchen Fan <wenchen@databricks.com>

Closes #10293 from cloud-fan/err-msg.
2015-12-16 13:20:12 -08:00
Yanbo Liang 1a8b2a17db [SPARK-12364][ML][SPARKR] Add ML example for SparkR
We have DataFrame example for SparkR, we also need to add ML example under ```examples/src/main/r```.

cc mengxr jkbradley shivaram

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10324 from yanboliang/spark-12364.
2015-12-16 12:59:22 -08:00
Joseph K. Bradley 8148cc7a5c [SPARK-11608][MLLIB][DOC] Added migration guide for MLlib 1.6
No known breaking changes, but some deprecations and changes of behavior.

CC: mengxr

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #10235 from jkbradley/mllib-guide-update-1.6.
2015-12-16 11:53:04 -08:00
Jeff Zhang 6a880afa83 [SPARK-12361][PYSPARK][TESTS] Should set PYSPARK_DRIVER_PYTHON before Python tests
Although this patch still doesn't solve the issue why the return code is 0 (see JIRA description), it resolves the issue of python version mismatch.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10322 from zjffdu/SPARK-12361.
2015-12-16 11:29:51 -08:00
Yanbo Liang d252b2d544 [SPARK-12309][ML] Use sqlContext from MLlibTestSparkContext for spark.ml test suites
Use ```sqlContext``` from ```MLlibTestSparkContext``` rather than creating new one for spark.ml test suites. I have checked thoroughly and found there are four test cases need to update.

cc mengxr jkbradley

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10279 from yanboliang/spark-12309.
2015-12-16 11:07:54 -08:00
Yanbo Liang 860dc7f2f8 [SPARK-9694][ML] Add random seed Param to Scala CrossValidator
Add random seed Param to Scala CrossValidator

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #9108 from yanboliang/spark-9694.
2015-12-16 11:05:37 -08:00
Yu ISHIKAWA 7b6dc29d0e [SPARK-6518][MLLIB][EXAMPLE][DOC] Add example code and user guide for bisecting k-means
This PR includes only an example code in order to finish it quickly.
I'll send another PR for the docs soon.

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #9952 from yu-iskw/SPARK-6518.
2015-12-16 10:55:42 -08:00
Timothy Chen ad8c1f0b84 [SPARK-12345][MESOS] Filter SPARK_HOME when submitting Spark jobs with Mesos cluster mode.
SPARK_HOME is now causing problem with Mesos cluster mode since spark-submit script has been changed recently to take precendence when running spark-class scripts to look in SPARK_HOME if it's defined.

We should skip passing SPARK_HOME from the Spark client in cluster mode with Mesos, since Mesos shouldn't use this configuration but should use spark.executor.home instead.

Author: Timothy Chen <tnachen@gmail.com>

Closes #10332 from tnachen/scheduler_ui.
2015-12-16 10:54:15 -08:00
Yu ISHIKAWA 26d70bd2b4 [SPARK-12215][ML][DOC] User guide section for KMeans in spark.ml
cc jkbradley

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #10244 from yu-iskw/SPARK-12215.
2015-12-16 10:43:45 -08:00
Yanbo Liang 22f6cd86fc [SPARK-12310][SPARKR] Add write.json and write.parquet for SparkR
Add ```write.json``` and ```write.parquet``` for SparkR, and deprecated ```saveAsParquetFile```.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #10281 from yanboliang/spark-12310.
2015-12-16 10:34:30 -08:00
Jeff Zhang 2eb5af5f0d [SPARK-12318][SPARKR] Save mode in SparkR should be error by default
shivaram  Please help review.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10290 from zjffdu/SPARK-12318.
2015-12-16 10:32:32 -08:00
Davies Liu 54c512ba90 [SPARK-8745] [SQL] remove GenerateProjection
cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #10316 from davies/remove_generate_projection.
2015-12-16 10:22:48 -08:00
Timothy Hunter a6325fc401 [SPARK-12324][MLLIB][DOC] Fixes the sidebar in the ML documentation
This fixes the sidebar, using a pure CSS mechanism to hide it when the browser's viewport is too narrow.
Credit goes to the original author Titan-C (mentioned in the NOTICE).

Note that I am not a CSS expert, so I can only address comments up to some extent.

Default view:
<img width="936" alt="screen shot 2015-12-14 at 12 46 39 pm" src="https://cloud.githubusercontent.com/assets/7594753/11793597/6d1d6eda-a261-11e5-836b-6eb2054e9054.png">

When collapsed manually by the user:
<img width="1004" alt="screen shot 2015-12-14 at 12 54 02 pm" src="https://cloud.githubusercontent.com/assets/7594753/11793669/c991989e-a261-11e5-8bf6-aecf3bdb6319.png">

Disappears when column is too narrow:
<img width="697" alt="screen shot 2015-12-14 at 12 47 22 pm" src="https://cloud.githubusercontent.com/assets/7594753/11793607/7754dbcc-a261-11e5-8b15-e0d074b0e47c.png">

Can still be opened by the user if necessary:
<img width="651" alt="screen shot 2015-12-14 at 12 51 15 pm" src="https://cloud.githubusercontent.com/assets/7594753/11793612/7bf82968-a261-11e5-9cc3-e827a7a6b2b0.png">

Author: Timothy Hunter <timhunter@databricks.com>

Closes #10297 from thunterdb/12324.
2015-12-16 10:12:33 -08:00
Reynold Xin 1a3d0cd9f0 Revert "[SPARK-12105] [SQL] add convenient show functions"
This reverts commit 31b391019f.
2015-12-16 00:57:34 -08:00
Reynold Xin 18ea11c3a8 Revert "[HOTFIX] Compile error from commit 31b3910"
This reverts commit 840bd2e008.
2015-12-16 00:57:07 -08:00
Reynold Xin 554d840a9a Style fix for the previous 3 JDBC filter push down commits. 2015-12-15 22:32:51 -08:00
hyukjinkwon 2aad2d3724 [SPARK-12315][SQL] isnotnull operator not pushed down for JDBC datasource.
https://issues.apache.org/jira/browse/SPARK-12315
`IsNotNull` filter is not being pushed down for JDBC datasource.

It looks it is SQL standard according to [SQL-92](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt), SQL:1999, [SQL:2003](http://www.wiscorp.com/sql_2003_standard.zip) and [SQL:201x](http://www.wiscorp.com/sql20nn.zip) and I believe most databases support this.

In this PR, I simply added the case for `IsNotNull` filter to produce a proper filter string.

Author: hyukjinkwon <gurwls223@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@databricks.com>

Closes #10287 from HyukjinKwon/SPARK-12315.
2015-12-15 22:30:35 -08:00