Commit graph

5293 commits

Author SHA1 Message Date
ptkool 259860d23d [SPARK-20463] Add support for IS [NOT] DISTINCT FROM.
## What changes were proposed in this pull request?

Add support for the SQL standard distinct predicate to SPARK SQL.

```
<expression> IS [NOT] DISTINCT FROM <expression>
```

## How was this patch tested?

Tested using unit tests, integration tests, manual tests.

Author: ptkool <michael.styles@shopify.com>

Closes #17764 from ptkool/is_not_distinct_from.
2017-05-01 17:05:35 -07:00
Sean Owen af726cd611 [SPARK-20459][SQL] JdbcUtils throws IllegalStateException: Cause already initialized after getting SQLException
## What changes were proposed in this pull request?

Avoid failing to initCause on JDBC exception with cause initialized to null

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17800 from srowen/SPARK-20459.
2017-05-01 17:01:05 -07:00
Kunal Khamar 6fc6cf88d8 [SPARK-20464][SS] Add a job group and description for streaming queries and fix cancellation of running jobs using the job group
## What changes were proposed in this pull request?

Job group: adding a job group is required to properly cancel running jobs related to a query.
Description: the new description makes it easier to group the batches of a query by sorting by name in the Spark Jobs UI.

## How was this patch tested?

- Unit tests
- UI screenshot

  - Order by job id:
![screen shot 2017-04-27 at 5 10 09 pm](https://cloud.githubusercontent.com/assets/7865120/25509468/15452274-2b6e-11e7-87ba-d929816688cf.png)

  - Order by description:
![screen shot 2017-04-27 at 5 10 22 pm](https://cloud.githubusercontent.com/assets/7865120/25509474/1c298512-2b6e-11e7-99b8-fef1ef7665c1.png)

  - Order by job id (no query name):
![screen shot 2017-04-27 at 5 21 33 pm](https://cloud.githubusercontent.com/assets/7865120/25509482/28c96dc8-2b6e-11e7-8df0-9d3cdbb05e36.png)

  - Order by description (no query name):
![screen shot 2017-04-27 at 5 21 44 pm](https://cloud.githubusercontent.com/assets/7865120/25509489/37674742-2b6e-11e7-9357-b5c38ec16ac4.png)

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17765 from kunalkhamar/sc-6696.
2017-05-01 11:37:30 -07:00
Herman van Hovell 6b44c4d63a [SPARK-20534][SQL] Make outer generate exec return empty rows
## What changes were proposed in this pull request?
Generate exec does not produce `null` values if the generator for the input row is empty and the generate operates in outer mode without join. This is caused by the fact that the `join=false` code path is different from the `join=true` code path, and that the `join=false` code path did deal with outer properly. This PR addresses this issue.

## How was this patch tested?
Updated `outer*` tests in `GeneratorFunctionSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17810 from hvanhovell/SPARK-20534.
2017-05-01 09:46:35 -07:00
hyukjinkwon 1ee494d086 [SPARK-20492][SQL] Do not print empty parentheses for invalid primitive types in parser
## What changes were proposed in this pull request?

Currently, when the type string is invalid, it looks printing empty parentheses. This PR proposes a small improvement in an error message by removing it in the parse as below:

```scala
spark.range(1).select($"col".cast("aa"))
```

**Before**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa() is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

**After**

```
org.apache.spark.sql.catalyst.parser.ParseException:
DataType aa is not supported.(line 1, pos 0)

== SQL ==
aa
^^^
```

## How was this patch tested?

Unit tests in `DataTypeParserSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17784 from HyukjinKwon/SPARK-20492.
2017-04-30 08:24:10 -07:00
hyukjinkwon d228cd0b02 [SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark
## What changes were proposed in this pull request?

This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.

Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.

Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.

## How was this patch tested?

Doc tests were added and manually tested with the commands below:

`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`

Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17737 from HyukjinKwon/SPARK-20442.
2017-04-29 13:46:40 -07:00
hyukjinkwon 70f1bcd7bc [SPARK-20493][R] De-duplicate parse logics for DDL-like type strings in R
## What changes were proposed in this pull request?

It seems we are using `SQLUtils.getSQLDataType` for type string in structField. It looks we can replace this with `CatalystSqlParser.parseDataType`.

They look similar DDL-like type definitions as below:

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.select($"_1".cast("struct<_1:string>")).show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

Such type strings looks identical when R’s one as below:

```R
> write.df(sql("SELECT named_struct('_1', 'a') as struct"), "/tmp/aa", "parquet")
> collect(read.df("/tmp/aa", "parquet", structType(structField("struct", "struct<_1:string>"))))
  struct
1      a
```

R’s one is stricter because we are checking the types via regular expressions in R side ahead.

Actual logics there look a bit different but as we check it ahead in R side, it looks replacing it would not introduce (I think) no behaviour changes. To make this sure, the tests dedicated for it were added in SPARK-20105. (It looks `structField` is the only place that calls this method).

## How was this patch tested?

Existing tests - https://github.com/apache/spark/blob/master/R/pkg/inst/tests/testthat/test_sparkSQL.R#L143-L194 should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17785 from HyukjinKwon/SPARK-20493.
2017-04-29 11:02:17 -07:00
Tejas Patil 814a61a867 [SPARK-20487][SQL] Display serde for HiveTableScan node in explained plan
## What changes were proposed in this pull request?

This was a suggestion by rxin at https://github.com/apache/spark/pull/17780#issuecomment-298073408

## How was this patch tested?

- modified existing unit test
- manual testing:

```
scala> hc.sql(" SELECT * FROM tejasp_bucketed_partitioned_1  where name = ''  ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = )
   +- 'UnresolvedRelation `tejasp_bucketed_partitioned_1`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#24L, name#25, ds#26]
+- Filter (name#25 = )
   +- SubqueryAlias tejasp_bucketed_partitioned_1
      +- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Optimized Logical Plan ==
Filter (isnotnull(name#25) && (name#25 = ))
+- CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]

== Physical Plan ==
*Filter (isnotnull(name#25) && (name#25 = ))
+- HiveTableScan [user_id#24L, name#25, ds#26], CatalogRelation `default`.`tejasp_bucketed_partitioned_1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [user_id#24L, name#25], [ds#26]
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17806 from tejasapatil/add_serde.
2017-04-28 23:12:26 -07:00
caoxuewen ebff519c5e [SPARK-20471] Remove AggregateBenchmark testsuite warning: Two level hashmap is disabled but vectorized hashmap is enabled
What changes were proposed in this pull request?

remove  AggregateBenchmark testsuite warning:
such as '14:26:33.220 WARN org.apache.spark.sql.execution.aggregate.HashAggregateExec: Two level hashmap is disabled but vectorized hashmap is enabled.'

How was this patch tested?
unit tests: AggregateBenchmark
Modify the 'ignore function for 'test funtion

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #17771 from heary-cao/AggregateBenchmark.
2017-04-28 14:47:17 -07:00
Takeshi Yamamuro 59e3a56444 [SPARK-14471][SQL] Aliases in SELECT could be used in GROUP BY
## What changes were proposed in this pull request?
This pr added a new rule in `Analyzer` to resolve aliases in `GROUP BY`.
The current master throws an exception if `GROUP BY` clauses have aliases in `SELECT`;
```
scala> spark.sql("select a a1, a1 + 1 as b, count(1) from t group by a1")
org.apache.spark.sql.AnalysisException: cannot resolve '`a1`' given input columns: [a]; line 1 pos 51;
'Aggregate ['a1], [a#83L AS a1#87L, ('a1 + 1) AS b#88, count(1) AS count(1)#90L]
+- SubqueryAlias t
   +- Project [id#80L AS a#83L]
      +- Range (0, 10, step=1, splits=Some(8))

  at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:77)
  at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
  at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
```

## How was this patch tested?
Added tests in `SQLQuerySuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17191 from maropu/SPARK-14471.
2017-04-28 14:41:53 +08:00
Xiao Li e3c8160433 [SPARK-20476][SQL] Block users to create a table that use commas in the column names
### What changes were proposed in this pull request?
```SQL
hive> create table t1(`a,` string);
OK
Time taken: 1.399 seconds

hive> create table t2(`a,` string, b string);
FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe: columns has 3 elements while columns.types has 2 elements!)

hive> create table t2(`a,` string, b string) stored as parquet;
FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.IllegalArgumentException: ParquetHiveSerde initialization failed. Number of column name and column type differs. columnNames = [a, , b], columnTypes = [string, string]
```
It has a bug in Hive metastore.

When users do not provide alias name in the SELECT query, we call `toPrettySQL` to generate the alias name. For example, the string `get_json_object(jstring, '$.f1')` will be the alias name for the function call in the statement
```SQL
SELECT key, get_json_object(jstring, '$.f1') FROM tempView
```
Above is not an issue for the SELECT query statements. However, for CTAS, we hit the issue due to a bug in Hive metastore. Hive metastore does not like the column names containing commas and returned a confusing error message, like:
```
17/04/26 23:12:56 ERROR [hive.log(397) -- main]: error in initSerDe: org.apache.hadoop.hive.serde2.SerDeException org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe: columns has 2 elements while columns.types has 1 elements!
org.apache.hadoop.hive.serde2.SerDeException: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe: columns has 2 elements while columns.types has 1 elements!
```

Thus, this PR is to block users to create a table in Hive metastore when the table table has a column containing commas in the name.

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17781 from gatorsmile/blockIllegalColumnNames.
2017-04-28 14:16:40 +08:00
Wenchen Fan b90bf520fd [SPARK-12837][CORE] Do not send the name of internal accumulator to executor side
## What changes were proposed in this pull request?

When sending accumulator updates back to driver, the network overhead is pretty big as there are a lot of accumulators, e.g. `TaskMetrics` will send about 20 accumulators everytime, there may be a lot of `SQLMetric` if the query plan is complicated.

Therefore, it's critical to reduce the size of serialized accumulator. A simple way is to not send the name of internal accumulators to executor side, as it's unnecessary. When executor sends accumulator updates back to driver, we can look up the accumulator name in `AccumulatorContext` easily. Note that, we still need to send names of normal accumulators, as the user code run at executor side may rely on accumulator names.

In the future, we should reimplement `TaskMetrics` to not rely on accumulators and use custom serialization.

Tried on the example in https://issues.apache.org/jira/browse/SPARK-12837, the size of serialized accumulator has been cut down by about 40%.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17596 from cloud-fan/oom.
2017-04-27 19:38:14 -07:00
Tejas Patil a4aa4665a6 [SPARK-20487][SQL] HiveTableScan node is quite verbose in explained plan
## What changes were proposed in this pull request?

Changed `TreeNode.argString` to handle `CatalogTable` separately (otherwise it would call the default `toString` on the `CatalogTable`)

## How was this patch tested?

- Expanded scope of existing unit test to ensure that verbose information is not present
- Manual testing

Before

```
scala> hc.sql(" SELECT * FROM my_table WHERE name = 'foo' ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = foo)
   +- 'UnresolvedRelation `my_table`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#13L, name#14, ds#15]
+- Filter (name#14 = foo)
   +- SubqueryAlias my_table
      +- CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]

== Optimized Logical Plan ==
Filter (isnotnull(name#14) && (name#14 = foo))
+- CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]

== Physical Plan ==
*Filter (isnotnull(name#14) && (name#14 = foo))
+- HiveTableScan [user_id#13L, name#14, ds#15], CatalogRelation CatalogTable(
Database: default
Table: my_table
Owner: tejasp
Created: Fri Apr 14 17:05:50 PDT 2017
Last Access: Wed Dec 31 16:00:00 PST 1969
Type: MANAGED
Provider: hive
Properties: [serialization.format=1]
Statistics: 9223372036854775807 bytes
Location: file:/tmp/warehouse/my_table
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.TextInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
Partition Provider: Catalog
Partition Columns: [`ds`]
Schema: root
-- user_id: long (nullable = true)
-- name: string (nullable = true)
-- ds: string (nullable = true)
), [user_id#13L, name#14], [ds#15]
```

After

```
scala> hc.sql(" SELECT * FROM my_table WHERE name = 'foo' ").explain(true)
== Parsed Logical Plan ==
'Project [*]
+- 'Filter ('name = foo)
   +- 'UnresolvedRelation `my_table`

== Analyzed Logical Plan ==
user_id: bigint, name: string, ds: string
Project [user_id#13L, name#14, ds#15]
+- Filter (name#14 = foo)
   +- SubqueryAlias my_table
      +- CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]

== Optimized Logical Plan ==
Filter (isnotnull(name#14) && (name#14 = foo))
+- CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]

== Physical Plan ==
*Filter (isnotnull(name#14) && (name#14 = foo))
+- HiveTableScan [user_id#13L, name#14, ds#15], CatalogRelation `default`.`my_table`, [user_id#13L, name#14], [ds#15]
```

Author: Tejas Patil <tejasp@fb.com>

Closes #17780 from tejasapatil/SPARK-20487_verbose_plan.
2017-04-27 12:13:16 -07:00
Kris Mok 26ac2ce05c [SPARK-20482][SQL] Resolving Casts is too strict on having time zone set
## What changes were proposed in this pull request?

Relax the requirement that a `TimeZoneAwareExpression` has to have its `timeZoneId` set to be considered resolved.
With this change, a `Cast` (which is a `TimeZoneAwareExpression`) can be considered resolved if the `(fromType, toType)` combination doesn't require time zone information.

Also de-relaxed test cases in `CastSuite` so Casts in that test suite don't get a default`timeZoneId = Option("GMT")`.

## How was this patch tested?

Ran the de-relaxed`CastSuite` and it's passing. Also ran the SQL unit tests and they're passing too.

Author: Kris Mok <kris.mok@databricks.com>

Closes #17777 from rednaxelafx/fix-catalyst-cast-timezone.
2017-04-27 12:08:16 -07:00
Takeshi Yamamuro b4724db19a [SPARK-20425][SQL] Support a vertical display mode for Dataset.show
## What changes were proposed in this pull request?
This pr added a new display mode for `Dataset.show` to print output rows vertically (one line per column value). In the current master, when printing Dataset with many columns, the readability is low like;

```
scala> val df = spark.range(100).selectExpr((0 until 100).map(i => s"rand() AS c$i"): _*)
scala> df.show(3, 0)
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
|c0                |c1                |c2                |c3                 |c4                |c5                |c6                 |c7                |c8                |c9                |c10               |c11                |c12               |c13               |c14               |c15                |c16                |c17                |c18               |c19               |c20                |c21               |c22                |c23               |c24                |c25                |c26                |c27                 |c28                |c29               |c30                |c31                 |c32               |c33               |c34                |c35                |c36                |c37               |c38               |c39                |c40               |c41               |c42                |c43                |c44                |c45               |c46                 |c47                 |c48                |c49                |c50                |c51                |c52                |c53                |c54                 |c55                |c56                |c57                |c58                |c59               |c60               |c61                |c62                |c63               |c64                |c65               |c66               |c67              |c68                |c69                |c70               |c71                |c72               |c73                |c74                |c75                |c76               |c77                |c78               |c79                |c80                |c81                |c82                |c83                |c84                |c85                |c86                |c87               |c88                |c89                |c90               |c91               |c92               |c93                |c94               |c95                |c96               |c97                |c98                |c99                |
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
|0.6306087152476858|0.9174349686288383|0.5511324165035159|0.3320844128641819 |0.7738486877101489|0.2154915886962553|0.4754997600674299 |0.922780639280355 |0.7136894772661909|0.2277580838165979|0.5926874459847249|0.40311408392226633|0.467830264333843 |0.8330466896984213|0.1893258482389527|0.6320849515511165 |0.7530911056912044 |0.06700254871955424|0.370528597355559 |0.2755437445193154|0.23704391110980128|0.8067400174905822|0.13597793616251852|0.1708888820162453|0.01672725007605702|0.983118121881555  |0.25040195628629924|0.060537253723083384|0.20000530582637488|0.3400572407133511|0.9375689433322597 |0.057039316954370256|0.8053269714347623|0.5247817572228813|0.28419308820527944|0.9798908885194533 |0.31805988175678146|0.7034448027077574|0.5400575751346084|0.25336322371116216|0.9361634546853429|0.6118681368289798|0.6295081549153907 |0.13417468943957422|0.41617137072255794|0.7267230869252035|0.023792726137561115|0.5776157058356362  |0.04884204913195467|0.26728716103441275|0.646680370807925  |0.9782712690657244 |0.16434031314818154|0.20985522381321275|0.24739842475440077 |0.26335189682977334|0.19604841662422068|0.10742950487300651|0.20283136488091502|0.3100312319723688|0.886959006630645 |0.25157102269776244|0.34428775168410786|0.3500506818575777|0.3781142441912052 |0.8560316444386715|0.4737104888956839|0.735903101602148|0.02236617130529006|0.8769074095835873 |0.2001426662503153|0.5534032319238532 |0.7289496620397098|0.41955191309992157|0.9337700133660436 |0.34059094378451005|0.6419144759403556|0.08167496930341167|0.9947099478497635|0.48010888605366586|0.22314796858167918|0.17786598882331306|0.7351521162297135 |0.5422057170020095 |0.9521927872726792 |0.7459825486368227 |0.40907708791990627|0.8903819313311575|0.7251413746923618 |0.2977174938745204 |0.9515209660203555|0.9375968604766713|0.5087851740042524|0.4255237544908751 |0.8023768698664653|0.48003189618006703|0.1775841829745185|0.09050775629268382|0.6743909291138167 |0.2498415755876865 |
|0.6866473844170801|0.4774360641212433|0.631696201340726 |0.33979113021468343|0.5663049010847052|0.7280190472258865|0.41370958502324806|0.9977433873622218|0.7671957338989901|0.2788708556233931|0.3355106391656496|0.88478952319287   |0.0333974166999893|0.6061744715862606|0.9617779139652359|0.22484954822341863|0.12770906021550898|0.5577789629508672 |0.2877649024640704|0.5566577406549361|0.9334933255278052 |0.9166720585157266|0.9689249324600591 |0.6367502457478598|0.7993572745928459 |0.23213222324218108|0.11928284054154137|0.6173493362456599  |0.0505122058694798 |0.9050228629552983|0.17112767911121707|0.47395598348370005 |0.5820498657823081|0.6241124650645072|0.18587258258036776|0.14987593554122225|0.3079446253653946 |0.9414228822867968|0.8362276265462365|0.9155655305576353 |0.5121559807153562|0.8963362656525707|0.22765970274318037|0.8177039187132797 |0.8190326635933787 |0.5256005177032199|0.8167598457269669  |0.030936807130934496|0.6733006585281015 |0.4208049626816347 |0.24603085738518538|0.22719198954208153|0.1622280557565281 |0.22217325159218038|0.014684419513742553|0.08987111517447499|0.2157764759142622 |0.8223414104088321 |0.4868624404491777 |0.4016191733088167|0.6169281906889263|0.15603611040433385|0.18289285085714913|0.9538408988218972|0.15037154865295121|0.5364516961987454|0.8077254873163031|0.712600478545675|0.7277477241003857 |0.19822912960348305|0.8305051199208777|0.18631911396566114|0.8909532487898342|0.3470409226992506 |0.35306974180587636|0.9107058868891469 |0.3321327206004986|0.48952332459050607|0.3630403307479373|0.5400046826340376 |0.5387377194310529 |0.42860539421837585|0.23214101630985995|0.21438968839794847|0.15370603160082352|0.04355605642700022|0.6096006707067466 |0.6933354157094292|0.06302172470859002|0.03174631856164001|0.664243581650643 |0.7833239547446621|0.696884598352864 |0.34626385933237736|0.9263495598791336|0.404818892816584  |0.2085585394755507|0.6150004897990109 |0.05391193524302473|0.28188484028329097|
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
only showing top 2 rows
```

`psql`, CLI for PostgreSQL, supports a vertical display mode for this case like:
http://stackoverflow.com/questions/9604723/alternate-output-format-for-psql

```
-RECORD 0-------------------
 c0  | 0.6306087152476858
 c1  | 0.9174349686288383
 c2  | 0.5511324165035159
...
 c98 | 0.05391193524302473
 c99 | 0.28188484028329097
-RECORD 1-------------------
 c0  | 0.6866473844170801
 c1  | 0.4774360641212433
 c2  | 0.631696201340726
...
 c98 | 0.05391193524302473
 c99 | 0.28188484028329097
only showing top 2 rows
```

## How was this patch tested?
Added tests in `DataFrameSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17733 from maropu/SPARK-20425.
2017-04-26 22:18:01 -07:00
Weiqing Yang 2ba1eba371 [SPARK-12868][SQL] Allow adding jars from hdfs
## What changes were proposed in this pull request?
Spark 2.2 is going to be cut, it'll be great if SPARK-12868 can be resolved before that. There have been several PRs for this like [PR#16324](https://github.com/apache/spark/pull/16324) , but all of them are inactivity for a long time or have been closed.

This PR added a SparkUrlStreamHandlerFactory, which relies on 'protocol' to choose the appropriate
UrlStreamHandlerFactory like FsUrlStreamHandlerFactory to create URLStreamHandler.

## How was this patch tested?
1. Add a new unit test.
2. Check manually.
Before: throw an exception with " failed unknown protocol: hdfs"
<img width="914" alt="screen shot 2017-03-17 at 9 07 36 pm" src="https://cloud.githubusercontent.com/assets/8546874/24075277/5abe0a7c-0bd5-11e7-900e-ec3d3105da0b.png">

After:
<img width="1148" alt="screen shot 2017-03-18 at 11 42 18 am" src="https://cloud.githubusercontent.com/assets/8546874/24075283/69382a60-0bd5-11e7-8d30-d9405c3aaaba.png">

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #17342 from weiqingy/SPARK-18910.
2017-04-26 13:54:40 -07:00
Michal Szafranski a277ae80a2 [SPARK-20474] Fixing OnHeapColumnVector reallocation
## What changes were proposed in this pull request?
OnHeapColumnVector reallocation copies to the new storage data up to 'elementsAppended'. This variable is only updated when using the ColumnVector.appendX API, while ColumnVector.putX is more commonly used.

## How was this patch tested?
Tested using existing unit tests.

Author: Michal Szafranski <michal@databricks.com>

Closes #17773 from michal-databricks/spark-20474.
2017-04-26 12:47:37 -07:00
Michal Szafranski 99c6cf9ef1 [SPARK-20473] Enabling missing types in ColumnVector.Array
## What changes were proposed in this pull request?
ColumnVector implementations originally did not support some Catalyst types (float, short, and boolean). Now that they do, those types should be also added to the ColumnVector.Array.

## How was this patch tested?
Tested using existing unit tests.

Author: Michal Szafranski <michal@databricks.com>

Closes #17772 from michal-databricks/spark-20473.
2017-04-26 11:21:25 -07:00
Eric Wasserman 57e1da3946 [SPARK-16548][SQL] Inconsistent error handling in JSON parsing SQL functions
## What changes were proposed in this pull request?

change to using Jackson's `com.fasterxml.jackson.core.JsonFactory`

    public JsonParser createParser(String content)

## How was this patch tested?

existing unit tests

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Eric Wasserman <ericw@sgn.com>

Closes #17693 from ewasserman/SPARK-20314.
2017-04-26 11:42:43 +08:00
Sameer Agarwal caf392025c [SPARK-18127] Add hooks and extension points to Spark
## What changes were proposed in this pull request?

This patch adds support for customizing the spark session by injecting user-defined custom extensions. This allows a user to add custom analyzer rules/checks, optimizer rules, planning strategies or even a customized parser.

## How was this patch tested?

Unit Tests in SparkSessionExtensionSuite

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17724 from sameeragarwal/session-extensions.
2017-04-25 17:05:20 -07:00
Sameer Agarwal 31345fde82 [SPARK-20451] Filter out nested mapType datatypes from sort order in randomSplit
## What changes were proposed in this pull request?

In `randomSplit`, It is possible that the underlying dataset doesn't guarantee the ordering of rows in its constituent partitions each time a split is materialized which could result in overlapping
splits.

To prevent this, as part of SPARK-12662, we explicitly sort each input partition to make the ordering deterministic. Given that `MapTypes` cannot be sorted this patch explicitly prunes them out from the sort order. Additionally, if the resulting sort order is empty, this patch then materializes the dataset to guarantee determinism.

## How was this patch tested?

Extended `randomSplit on reordered partitions` in `DataFrameStatSuite` to also test for dataframes with mapTypes nested mapTypes.

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17751 from sameeragarwal/randomsplit2.
2017-04-25 13:05:20 +08:00
Josh Rosen f44c8a843c [SPARK-20453] Bump master branch version to 2.3.0-SNAPSHOT
This patch bumps the master branch version to `2.3.0-SNAPSHOT`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17753 from JoshRosen/SPARK-20453.
2017-04-24 21:48:04 -07:00
Xiao Li 776a2c0e91 [SPARK-20439][SQL] Fix Catalog API listTables and getTable when failed to fetch table metadata
### What changes were proposed in this pull request?

`spark.catalog.listTables` and `spark.catalog.getTable` does not work if we are unable to retrieve table metadata due to any reason (e.g., table serde class is not accessible or the table type is not accepted by Spark SQL). After this PR, the APIs still return the corresponding Table without the description and tableType)

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17730 from gatorsmile/listTables.
2017-04-24 17:21:42 +08:00
Takeshi Yamamuro b3c572a6b3 [SPARK-20430][SQL] Initialise RangeExec parameters in a driver side
## What changes were proposed in this pull request?
This pr initialised `RangeExec` parameters in a driver side.
In the current master, a query below throws `NullPointerException`;
```
sql("SET spark.sql.codegen.wholeStage=false")
sql("SELECT * FROM range(1)").show

17/04/20 17:11:05 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.NullPointerException
        at org.apache.spark.sql.execution.SparkPlan.sparkContext(SparkPlan.scala:54)
        at org.apache.spark.sql.execution.RangeExec.numSlices(basicPhysicalOperators.scala:343)
        at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:506)
        at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:505)
        at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844)
        at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
        at org.apache.spark.scheduler.Task.run(Task.scala:108)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:320)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
```

## How was this patch tested?
Added a test in `DataFrameRangeSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17717 from maropu/SPARK-20430.
2017-04-22 09:41:58 -07:00
Kazuaki Ishizaki a750a59597 [SPARK-20341][SQL] Support BigInt's value that does not fit in long value range
## What changes were proposed in this pull request?

This PR avoids an exception in the case where `scala.math.BigInt` has a value that does not fit into long value range (e.g. `Long.MAX_VALUE+1`). When we run the following code by using the current Spark, the following exception is thrown.

This PR keeps the value using `BigDecimal` if we detect such an overflow case by catching `ArithmeticException`.

Sample program:
```
case class BigIntWrapper(value:scala.math.BigInt)```
spark.createDataset(BigIntWrapper(scala.math.BigInt("10000000000000000002"))::Nil).show
```
Exception:
```
Error while encoding: java.lang.ArithmeticException: BigInteger out of long range
staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0
java.lang.RuntimeException: Error while encoding: java.lang.ArithmeticException: BigInteger out of long range
staticinvoke(class org.apache.spark.sql.types.Decimal$, DecimalType(38,0), apply, assertnotnull(assertnotnull(input[0, org.apache.spark.sql.BigIntWrapper, true])).value, true) AS value#0
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:290)
	at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454)
	at org.apache.spark.sql.SparkSession$$anonfun$2.apply(SparkSession.scala:454)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.immutable.List.map(List.scala:285)
	at org.apache.spark.sql.SparkSession.createDataset(SparkSession.scala:454)
	at org.apache.spark.sql.Agg$$anonfun$18.apply$mcV$sp(MySuite.scala:192)
	at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192)
	at org.apache.spark.sql.Agg$$anonfun$18.apply(MySuite.scala:192)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
...
Caused by: java.lang.ArithmeticException: BigInteger out of long range
	at java.math.BigInteger.longValueExact(BigInteger.java:4531)
	at org.apache.spark.sql.types.Decimal.set(Decimal.scala:140)
	at org.apache.spark.sql.types.Decimal$.apply(Decimal.scala:434)
	at org.apache.spark.sql.types.Decimal.apply(Decimal.scala)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:287)
	... 59 more
```

## How was this patch tested?

Add new test suite into `DecimalSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #17684 from kiszk/SPARK-20341.
2017-04-21 22:25:35 +08:00
Juliusz Sompolski c9e6035e1f [SPARK-20412] Throw ParseException from visitNonOptionalPartitionSpec instead of returning null values.
## What changes were proposed in this pull request?

If a partitionSpec is supposed to not contain optional values, a ParseException should be thrown, and not nulls returned.
The nulls can later cause NullPointerExceptions in places not expecting them.

## How was this patch tested?

A query like "SHOW PARTITIONS tbl PARTITION(col1='val1', col2)" used to throw a NullPointerException.
Now it throws a ParseException.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17707 from juliuszsompolski/SPARK-20412.
2017-04-21 22:11:24 +08:00
Herman van Hovell e2b3d2367a [SPARK-20420][SQL] Add events to the external catalog
## What changes were proposed in this pull request?
It is often useful to be able to track changes to the `ExternalCatalog`. This PR makes the `ExternalCatalog` emit events when a catalog object is changed. Events are fired before and after the change.

The following events are fired per object:

- Database
  - CreateDatabasePreEvent: event fired before the database is created.
  - CreateDatabaseEvent: event fired after the database has been created.
  - DropDatabasePreEvent: event fired before the database is dropped.
  - DropDatabaseEvent: event fired after the database has been dropped.
- Table
  - CreateTablePreEvent: event fired before the table is created.
  - CreateTableEvent: event fired after the table has been created.
  - RenameTablePreEvent: event fired before the table is renamed.
  - RenameTableEvent: event fired after the table has been renamed.
  - DropTablePreEvent: event fired before the table is dropped.
  - DropTableEvent: event fired after the table has been dropped.
- Function
  - CreateFunctionPreEvent: event fired before the function is created.
  - CreateFunctionEvent: event fired after the function has been created.
  - RenameFunctionPreEvent: event fired before the function is renamed.
  - RenameFunctionEvent: event fired after the function has been renamed.
  - DropFunctionPreEvent: event fired before the function is dropped.
  - DropFunctionPreEvent: event fired after the function has been dropped.

The current events currently only contain the names of the object modified. We add more events, and more details at a later point.

A user can monitor changes to the external catalog by adding a listener to the Spark listener bus checking for `ExternalCatalogEvent`s using the `SparkListener.onOtherEvent` hook. A more direct approach is add listener directly to the `ExternalCatalog`.

## How was this patch tested?
Added the `ExternalCatalogEventSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17710 from hvanhovell/SPARK-20420.
2017-04-21 00:05:03 -07:00
Takeshi Yamamuro 48d760d028 [SPARK-20281][SQL] Print the identical Range parameters of SparkContext APIs and SQL in explain
## What changes were proposed in this pull request?
This pr modified code to print the identical `Range` parameters of SparkContext APIs and SQL in `explain` output. In the current master, they internally use `defaultParallelism` for `splits` by default though, they print different strings in explain output;

```
scala> spark.range(4).explain
== Physical Plan ==
*Range (0, 4, step=1, splits=Some(8))

scala> sql("select * from range(4)").explain
== Physical Plan ==
*Range (0, 4, step=1, splits=None)
```

## How was this patch tested?
Added tests in `SQLQuerySuite` and modified some results in the existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17670 from maropu/SPARK-20281.
2017-04-20 19:40:21 -07:00
Herman van Hovell 760c8d088d [SPARK-20329][SQL] Make timezone aware expression without timezone unresolved
## What changes were proposed in this pull request?
A cast expression with a resolved time zone is not equal to a cast expression without a resolved time zone. The `ResolveAggregateFunction` assumed that these expression were the same, and would fail to resolve `HAVING` clauses which contain a `Cast` expression.

This is in essence caused by the fact that a `TimeZoneAwareExpression` can be resolved without a set time zone. This PR fixes this, and makes a `TimeZoneAwareExpression` unresolved as long as it has no TimeZone set.

## How was this patch tested?
Added a regression test to the `SQLQueryTestSuite.having` file.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17641 from hvanhovell/SPARK-20329.
2017-04-21 10:06:12 +08:00
Juliusz Sompolski 0368eb9d86 [SPARK-20367] Properly unescape column names of partitioning columns parsed from paths.
## What changes were proposed in this pull request?

When infering partitioning schema from paths, the column in parsePartitionColumn should be unescaped with unescapePathName, just like it is being done in e.g. parsePathFragmentAsSeq.

## How was this patch tested?

Added a test to FileIndexSuite.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17703 from juliuszsompolski/SPARK-20367.
2017-04-21 09:49:42 +08:00
Herman van Hovell 0332063553 [SPARK-20410][SQL] Make sparkConf a def in SharedSQLContext
## What changes were proposed in this pull request?
It is kind of annoying that `SharedSQLContext.sparkConf` is a val when overriding test cases, because you cannot call `super` on it. This PR makes it a function.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17705 from hvanhovell/SPARK-20410.
2017-04-20 22:37:04 +02:00
Dilip Biswal d95e4d9d6a [SPARK-20334][SQL] Return a better error message when correlated predicates contain aggregate expression that has mixture of outer and local references.
## What changes were proposed in this pull request?
Address a follow up in [comment](https://github.com/apache/spark/pull/16954#discussion_r105718880)
Currently subqueries with correlated predicates containing aggregate expression having mixture of outer references and local references generate a codegen error like following :

```SQL
SELECT t1a
FROM   t1
GROUP  BY 1
HAVING EXISTS (SELECT 1
               FROM  t2
               WHERE t2a < min(t1a + t2a));
```
Exception snippet.
```
Cannot evaluate expression: min((input[0, int, false] + input[4, int, false]))
	at org.apache.spark.sql.catalyst.expressions.Unevaluable$class.doGenCode(Expression.scala:226)
	at org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression.doGenCode(interfaces.scala:87)
	at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:106)
	at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$genCode$2.apply(Expression.scala:103)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.sql.catalyst.expressions.Expression.genCode(Expression.scala:103)

```
After this PR, a better error message is issued.
```
org.apache.spark.sql.AnalysisException
Error in query: Found an aggregate expression in a correlated
predicate that has both outer and local references, which is not supported yet.
Aggregate expression: min((t1.`t1a` + t2.`t2a`)),
Outer references: t1.`t1a`,
Local references: t2.`t2a`.;
```
## How was this patch tested?
Added tests in SQLQueryTestSuite.

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

Closes #17636 from dilipbiswal/subquery_followup1.
2017-04-20 22:35:48 +02:00
Bogdan Raducanu c5a31d160f [SPARK-20407][TESTS] ParquetQuerySuite 'Enabling/disabling ignoreCorruptFiles' flaky test
## What changes were proposed in this pull request?

SharedSQLContext.afterEach now calls DebugFilesystem.assertNoOpenStreams inside eventually.
SQLTestUtils withTempDir calls waitForTasksToFinish before deleting the directory.

## How was this patch tested?
Added new test in ParquetQuerySuite based on the flaky test

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #17701 from bogdanrdc/SPARK-20407.
2017-04-20 18:49:39 +02:00
Wenchen Fan b91873db09 [SPARK-20409][SQL] fail early if aggregate function in GROUP BY
## What changes were proposed in this pull request?

It's illegal to have aggregate function in GROUP BY, and we should fail at analysis phase, if this happens.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17704 from cloud-fan/minor.
2017-04-20 16:59:38 +02:00
Reynold Xin c6f62c5b81 [SPARK-20405][SQL] Dataset.withNewExecutionId should be private
## What changes were proposed in this pull request?
Dataset.withNewExecutionId is only used in Dataset itself and should be private.

## How was this patch tested?
N/A - this is a simple visibility change.

Author: Reynold Xin <rxin@databricks.com>

Closes #17699 from rxin/SPARK-20405.
2017-04-20 14:29:59 +02:00
Xiao Li 55bea56911 [SPARK-20156][SQL][FOLLOW-UP] Java String toLowerCase "Turkish locale bug" in Database and Table DDLs
### What changes were proposed in this pull request?
Database and Table names conform the Hive standard ("[a-zA-z_0-9]+"), i.e. if this name only contains characters, numbers, and _.

When calling `toLowerCase` on the names, we should add `Locale.ROOT` to the `toLowerCase`for avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

### How was this patch tested?
Added a test case

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17655 from gatorsmile/locale.
2017-04-20 11:13:48 +01:00
Eric Liang dd6d55d5de [SPARK-20398][SQL] range() operator should include cancellation reason when killed
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-19820 adds a reason field for why tasks were killed. However, for backwards compatibility it left the old TaskKilledException constructor which defaults to "unknown reason".
The range() operator should use the constructor that fills in the reason rather than dropping it on task kill.

## How was this patch tested?

Existing tests, and I tested this manually.

Author: Eric Liang <ekl@databricks.com>

Closes #17692 from ericl/fix-kill-reason-in-range.
2017-04-19 19:53:40 -07:00
Shixiong Zhu 39e303a8b6 [MINOR][SS] Fix a missing space in UnsupportedOperationChecker error message
## What changes were proposed in this pull request?

Also went through the same file to ensure other string concatenation are correct.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17691 from zsxwing/fix-error-message.
2017-04-19 18:58:14 -07:00
ptkool 63824b2c8e [SPARK-20350] Add optimization rules to apply Complementation Laws.
## What changes were proposed in this pull request?

Apply Complementation Laws during boolean expression simplification.

## How was this patch tested?

Tested using unit tests, integration tests, and manual tests.

Author: ptkool <michael.styles@shopify.com>
Author: Michael Styles <michael.styles@shopify.com>

Closes #17650 from ptkool/apply_complementation_laws.
2017-04-20 09:51:13 +08:00
Liang-Chi Hsieh 773754b6c1 [SPARK-20356][SQL] Pruned InMemoryTableScanExec should have correct output partitioning and ordering
## What changes were proposed in this pull request?

The output of `InMemoryTableScanExec` can be pruned and mismatch with `InMemoryRelation` and its child plan's output. This causes wrong output partitioning and ordering.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #17679 from viirya/SPARK-20356.
2017-04-19 16:01:28 +08:00
Koert Kuipers 608bf30f0b [SPARK-20359][SQL] Avoid unnecessary execution in EliminateOuterJoin optimization that can lead to NPE
Avoid necessary execution that can lead to NPE in EliminateOuterJoin and add test in DataFrameSuite to confirm NPE is no longer thrown

## What changes were proposed in this pull request?
Change leftHasNonNullPredicate and rightHasNonNullPredicate to lazy so they are only executed when needed.

## How was this patch tested?

Added test in DataFrameSuite that failed before this fix and now succeeds. Note that a test in catalyst project would be better but i am unsure how to do this.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Koert Kuipers <koert@tresata.com>

Closes #17660 from koertkuipers/feat-catch-npe-in-eliminate-outer-join.
2017-04-19 15:52:47 +08:00
Kazuaki Ishizaki e468a96c40 [SPARK-20254][SQL] Remove unnecessary data conversion for Dataset with primitive array
## What changes were proposed in this pull request?

This PR elminates unnecessary data conversion, which is introduced by SPARK-19716, for Dataset with primitve array in the generated Java code.
When we run the following example program, now we get the Java code "Without this PR". In this code, lines 56-82 are unnecessary since the primitive array in ArrayData can be converted into Java primitive array by using ``toDoubleArray()`` method. ``GenericArrayData`` is not required.

```java
val ds = sparkContext.parallelize(Seq(Array(1.1, 2.2)), 1).toDS.cache
ds.count
ds.map(e => e).show
```

Without this PR
```
== Parsed Logical Plan ==
'SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- 'MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D
   +- 'DeserializeToObject unresolveddeserializer(unresolvedmapobjects(<function1>, getcolumnbyordinal(0, ArrayType(DoubleType,false)), None).toDoubleArray), obj#23: [D
      +- SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
         +- ExternalRDD [obj#1]

== Analyzed Logical Plan ==
value: array<double>
SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D
   +- DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D
      +- SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
         +- ExternalRDD [obj#1]

== Optimized Logical Plan ==
SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- MapElements <function1>, class [D, [StructField(value,ArrayType(DoubleType,false),true)], obj#24: [D
   +- DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D
      +- InMemoryRelation [value#2], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
            +- *SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
               +- Scan ExternalRDDScan[obj#1]

== Physical Plan ==
*SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#25]
+- *MapElements <function1>, obj#24: [D
   +- *DeserializeToObject mapobjects(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, assertnotnull(lambdavariable(MapObjects_loopValue5, MapObjects_loopIsNull5, DoubleType, true), - array element class: "scala.Double", - root class: "scala.Array"), value#2, None, MapObjects_builderValue5).toDoubleArray, obj#23: [D
      +- InMemoryTableScan [value#2]
            +- InMemoryRelation [value#2], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
                  +- *SerializeFromObject [staticinvoke(class org.apache.spark.sql.catalyst.expressions.UnsafeArrayData, ArrayType(DoubleType,false), fromPrimitiveArray, input[0, [D, true], true) AS value#2]
                     +- Scan ExternalRDDScan[obj#1]
```

```java
/* 050 */   protected void processNext() throws java.io.IOException {
/* 051 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 052 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 053 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 054 */       ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0));
/* 055 */
/* 056 */       ArrayData deserializetoobject_value1 = null;
/* 057 */
/* 058 */       if (!inputadapter_isNull) {
/* 059 */         int deserializetoobject_dataLength = inputadapter_value.numElements();
/* 060 */
/* 061 */         Double[] deserializetoobject_convertedArray = null;
/* 062 */         deserializetoobject_convertedArray = new Double[deserializetoobject_dataLength];
/* 063 */
/* 064 */         int deserializetoobject_loopIndex = 0;
/* 065 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 066 */           MapObjects_loopValue2 = (double) (inputadapter_value.getDouble(deserializetoobject_loopIndex));
/* 067 */           MapObjects_loopIsNull2 = inputadapter_value.isNullAt(deserializetoobject_loopIndex);
/* 068 */
/* 069 */           if (MapObjects_loopIsNull2) {
/* 070 */             throw new RuntimeException(((java.lang.String) references[0]));
/* 071 */           }
/* 072 */           if (false) {
/* 073 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = null;
/* 074 */           } else {
/* 075 */             deserializetoobject_convertedArray[deserializetoobject_loopIndex] = MapObjects_loopValue2;
/* 076 */           }
/* 077 */
/* 078 */           deserializetoobject_loopIndex += 1;
/* 079 */         }
/* 080 */
/* 081 */         deserializetoobject_value1 = new org.apache.spark.sql.catalyst.util.GenericArrayData(deserializetoobject_convertedArray); /*###*/
/* 082 */       }
/* 083 */       boolean deserializetoobject_isNull = true;
/* 084 */       double[] deserializetoobject_value = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull = false;
/* 087 */         if (!deserializetoobject_isNull) {
/* 088 */           Object deserializetoobject_funcResult = null;
/* 089 */           deserializetoobject_funcResult = deserializetoobject_value1.toDoubleArray();
/* 090 */           if (deserializetoobject_funcResult == null) {
/* 091 */             deserializetoobject_isNull = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value = (double[]) deserializetoobject_funcResult;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull = deserializetoobject_value == null;
/* 098 */       }
/* 099 */
/* 100 */       boolean mapelements_isNull = true;
/* 101 */       double[] mapelements_value = null;
/* 102 */       if (!false) {
/* 103 */         mapelements_resultIsNull = false;
/* 104 */
/* 105 */         if (!mapelements_resultIsNull) {
/* 106 */           mapelements_resultIsNull = deserializetoobject_isNull;
/* 107 */           mapelements_argValue = deserializetoobject_value;
/* 108 */         }
/* 109 */
/* 110 */         mapelements_isNull = mapelements_resultIsNull;
/* 111 */         if (!mapelements_isNull) {
/* 112 */           Object mapelements_funcResult = null;
/* 113 */           mapelements_funcResult = ((scala.Function1) references[1]).apply(mapelements_argValue);
/* 114 */           if (mapelements_funcResult == null) {
/* 115 */             mapelements_isNull = true;
/* 116 */           } else {
/* 117 */             mapelements_value = (double[]) mapelements_funcResult;
/* 118 */           }
/* 119 */
/* 120 */         }
/* 121 */         mapelements_isNull = mapelements_value == null;
/* 122 */       }
/* 123 */
/* 124 */       serializefromobject_resultIsNull = false;
/* 125 */
/* 126 */       if (!serializefromobject_resultIsNull) {
/* 127 */         serializefromobject_resultIsNull = mapelements_isNull;
/* 128 */         serializefromobject_argValue = mapelements_value;
/* 129 */       }
/* 130 */
/* 131 */       boolean serializefromobject_isNull = serializefromobject_resultIsNull;
/* 132 */       final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue);
/* 133 */       serializefromobject_isNull = serializefromobject_value == null;
/* 134 */       serializefromobject_holder.reset();
/* 135 */
/* 136 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 137 */
/* 138 */       if (serializefromobject_isNull) {
/* 139 */         serializefromobject_rowWriter.setNullAt(0);
/* 140 */       } else {
/* 141 */         // Remember the current cursor so that we can calculate how many bytes are
/* 142 */         // written later.
/* 143 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 144 */
/* 145 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 146 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 147 */           // grow the global buffer before writing data.
/* 148 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 149 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 150 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 151 */
/* 152 */         } else {
/* 153 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 154 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8);
/* 155 */
/* 156 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 157 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 158 */               serializefromobject_arrayWriter.setNullDouble(serializefromobject_index);
/* 159 */             } else {
/* 160 */               final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index);
/* 161 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 162 */             }
/* 163 */           }
/* 164 */         }
/* 165 */
/* 166 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 167 */       }
/* 168 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 169 */       append(serializefromobject_result);
/* 170 */       if (shouldStop()) return;
/* 171 */     }
/* 172 */   }
```

With this PR (eliminated lines 56-62 in the above code)
```java
/* 047 */   protected void processNext() throws java.io.IOException {
/* 048 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 049 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 050 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 051 */       ArrayData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getArray(0));
/* 052 */
/* 053 */       boolean deserializetoobject_isNull = true;
/* 054 */       double[] deserializetoobject_value = null;
/* 055 */       if (!inputadapter_isNull) {
/* 056 */         deserializetoobject_isNull = false;
/* 057 */         if (!deserializetoobject_isNull) {
/* 058 */           Object deserializetoobject_funcResult = null;
/* 059 */           deserializetoobject_funcResult = inputadapter_value.toDoubleArray();
/* 060 */           if (deserializetoobject_funcResult == null) {
/* 061 */             deserializetoobject_isNull = true;
/* 062 */           } else {
/* 063 */             deserializetoobject_value = (double[]) deserializetoobject_funcResult;
/* 064 */           }
/* 065 */
/* 066 */         }
/* 067 */         deserializetoobject_isNull = deserializetoobject_value == null;
/* 068 */       }
/* 069 */
/* 070 */       boolean mapelements_isNull = true;
/* 071 */       double[] mapelements_value = null;
/* 072 */       if (!false) {
/* 073 */         mapelements_resultIsNull = false;
/* 074 */
/* 075 */         if (!mapelements_resultIsNull) {
/* 076 */           mapelements_resultIsNull = deserializetoobject_isNull;
/* 077 */           mapelements_argValue = deserializetoobject_value;
/* 078 */         }
/* 079 */
/* 080 */         mapelements_isNull = mapelements_resultIsNull;
/* 081 */         if (!mapelements_isNull) {
/* 082 */           Object mapelements_funcResult = null;
/* 083 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 084 */           if (mapelements_funcResult == null) {
/* 085 */             mapelements_isNull = true;
/* 086 */           } else {
/* 087 */             mapelements_value = (double[]) mapelements_funcResult;
/* 088 */           }
/* 089 */
/* 090 */         }
/* 091 */         mapelements_isNull = mapelements_value == null;
/* 092 */       }
/* 093 */
/* 094 */       serializefromobject_resultIsNull = false;
/* 095 */
/* 096 */       if (!serializefromobject_resultIsNull) {
/* 097 */         serializefromobject_resultIsNull = mapelements_isNull;
/* 098 */         serializefromobject_argValue = mapelements_value;
/* 099 */       }
/* 100 */
/* 101 */       boolean serializefromobject_isNull = serializefromobject_resultIsNull;
/* 102 */       final ArrayData serializefromobject_value = serializefromobject_resultIsNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(serializefromobject_argValue);
/* 103 */       serializefromobject_isNull = serializefromobject_value == null;
/* 104 */       serializefromobject_holder.reset();
/* 105 */
/* 106 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 107 */
/* 108 */       if (serializefromobject_isNull) {
/* 109 */         serializefromobject_rowWriter.setNullAt(0);
/* 110 */       } else {
/* 111 */         // Remember the current cursor so that we can calculate how many bytes are
/* 112 */         // written later.
/* 113 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 114 */
/* 115 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 116 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 117 */           // grow the global buffer before writing data.
/* 118 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 119 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 120 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 121 */
/* 122 */         } else {
/* 123 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 124 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 8);
/* 125 */
/* 126 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 127 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 128 */               serializefromobject_arrayWriter.setNullDouble(serializefromobject_index);
/* 129 */             } else {
/* 130 */               final double serializefromobject_element = serializefromobject_value.getDouble(serializefromobject_index);
/* 131 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 132 */             }
/* 133 */           }
/* 134 */         }
/* 135 */
/* 136 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 137 */       }
/* 138 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 139 */       append(serializefromobject_result);
/* 140 */       if (shouldStop()) return;
/* 141 */     }
/* 142 */   }
```

## How was this patch tested?

Add test suites into `DatasetPrimitiveSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #17568 from kiszk/SPARK-20254.
2017-04-19 10:58:05 +08:00
wangzhenhua 321b4f03bc [SPARK-20366][SQL] Fix recursive join reordering: inside joins are not reordered
## What changes were proposed in this pull request?

If a plan has multi-level successive joins, e.g.:
```
         Join
         /   \
     Union   t5
      /   \
    Join  t4
    /   \
  Join  t3
  /  \
 t1   t2
```
Currently we fail to reorder the inside joins, i.e. t1, t2, t3.

In join reorder, we use `OrderedJoin` to indicate a join has been ordered, such that when transforming down the plan, these joins don't need to be rerodered again.

But there's a problem in the definition of `OrderedJoin`:
The real join node is a parameter, but not a child. This breaks the transform procedure because `mapChildren` applies transform function on parameters which should be children.

In this patch, we change `OrderedJoin` to a class having the same structure as a join node.

## How was this patch tested?

Add a corresponding test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17668 from wzhfy/recursiveReorder.
2017-04-18 20:12:21 +08:00
Felix Cheung b0a1e93e93 [SPARK-17647][SQL][FOLLOWUP][MINOR] fix typo
## What changes were proposed in this pull request?

fix typo

## How was this patch tested?

manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17663 from felixcheung/likedoctypo.
2017-04-17 23:55:40 -07:00
Jacek Laskowski 33ea908af9 [TEST][MINOR] Replace repartitionBy with distribute in CollapseRepartitionSuite
## What changes were proposed in this pull request?

Replace non-existent `repartitionBy` with `distribute` in `CollapseRepartitionSuite`.

## How was this patch tested?

local build and `catalyst/testOnly *CollapseRepartitionSuite`

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17657 from jaceklaskowski/CollapseRepartitionSuite.
2017-04-17 17:58:10 -07:00
Jakob Odersky e5fee3e4f8 [SPARK-17647][SQL] Fix backslash escaping in 'LIKE' patterns.
## What changes were proposed in this pull request?

This patch fixes a bug in the way LIKE patterns are translated to Java regexes. The bug causes any character following an escaped backslash to be escaped, i.e. there is double-escaping.
A concrete example is the following pattern:`'%\\%'`. The expected Java regex that this pattern should correspond to (according to the behavior described below) is `'.*\\.*'`, however the current situation leads to `'.*\\%'` instead.

---

Update: in light of the discussion that ensued, we should explicitly define the expected behaviour of LIKE expressions, especially in certain edge cases. With the help of gatorsmile, we put together a list of different RDBMS and their variations wrt to certain standard features.

| RDBMS\Features | Wildcards | Default escape [1] | Case sensitivity |
| --- | --- | --- | --- |
| [MS SQL Server](https://msdn.microsoft.com/en-us/library/ms179859.aspx) | _, %, [], [^] | none | no |
| [Oracle](https://docs.oracle.com/cd/B12037_01/server.101/b10759/conditions016.htm) | _, % | none | yes |
| [DB2 z/OS](http://www.ibm.com/support/knowledgecenter/SSEPEK_11.0.0/sqlref/src/tpc/db2z_likepredicate.html) | _, % | none | yes |
| [MySQL](http://dev.mysql.com/doc/refman/5.7/en/string-comparison-functions.html) | _, % | none | no |
| [PostreSQL](https://www.postgresql.org/docs/9.0/static/functions-matching.html) | _, % | \ | yes |
| [Hive](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF) | _, % | none | yes |
| Current Spark | _, % | \ | yes |

[1] Default escape character: most systems do not have a default escape character, instead the user can specify one by calling a like expression with an escape argument [A] LIKE [B] ESCAPE [C]. This syntax is currently not supported by Spark, however I would volunteer to implement this feature in a separate ticket.

The specifications are often quite terse and certain scenarios are undocumented, so here is a list of scenarios that I am uncertain about and would appreciate any input. Specifically I am looking for feedback on whether or not Spark's current behavior should be changed.
1. [x] Ending a pattern with the escape sequence, e.g. `like 'a\'`.
   PostreSQL gives an error: 'LIKE pattern must not end with escape character', which I personally find logical. Currently, Spark allows "non-terminated" escapes and simply ignores them as part of the pattern.
   According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), ending a pattern in an escape character is invalid.
   _Proposed new behaviour in Spark: throw AnalysisException_
2. [x] Empty input, e.g. `'' like ''`
   Postgres and DB2 will match empty input only if the pattern is empty as well, any other combination of empty input will not match. Spark currently follows this rule.
3. [x] Escape before a non-special character, e.g. `'a' like '\a'`.
   Escaping a non-wildcard character is not really documented but PostgreSQL just treats it verbatim, which I also find the least surprising behavior. Spark does the same.
   According to [DB2's documentation](http://www.ibm.com/support/knowledgecenter/SSEPGG_9.7.0/com.ibm.db2.luw.messages.sql.doc/doc/msql00130n.html), it is invalid to follow an escape character with anything other than an escape character, an underscore or a percent sign.
   _Proposed new behaviour in Spark: throw AnalysisException_

The current specification is also described in the operator's source code in this patch.
## How was this patch tested?

Extra case in regex unit tests.

Author: Jakob Odersky <jakob@odersky.com>

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

Closes #15398 from jodersky/SPARK-17647.
2017-04-17 11:17:57 -07:00
Xiao Li 01ff0350a8 [SPARK-20349][SQL] ListFunctions returns duplicate functions after using persistent functions
### What changes were proposed in this pull request?
The session catalog caches some persistent functions in the `FunctionRegistry`, so there can be duplicates. Our Catalog API `listFunctions` does not handle it.

It would be better if `SessionCatalog` API can de-duplciate the records, instead of doing it by each API caller. In `FunctionRegistry`, our functions are identified by the unquoted string. Thus, this PR is try to parse it using our parser interface and then de-duplicate the names.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17646 from gatorsmile/showFunctions.
2017-04-17 09:50:20 -07:00
Xiao Li e090f3c0ce [SPARK-20335][SQL] Children expressions of Hive UDF impacts the determinism of Hive UDF
### What changes were proposed in this pull request?
```JAVA
  /**
   * Certain optimizations should not be applied if UDF is not deterministic.
   * Deterministic UDF returns same result each time it is invoked with a
   * particular input. This determinism just needs to hold within the context of
   * a query.
   *
   * return true if the UDF is deterministic
   */
  boolean deterministic() default true;
```

Based on the definition of [UDFType](https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFType.java#L42-L50), when Hive UDF's children are non-deterministic, Hive UDF is also non-deterministic.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17635 from gatorsmile/udfDeterministic.
2017-04-16 12:09:34 +08:00
Wenchen Fan 35e5ae4f81 [SPARK-19716][SQL][FOLLOW-UP] UnresolvedMapObjects should always be serializable
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/17398 we introduced `UnresolvedMapObjects` as a placeholder of `MapObjects`. Unfortunately `UnresolvedMapObjects` is not serializable as its `function` may reference Scala `Type` which is not serializable.

Ideally this is fine, as we will never serialize and send unresolved expressions to executors. However users may accidentally do this, e.g. mistakenly reference an encoder instance when implementing `Aggregator`, we should fix it so that it's just a performance issue(more network traffic) and should not fail the query.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17639 from cloud-fan/minor.
2017-04-16 11:14:18 +08:00
ouyangxiaochen 98b41ecbcb [SPARK-20316][SQL] Val and Var should strictly follow the Scala syntax
## What changes were proposed in this pull request?

val and var should strictly follow the Scala syntax

## How was this patch tested?

manual test and exisiting test cases

Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>

Closes #17628 from ouyangxiaochen/spark-413.
2017-04-15 10:34:57 +01:00