Commit graph

2500 commits

Author SHA1 Message Date
Parth Brahmbhatt 4acababcab [SPARK-15365][SQL] When table size statistics are not available from metastore, we should fallback to HDFS
## What changes were proposed in this pull request?
Currently if a table is used in join operation we rely on Metastore returned size to calculate if we can convert the operation to Broadcast join. This optimization only kicks in for table's that have the statistics available in metastore. Hive generally rolls over to HDFS if the statistics are not available directly from metastore and this seems like a reasonable choice to adopt given the optimization benefit of using broadcast joins.

## How was this patch tested?
I have executed queries locally to test.

Author: Parth Brahmbhatt <pbrahmbhatt@netflix.com>

Closes #13150 from Parth-Brahmbhatt/SPARK-15365.
2016-05-24 20:58:20 -07:00
Dongjoon Hyun f08bf587b1 [SPARK-15512][CORE] repartition(0) should raise IllegalArgumentException
## What changes were proposed in this pull request?

Previously, SPARK-8893 added the constraints on positive number of partitions for repartition/coalesce operations in general. This PR adds one missing part for that and adds explicit two testcases.

**Before**
```scala
scala> sc.parallelize(1 to 5).coalesce(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> sc.parallelize(1 to 5).repartition(0).collect()
res1: Array[Int] = Array()   // empty
scala> spark.sql("select 1").coalesce(0)
res2: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int]
scala> spark.sql("select 1").coalesce(0).collect()
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
scala> spark.sql("select 1").repartition(0)
res3: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int]
scala> spark.sql("select 1").repartition(0).collect()
res4: Array[org.apache.spark.sql.Row] = Array()  // empty
```

**After**
```scala
scala> sc.parallelize(1 to 5).coalesce(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> sc.parallelize(1 to 5).repartition(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> spark.sql("select 1").coalesce(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> spark.sql("select 1").repartition(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
```

## How was this patch tested?

Pass the Jenkins tests with new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13282 from dongjoon-hyun/SPARK-15512.
2016-05-24 18:55:23 -07:00
Tathagata Das e631b819fe [SPARK-15458][SQL][STREAMING] Disable schema inference for streaming datasets on file streams
## What changes were proposed in this pull request?

If the user relies on the schema to be inferred in file streams can break easily for multiple reasons
- accidentally running on a directory which has no data
- schema changing underneath
- on restart, the query will infer schema again, and may unexpectedly infer incorrect schema, as the file in the directory may be different at the time of the restart.

To avoid these complicated scenarios, for Spark 2.0, we are going to disable schema inferencing by default with a config, so that user is forced to consider explicitly what is the schema it wants, rather than the system trying to infer it and run into weird corner cases.

In this PR, I introduce a SQLConf that determines whether schema inference for file streams is allowed or not. It is disabled by default.

## How was this patch tested?
Updated unit tests that test error behavior with and without schema inference enabled.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #13238 from tdas/SPARK-15458.
2016-05-24 14:27:39 -07:00
Dongjoon Hyun f8763b80ec [SPARK-13135] [SQL] Don't print expressions recursively in generated code
## What changes were proposed in this pull request?

This PR is an up-to-date and a little bit improved version of #11019 of rxin for
- (1) preventing recursive printing of expressions in generated code.

Since the major function of this PR is indeed the above,  he should be credited for the work he did. In addition to #11019, this PR improves the followings in code generation.
- (2) Improve multiline comment indentation.
- (3) Reduce the number of empty lines (mainly consecutive empty lines).
- (4) Remove all space characters on empty lines.

**Example**
```scala
spark.range(1, 1000).select('id+1+2+3, 'id+4+5+6)
```

**Before**
```
Generated code:
/* 001 */ public Object generate(Object[] references) {
...
/* 005 */ /**
/* 006 */ * Codegend pipeline for
/* 007 */ * Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L]
/* 008 */ * +- Range 1, 1, 8, 999, [id#0L]
/* 009 */ */
...
/* 075 */     // PRODUCE: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L]
/* 076 */
/* 077 */     // PRODUCE: Range 1, 1, 8, 999, [id#0L]
/* 078 */
/* 079 */     // initialize Range
...
/* 092 */       // CONSUME: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L]
/* 093 */
/* 094 */       // CONSUME: WholeStageCodegen
/* 095 */
/* 096 */       // (((input[0, bigint, false] + 1) + 2) + 3)
/* 097 */       // ((input[0, bigint, false] + 1) + 2)
/* 098 */       // (input[0, bigint, false] + 1)
...
/* 107 */       // (((input[0, bigint, false] + 4) + 5) + 6)
/* 108 */       // ((input[0, bigint, false] + 4) + 5)
/* 109 */       // (input[0, bigint, false] + 4)
...
/* 126 */ }
```

**After**
```
Generated code:
/* 001 */ public Object generate(Object[] references) {
...
/* 005 */ /**
/* 006 */  * Codegend pipeline for
/* 007 */  * Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L]
/* 008 */  * +- Range 1, 1, 8, 999, [id#0L]
/* 009 */  */
...
/* 075 */     // PRODUCE: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L]
/* 076 */     // PRODUCE: Range 1, 1, 8, 999, [id#0L]
/* 077 */     // initialize Range
...
/* 090 */       // CONSUME: Project [(((id#0L + 1) + 2) + 3) AS (((id + 1) + 2) + 3)#3L,(((id#0L + 4) + 5) + 6) AS (((id + 4) + 5) + 6)#4L]
/* 091 */       // CONSUME: WholeStageCodegen
/* 092 */       // (((input[0, bigint, false] + 1) + 2) + 3)
...
/* 101 */       // (((input[0, bigint, false] + 4) + 5) + 6)
...
/* 118 */ }
```

## How was this patch tested?

Pass the Jenkins tests and see the result of the following command manually.
```scala
scala> spark.range(1, 1000).select('id+1+2+3, 'id+4+5+6).queryExecution.debug.codegen()
```

Author: Dongjoon Hyun <dongjoonapache.org>
Author: Reynold Xin <rxindatabricks.com>

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13192 from dongjoon-hyun/SPARK-13135.
2016-05-24 10:08:14 -07:00
Liang-Chi Hsieh c24b6b679c [SPARK-11753][SQL][TEST-HADOOP2.2] Make allowNonNumericNumbers option work
## What changes were proposed in this pull request?

Jackson suppprts `allowNonNumericNumbers` option to parse non-standard non-numeric numbers such as "NaN", "Infinity", "INF".  Currently used Jackson version (2.5.3) doesn't support it all. This patch upgrades the library and make the two ignored tests in `JsonParsingOptionsSuite` passed.

## How was this patch tested?

`JsonParsingOptionsSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #9759 from viirya/fix-json-nonnumric.
2016-05-24 09:43:39 -07:00
Daoyuan Wang d642b27354 [SPARK-15397][SQL] fix string udf locate as hive
## What changes were proposed in this pull request?

in hive, `locate("aa", "aaa", 0)` would yield 0, `locate("aa", "aaa", 1)` would yield 1 and `locate("aa", "aaa", 2)` would yield 2, while in Spark, `locate("aa", "aaa", 0)` would yield 1,  `locate("aa", "aaa", 1)` would yield 2 and  `locate("aa", "aaa", 2)` would yield 0. This results from the different understanding of the third parameter in udf `locate`. It means the starting index and starts from 1, so when we use 0, the return would always be 0.

## How was this patch tested?

tested with modified `StringExpressionsSuite` and `StringFunctionsSuite`

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #13186 from adrian-wang/locate.
2016-05-23 23:29:15 -07:00
Andrew Or de726b0d53 Revert "[SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB"
This reverts commit fa244e5a90.
2016-05-23 21:43:11 -07:00
Kazuaki Ishizaki fa244e5a90 [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows beyond 64 KB
## What changes were proposed in this pull request?

This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method.

## How was this patch tested?

Added new tests

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

Closes #13243 from kiszk/SPARK-15285.
2016-05-23 21:12:34 -07:00
gatorsmile 5afd927a47 [SPARK-15311][SQL] Disallow DML on Regular Tables when Using In-Memory Catalog
#### What changes were proposed in this pull request?
So far, when using In-Memory Catalog, we allow DDL operations for the tables. However, the corresponding DML operations are not supported for the tables that are neither temporary nor data source tables. For example,
```SQL
CREATE TABLE tabName(i INT, j STRING)
SELECT * FROM tabName
INSERT OVERWRITE TABLE tabName SELECT 1, 'a'
```
In the above example, before this PR fix, we will get very confusing exception messages for either `SELECT` or `INSERT`
```
org.apache.spark.sql.AnalysisException: unresolved operator 'SimpleCatalogRelation default, CatalogTable(`default`.`tbl`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(i,int,true,None), CatalogColumn(j,string,true,None)),List(),List(),List(),-1,,1463928681802,-1,Map(),None,None,None,List()), None;
```

This PR is to issue appropriate exceptions in this case. The message will be like
```
org.apache.spark.sql.AnalysisException: Please enable Hive support when operating non-temporary tables: `tbl`;
```
#### How was this patch tested?
Added a test case in `DDLSuite`.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #13093 from gatorsmile/selectAfterCreate.
2016-05-23 18:03:45 -07:00
Xin Wu 01659bc50c [SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively
## What changes were proposed in this pull request?
Currently command `ADD FILE|JAR <filepath | jarpath>` is supported natively in SparkSQL. However, when this command is run, the file/jar is added to the resources that can not be looked up by `LIST FILE(s)|JAR(s)` command because the `LIST` command is passed to Hive command processor in Spark-SQL or simply not supported in Spark-shell. There is no way users can find out what files/jars are added to the spark context.
Refer to [Hive commands](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli)

This PR is to support following commands:
`LIST (FILE[s] [filepath ...] | JAR[s] [jarfile ...])`

### For example:
##### LIST FILE(s)
```
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt")
res1: org.apache.spark.sql.DataFrame = []
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt")
res2: org.apache.spark.sql.DataFrame = []

scala> spark.sql("list file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt").show(false)
+----------------------------------------------+
|result                                        |
+----------------------------------------------+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
+----------------------------------------------+

scala> spark.sql("list files").show(false)
+----------------------------------------------+
|result                                        |
+----------------------------------------------+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt |
+----------------------------------------------+
```

##### LIST JAR(s)
```
scala> spark.sql("add jar /Users/xinwu/spark/core/src/test/resources/TestUDTF.jar")
res9: org.apache.spark.sql.DataFrame = [result: int]

scala> spark.sql("list jar TestUDTF.jar").show(false)
+---------------------------------------------+
|result                                       |
+---------------------------------------------+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+---------------------------------------------+

scala> spark.sql("list jars").show(false)
+---------------------------------------------+
|result                                       |
+---------------------------------------------+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+---------------------------------------------+
```
## How was this patch tested?
New test cases are added for Spark-SQL, Spark-Shell and SparkContext API code path.

Author: Xin Wu <xinwu@us.ibm.com>
Author: xin Wu <xinwu@us.ibm.com>

Closes #13212 from xwu0226/list_command.
2016-05-23 17:32:01 -07:00
sureshthalamati 03c7b7c4b9 [SPARK-15315][SQL] Adding error check to the CSV datasource writer for unsupported complex data types.
## What changes were proposed in this pull request?

Adds error handling to the CSV writer  for unsupported complex data types.  Currently garbage gets written to the output csv files if the data frame schema has complex data types.

## How was this patch tested?

Added new unit test case.

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #13105 from sureshthalamati/csv_complex_types_SPARK-15315.
2016-05-23 17:15:19 -07:00
Dongjoon Hyun 37c617e4f5 [MINOR][SQL][DOCS] Add notes of the deterministic assumption on UDF functions
## What changes were proposed in this pull request?

Spark assumes that UDF functions are deterministic. This PR adds explicit notes about that.

## How was this patch tested?

It's only about docs.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13087 from dongjoon-hyun/SPARK-15282.
2016-05-23 14:19:25 -07:00
Andrew Or 2585d2b322 [SPARK-15279][SQL] Catch conflicting SerDe when creating table
## What changes were proposed in this pull request?

The user may do something like:
```
CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS PARQUET
CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS ... SERDE 'myserde'
CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ORC
CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ... SERDE 'myserde'
```
None of these should be allowed because the SerDe's conflict. As of this patch:
- `ROW FORMAT DELIMITED` is only compatible with `TEXTFILE`
- `ROW FORMAT SERDE` is only compatible with `TEXTFILE`, `RCFILE` and `SEQUENCEFILE`

## How was this patch tested?

New tests in `DDLCommandSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #13068 from andrewor14/row-format-conflict.
2016-05-23 11:55:03 -07:00
Davies Liu 80091b8a68 [SPARK-14031][SQL] speedup CSV writer
## What changes were proposed in this pull request?

Currently, we create an CSVWriter for every row, it's very expensive and memory hungry, took about 15 seconds to write out 1 mm rows (two columns).

This PR will write the rows in batch mode, create a CSVWriter for every 1k rows, which could write out 1 mm rows in about 1 seconds (15X faster).

## How was this patch tested?

Manually benchmark it.

Author: Davies Liu <davies@databricks.com>

Closes #13229 from davies/csv_writer.
2016-05-23 10:48:25 -07:00
Sameer Agarwal dafcb05c2e [SPARK-15425][SQL] Disallow cross joins by default
## What changes were proposed in this pull request?

In order to prevent users from inadvertently writing queries with cartesian joins, this patch introduces a new conf `spark.sql.crossJoin.enabled` (set to `false` by default) that if not set, results in a `SparkException` if the query contains one or more cartesian products.

## How was this patch tested?

Added a test to verify the new behavior in `JoinSuite`. Additionally, `SQLQuerySuite` and `SQLMetricsSuite` were modified to explicitly enable cartesian products.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13209 from sameeragarwal/disallow-cartesian.
2016-05-22 23:32:39 -07:00
Tathagata Das 1ffa608ba5 [SPARK-15428][SQL] Disable multiple streaming aggregations
## What changes were proposed in this pull request?

Incrementalizing plans of with multiple streaming aggregation is tricky and we dont have the necessary support for "delta" to implement correctly. So disabling the support for multiple streaming aggregations.

## How was this patch tested?
Additional unit tests

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #13210 from tdas/SPARK-15428.
2016-05-22 02:08:18 -07:00
Reynold Xin 845e447fa0 [SPARK-15459][SQL] Make Range logical and physical explain consistent
## What changes were proposed in this pull request?
This patch simplifies the implementation of Range operator and make the explain string consistent between logical plan and physical plan. To do this, I changed RangeExec to embed a Range logical plan in it.

Before this patch (note that the logical Range and physical Range actually output different information):
```
== Optimized Logical Plan ==
Range 0, 100, 2, 2, [id#8L]

== Physical Plan ==
*Range 0, 2, 2, 50, [id#8L]
```

After this patch:
If step size is 1:
```
== Optimized Logical Plan ==
Range(0, 100, splits=2)

== Physical Plan ==
*Range(0, 100, splits=2)
```

If step size is not 1:
```
== Optimized Logical Plan ==
Range (0, 100, step=2, splits=2)

== Physical Plan ==
*Range (0, 100, step=2, splits=2)
```

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #13239 from rxin/SPARK-15459.
2016-05-22 00:03:37 -07:00
gatorsmile a11175eeca [SPARK-15312][SQL] Detect Duplicate Key in Partition Spec and Table Properties
#### What changes were proposed in this pull request?
When there are duplicate keys in the partition specs or table properties, we always use the last value and ignore all the previous values. This is caused by the function call `toMap`.

partition specs or table properties are widely used in multiple DDL statements.

This PR is to detect the duplicates and issue an exception if found.

#### How was this patch tested?
Added test cases in DDLSuite

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13095 from gatorsmile/detectDuplicate.
2016-05-21 23:56:10 -07:00
Reynold Xin 6d0bfb9601 Small documentation and style fix. 2016-05-21 23:12:56 -07:00
Jurriaan Pruis 223f633908 [SPARK-15415][SQL] Fix BroadcastHint when autoBroadcastJoinThreshold is 0 or -1
## What changes were proposed in this pull request?

This PR makes BroadcastHint more deterministic by using a special isBroadcastable property
instead of setting the sizeInBytes to 1.

See https://issues.apache.org/jira/browse/SPARK-15415

## How was this patch tested?

Added testcases to test if the broadcast hash join is included in the plan when the BroadcastHint is supplied and also tests for propagation of the joins.

Author: Jurriaan Pruis <email@jurriaanpruis.nl>

Closes #13244 from jurriaan/broadcast-hint.
2016-05-21 23:01:14 -07:00
gatorsmile 8f0a3d5bcb [SPARK-15330][SQL] Implement Reset Command
#### What changes were proposed in this pull request?
Like `Set` Command in Hive, `Reset` is also supported by Hive. See the link: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli

Below is the related Hive JIRA: https://issues.apache.org/jira/browse/HIVE-3202

This PR is to implement such a command for resetting the SQL-related configuration to the default values. One of the use case shown in HIVE-3202 is listed below:

> For the purpose of optimization we set various configs per query. It's worthy but all those configs should be reset every time for next query.

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

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #13121 from gatorsmile/resetCommand.
2016-05-21 20:07:34 -07:00
Reynold Xin 201a51f366 [SPARK-15452][SQL] Mark aggregator API as experimental
## What changes were proposed in this pull request?
The Aggregator API was introduced in 2.0 for Dataset. All typed Dataset APIs should still be marked as experimental in 2.0.

## How was this patch tested?
N/A - annotation only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #13226 from rxin/SPARK-15452.
2016-05-21 12:46:25 -07:00
Dilip Biswal 5e1ee28984 [SPARK-15114][SQL] Column name generated by typed aggregate is super verbose
## What changes were proposed in this pull request?

Generate a shorter default alias for `AggregateExpression `, In this PR, aggregate function name along with a index is used for generating the alias name.

```SQL
val ds = Seq(1, 3, 2, 5).toDS()
ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i)).show()
```

Output before change.
```SQL
+-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+
|typedsumdouble(unresolveddeserializer(upcast(input[0, int], IntegerType, - root class: "scala.Int"), value#1), upcast(value))|typedaverage(unresolveddeserializer(upcast(input[0, int], IntegerType, - root class: "scala.Int"), value#1), newInstance(class scala.Tuple2))|
+-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+
|                                                                                                                         11.0|                                                                                                                                         2.75|
+-----------------------------------------------------------------------------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------+
```
Output after change:
```SQL
+-----------------+---------------+
|typedsumdouble_c1|typedaverage_c2|
+-----------------+---------------+
|             11.0|           2.75|
+-----------------+---------------+
```

Note: There is one test in ParquetSuites.scala which shows that that the system picked alias
name is not usable and is rejected.  [test](https://github.com/apache/spark/blob/master/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala#L672-#L687)
## How was this patch tested?

A new test was added in DataSetAggregatorSuite.

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

Closes #13045 from dilipbiswal/spark-15114.
2016-05-21 08:36:08 -07:00
Zheng RuiFeng 127bf1bb07 [SPARK-15031][EXAMPLE] Use SparkSession in examples
## What changes were proposed in this pull request?
Use `SparkSession` according to [SPARK-15031](https://issues.apache.org/jira/browse/SPARK-15031)

`MLLLIB` is not recommended to use now, so examples in `MLLIB` are ignored in this PR.
`StreamingContext` can not be directly obtained from `SparkSession`, so example in `Streaming` are ignored too.

cc andrewor14

## How was this patch tested?
manual tests with spark-submit

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #13164 from zhengruifeng/use_sparksession_ii.
2016-05-20 16:40:33 -07:00
Sameer Agarwal a78d6ce376 [SPARK-15078] [SQL] Add all TPCDS 1.4 benchmark queries for SparkSQL
## What changes were proposed in this pull request?

Now that SparkSQL supports all TPC-DS queries, this patch adds all 99 benchmark queries inside SparkSQL.

## How was this patch tested?

Benchmark only

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13188 from sameeragarwal/tpcds-all.
2016-05-20 15:19:28 -07:00
Reynold Xin dcac8e6f49 [SPARK-15454][SQL] Filter out files starting with _
## What changes were proposed in this pull request?
Many other systems (e.g. Impala) uses _xxx as staging, and Spark should not be reading those files.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #13227 from rxin/SPARK-15454.
2016-05-20 14:49:54 -07:00
Davies Liu 0e70fd61b4 [SPARK-15438][SQL] improve explain of whole stage codegen
## What changes were proposed in this pull request?

Currently, the explain of a query with whole-stage codegen looks like this
```
>>> df = sqlCtx.range(1000);df2 = sqlCtx.range(1000);df.join(pyspark.sql.functions.broadcast(df2), 'id').explain()
== Physical Plan ==
WholeStageCodegen
:  +- Project [id#1L]
:     +- BroadcastHashJoin [id#1L], [id#4L], Inner, BuildRight, None
:        :- Range 0, 1, 4, 1000, [id#1L]
:        +- INPUT
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint]))
   +- WholeStageCodegen
      :  +- Range 0, 1, 4, 1000, [id#4L]
```

The problem is that the plan looks much different than logical plan, make us hard to understand the plan (especially when the logical plan is not showed together).

This PR will change it to:

```
>>> df = sqlCtx.range(1000);df2 = sqlCtx.range(1000);df.join(pyspark.sql.functions.broadcast(df2), 'id').explain()
== Physical Plan ==
*Project [id#0L]
+- *BroadcastHashJoin [id#0L], [id#3L], Inner, BuildRight, None
   :- *Range 0, 1, 4, 1000, [id#0L]
   +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, false]))
      +- *Range 0, 1, 4, 1000, [id#3L]
```

The `*`before the plan means that it's part of whole-stage codegen, it's easy to understand.

## How was this patch tested?

Manually ran some queries and check the explain.

Author: Davies Liu <davies@databricks.com>

Closes #13204 from davies/explain_codegen.
2016-05-20 13:21:53 -07:00
Michael Armbrust 2ba3ff0449 [SPARK-10216][SQL] Revert "[] Avoid creating empty files during overwrit…
This reverts commit 8d05a7a from #12855, which seems to have caused regressions when working with empty DataFrames.

Author: Michael Armbrust <michael@databricks.com>

Closes #13181 from marmbrus/revert12855.
2016-05-20 13:00:29 -07:00
Kousuke Saruta 22947cd021 [SPARK-15165] [SPARK-15205] [SQL] Introduce place holder for comments in generated code
## What changes were proposed in this pull request?

This PR introduce place holder for comment in generated code and the purpose  is same for #12939 but much safer.

Generated code to be compiled doesn't include actual comments but includes place holder instead.

Place holders in generated code will be replaced with actual comments only at the time of  logging.

Also, this PR can resolve SPARK-15205.

## How was this patch tested?

Existing tests.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #12979 from sarutak/SPARK-15205.
2016-05-20 10:56:35 -07:00
Davies Liu 5a25cd4ff3 [HOTFIX] disable stress test 2016-05-20 10:44:26 -07:00
Reynold Xin e8adc552df [SPARK-15435][SQL] Append Command to all commands
## What changes were proposed in this pull request?
We started this convention to append Command suffix to all SQL commands. However, not all commands follow that convention. This patch adds Command suffix to all RunnableCommands.

## How was this patch tested?
Updated test cases to reflect the renames.

Author: Reynold Xin <rxin@databricks.com>

Closes #13215 from rxin/SPARK-15435.
2016-05-20 09:36:14 -07:00
Andrew Or 2573750192 [SPARK-15421][SQL] Validate DDL property values
## What changes were proposed in this pull request?

When we parse DDLs involving table or database properties, we need to validate the values.
E.g. if we alter a database's property without providing a value:
```
ALTER DATABASE my_db SET DBPROPERTIES('some_key')
```
Then we'll ignore it with Hive, but override the property with the in-memory catalog. Inconsistencies like these arise because we don't validate the property values.

In such cases, we should throw exceptions instead.

## How was this patch tested?

`DDLCommandSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #13205 from andrewor14/ddl-prop-values.
2016-05-19 23:43:01 -07:00
gatorsmile 39fd469078 [SPARK-15367][SQL] Add refreshTable back
#### What changes were proposed in this pull request?
`refreshTable` was a method in `HiveContext`. It was deleted accidentally while we were migrating the APIs. This PR is to add it back to `HiveContext`.

In addition, in `SparkSession`, we put it under the catalog namespace (`SparkSession.catalog.refreshTable`).

#### How was this patch tested?
Changed the existing test cases to use the function `refreshTable`. Also added a test case for refreshTable in `hivecontext-compatibility`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13156 from gatorsmile/refreshTable.
2016-05-20 14:38:25 +08:00
Lianhui Wang 09a00510c4 [SPARK-15335][SQL] Implement TRUNCATE TABLE Command
## What changes were proposed in this pull request?

Like TRUNCATE TABLE Command in Hive, TRUNCATE TABLE is also supported by Hive. See the link: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
Below is the related Hive JIRA: https://issues.apache.org/jira/browse/HIVE-446
This PR is to implement such a command for truncate table excluded column truncation(HIVE-4005).

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

Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #13170 from lianhuiwang/truncate.
2016-05-19 23:03:59 -07:00
Takuya UESHIN d5e1c5acde [SPARK-15313][SQL] EmbedSerializerInFilter rule should keep exprIds of output of surrounded SerializeFromObject.
## What changes were proposed in this pull request?

The following code:

```
val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
ds.filter(_._1 == "b").select(expr("_1").as[String]).foreach(println(_))
```

throws an Exception:

```
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: _1#420
 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50)
 at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88)
 at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87)

...
 Cause: java.lang.RuntimeException: Couldn't find _1#420 in [_1#416,_2#417]
 at scala.sys.package$.error(package.scala:27)
 at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:94)
 at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:88)
 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49)
 at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:88)
 at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:87)
...
```

This is because `EmbedSerializerInFilter` rule drops the `exprId`s of output of surrounded `SerializeFromObject`.

The analyzed and optimized plans of the above example are as follows:

```
== Analyzed Logical Plan ==
_1: string
Project [_1#420]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421]
   +- Filter <function1>.apply
      +- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2
         +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]

== Optimized Logical Plan ==
!Project [_1#420]
+- Filter <function1>.apply
   +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]
```

This PR fixes `EmbedSerializerInFilter` rule to keep `exprId`s of output of surrounded `SerializeFromObject`.

The plans after this patch are as follows:

```
== Analyzed Logical Plan ==
_1: string
Project [_1#420]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, scala.Tuple2]._1, true) AS _1#420,input[0, scala.Tuple2]._2 AS _2#421]
   +- Filter <function1>.apply
      +- DeserializeToObject newInstance(class scala.Tuple2), obj#419: scala.Tuple2
         +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]

== Optimized Logical Plan ==
Project [_1#416]
+- Filter <function1>.apply
   +- LocalRelation [_1#416,_2#417], [[0,1800000001,1,61],[0,1800000001,2,62],[0,1800000001,3,63]]
```

## How was this patch tested?

Existing tests and I added a test to check if `filter and then select` works.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #13096 from ueshin/issues/SPARK-15313.
2016-05-19 22:55:44 -07:00
Reynold Xin f2ee0ed4b7 [SPARK-15075][SPARK-15345][SQL] Clean up SparkSession builder and propagate config options to existing sessions if specified
## What changes were proposed in this pull request?
Currently SparkSession.Builder use SQLContext.getOrCreate. It should probably the the other way around, i.e. all the core logic goes in SparkSession, and SQLContext just calls that. This patch does that.

This patch also makes sure config options specified in the builder are propagated to the existing (and of course the new) SparkSession.

## How was this patch tested?
Updated tests to reflect the change, and also introduced a new SparkSessionBuilderSuite that should cover all the branches.

Author: Reynold Xin <rxin@databricks.com>

Closes #13200 from rxin/SPARK-15075.
2016-05-19 21:53:26 -07:00
Kevin Yu 17591d90e6 [SPARK-11827][SQL] Adding java.math.BigInteger support in Java type inference for POJOs and Java collections
Hello : Can you help check this PR? I am adding support for the java.math.BigInteger for java bean code path. I saw internally spark is converting the BigInteger to BigDecimal in ColumnType.scala and CatalystRowConverter.scala. I use the similar way and convert the BigInteger to the BigDecimal. .

Author: Kevin Yu <qyu@us.ibm.com>

Closes #10125 from kevinyu98/working_on_spark-11827.
2016-05-20 12:41:14 +08:00
Shixiong Zhu 16ba71aba4 [SPARK-15416][SQL] Display a better message for not finding classes removed in Spark 2.0
## What changes were proposed in this pull request?

If finding `NoClassDefFoundError` or `ClassNotFoundException`, check if the class name is removed in Spark 2.0. If so, the user must be using an incompatible library and we can provide a better message.

## How was this patch tested?

1. Run `bin/pyspark --packages com.databricks:spark-avro_2.10:2.0.1`
2. type `sqlContext.read.format("com.databricks.spark.avro").load("src/test/resources/episodes.avro")`.

It will show `java.lang.ClassNotFoundException: org.apache.spark.sql.sources.HadoopFsRelationProvider is removed in Spark 2.0. Please check if your library is compatible with Spark 2.0`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13201 from zsxwing/better-message.
2016-05-19 18:31:05 -07:00
jerryshao dcf407de67 [SPARK-15375][SQL][STREAMING] Add ConsoleSink to structure streaming
## What changes were proposed in this pull request?

Add ConsoleSink to structure streaming, user could use it to display dataframes on the console (useful for debugging and demostrating), similar to the functionality of `DStream#print`, to use it:

```
    val query = result.write
      .format("console")
      .trigger(ProcessingTime("2 seconds"))
      .startStream()
```

## How was this patch tested?

local verified.

Not sure it is suitable to add into structure streaming, please review and help to comment, thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #13162 from jerryshao/SPARK-15375.
2016-05-19 17:42:59 -07:00
Davies Liu 5ccecc078a [SPARK-15392][SQL] fix default value of size estimation of logical plan
## What changes were proposed in this pull request?

We use autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD.

This PR change the default value to Long.MaxValue.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #13183 from davies/fix_default_size.
2016-05-19 12:12:42 -07:00
Shixiong Zhu 4e3cb7a5d9 [SPARK-15317][CORE] Don't store accumulators for every task in listeners
## What changes were proposed in this pull request?

In general, the Web UI doesn't need to store the Accumulator/AccumulableInfo for every task. It only needs the Accumulator values.

In this PR, it creates new UIData classes to store the necessary fields and make `JobProgressListener` store only these new classes, so that `JobProgressListener` won't store Accumulator/AccumulableInfo and the size of `JobProgressListener` becomes pretty small. I also eliminates `AccumulableInfo` from `SQLListener` so that we don't keep any references for those unused `AccumulableInfo`s.

## How was this patch tested?

I ran two tests reported in JIRA locally:

The first one is:
```
val data = spark.range(0, 10000, 1, 10000)
data.cache().count()
```
The retained size of JobProgressListener decreases from 60.7M to 6.9M.

The second one is:
```
import org.apache.spark.ml.CC
import org.apache.spark.sql.SQLContext
val sqlContext = SQLContext.getOrCreate(sc)
CC.runTest(sqlContext)
```

This test won't cause OOM after applying this patch.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13153 from zsxwing/memory.
2016-05-19 12:05:17 -07:00
Cheng Lian 6ac1c3a040 [SPARK-14346][SQL] Lists unsupported Hive features in SHOW CREATE TABLE output
## What changes were proposed in this pull request?

This PR is a follow-up of #13079. It replaces `hasUnsupportedFeatures: Boolean` in `CatalogTable` with `unsupportedFeatures: Seq[String]`, which contains unsupported Hive features of the underlying Hive table. In this way, we can accurately report all unsupported Hive features in the exception message.

## How was this patch tested?

Updated existing test case to check exception message.

Author: Cheng Lian <lian@databricks.com>

Closes #13173 from liancheng/spark-14346-follow-up.
2016-05-19 12:02:41 -07:00
hyukjinkwon f5065abf49 [SPARK-15322][SQL][FOLLOW-UP] Update deprecated accumulator usage into accumulatorV2
## What changes were proposed in this pull request?

This PR corrects another case that uses deprecated `accumulableCollection` to use `listAccumulator`, which seems the previous PR missed.

Since `ArrayBuffer[InternalRow].asJava` is `java.util.List[InternalRow]`, it seems ok to replace the usage.

## How was this patch tested?

Related existing tests `InMemoryColumnarQuerySuite` and `CachedTableSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13187 from HyukjinKwon/SPARK-15322.
2016-05-19 11:54:50 -07:00
Davies Liu 9308bf1192 [SPARK-15390] fix broadcast with 100 millions rows
## What changes were proposed in this pull request?

When broadcast a table with more than 100 millions rows (should not ideally), the size of needed memory will overflow.

This PR fix the overflow by converting it to Long when calculating the size of memory.

Also add more checking in broadcast to show reasonable messages.

## How was this patch tested?

Add test.

Author: Davies Liu <davies@databricks.com>

Closes #13182 from davies/fix_broadcast.
2016-05-19 11:45:18 -07:00
Dongjoon Hyun 5907ebfc11 [SPARK-14939][SQL] Add FoldablePropagation optimizer
## What changes were proposed in this pull request?

This PR aims to add new **FoldablePropagation** optimizer that propagates foldable expressions by replacing all attributes with the aliases of original foldable expression. Other optimizations will take advantage of the propagated foldable expressions: e.g. `EliminateSorts` optimizer now can handle the following Case 2 and 3. (Case 1 is the previous implementation.)

1. Literals and foldable expression, e.g. "ORDER BY 1.0, 'abc', Now()"
2. Foldable ordinals, e.g. "SELECT 1.0, 'abc', Now() ORDER BY 1, 2, 3"
3. Foldable aliases, e.g. "SELECT 1.0 x, 'abc' y, Now() z ORDER BY x, y, z"

This PR has been generalized based on cloud-fan 's key ideas many times; he should be credited for the work he did.

**Before**
```
scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain
== Physical Plan ==
WholeStageCodegen
:  +- Sort [1.0#5 ASC,x#0 ASC], true, 0
:     +- INPUT
+- Exchange rangepartitioning(1.0#5 ASC, x#0 ASC, 200), None
   +- WholeStageCodegen
      :  +- Project [1.0 AS 1.0#5,1461873043577000 AS x#0]
      :     +- INPUT
      +- Scan OneRowRelation[]
```

**After**
```
scala> sql("SELECT 1.0, Now() x ORDER BY 1, x").explain
== Physical Plan ==
WholeStageCodegen
:  +- Project [1.0 AS 1.0#5,1461873079484000 AS x#0]
:     +- INPUT
+- Scan OneRowRelation[]
```

## How was this patch tested?

Pass the Jenkins tests including a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12719 from dongjoon-hyun/SPARK-14939.
2016-05-19 15:57:44 +08:00
Wenchen Fan 661c21049b [SPARK-15381] [SQL] physical object operator should define reference correctly
## What changes were proposed in this pull request?

Whole Stage Codegen depends on `SparkPlan.reference` to do some optimization. For physical object operators, they should be consistent with their logical version and set the `reference` correctly.

## How was this patch tested?

new test in DatasetSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13167 from cloud-fan/bug.
2016-05-18 21:43:07 -07:00
Reynold Xin 4987f39ac7 [SPARK-14463][SQL] Document the semantics for read.text
## What changes were proposed in this pull request?
This patch is a follow-up to https://github.com/apache/spark/pull/13104 and adds documentation to clarify the semantics of read.text with respect to partitioning.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #13184 from rxin/SPARK-14463.
2016-05-18 19:16:28 -07:00
gatorsmile 9c2a376e41 [SPARK-15297][SQL] Fix Set -V Command
#### What changes were proposed in this pull request?
The command `SET -v` always outputs the default values even if we set the parameter. This behavior is incorrect. Instead, if users override it, we should output the user-specified value.

In addition, the output schema of `SET -v` is wrong. We should use the column `value` instead of `default` for the parameter value.

This PR is to fix the above two issues.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13081 from gatorsmile/setVcommand.
2016-05-19 10:05:53 +08:00
Wenchen Fan ebfe3a1f2c [SPARK-15192][SQL] null check for SparkSession.createDataFrame
## What changes were proposed in this pull request?

This PR adds null check in `SparkSession.createDataFrame`, so that we can make sure the passed in rows matches the given schema.

## How was this patch tested?

new tests in `DatasetSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13008 from cloud-fan/row-encoder.
2016-05-18 18:06:38 -07:00
Jurriaan Pruis 32be51fba4 [SPARK-15323][SPARK-14463][SQL] Fix reading of partitioned format=text datasets
https://issues.apache.org/jira/browse/SPARK-15323

I was using partitioned text datasets in Spark 1.6.1 but it broke in Spark 2.0.0.

It would be logical if you could also write those,
but not entirely sure how to solve this with the new DataSet implementation.

Also it doesn't work using `sqlContext.read.text`, since that method returns a `DataSet[String]`.
See https://issues.apache.org/jira/browse/SPARK-14463 for that issue.

Author: Jurriaan Pruis <email@jurriaanpruis.nl>

Closes #13104 from jurriaan/fix-partitioned-text-reads.
2016-05-18 16:15:09 -07:00
Davies Liu 84b23453dd Revert "[SPARK-15392][SQL] fix default value of size estimation of logical plan"
This reverts commit fc29b896da.
2016-05-18 16:02:52 -07:00
Davies Liu fc29b896da [SPARK-15392][SQL] fix default value of size estimation of logical plan
## What changes were proposed in this pull request?

We use  autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD.

This PR change the default value to Long.MaxValue.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #13179 from davies/fix_default_size.
2016-05-18 15:45:59 -07:00
Davies Liu 8fb1d1c7f3 [SPARK-15357] Cooperative spilling should check consumer memory mode
## What changes were proposed in this pull request?

Since we support forced spilling for Spillable, which only works in OnHeap mode, different from other SQL operators (could be OnHeap or OffHeap), we should considering the mode of consumer before calling trigger forced spilling.

## How was this patch tested?

Add new test.

Author: Davies Liu <davies@databricks.com>

Closes #13151 from davies/fix_mode.
2016-05-18 09:44:21 -07:00
WeichenXu 2f9047b5eb [SPARK-15322][MLLIB][CORE][SQL] update deprecate accumulator usage into accumulatorV2 in spark project
## What changes were proposed in this pull request?

I use Intellj-IDEA to search usage of deprecate SparkContext.accumulator in the whole spark project, and update the code.(except those test code for accumulator method itself)

## How was this patch tested?

Exisiting unit tests

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #13112 from WeichenXu123/update_accuV2_in_mllib.
2016-05-18 11:48:46 +01:00
Davies Liu 33814f887a [SPARK-15307][SQL] speed up listing files for data source
## What changes were proposed in this pull request?

Currently, listing files is very slow if there is thousands files, especially on local file system, because:
1) FileStatus.getPermission() is very slow on local file system, which is launch a subprocess and parse the stdout.
2) Create an JobConf is very expensive (ClassUtil.findContainingJar() is slow).

This PR improve these by:
1) Use another constructor of LocatedFileStatus to avoid calling FileStatus.getPermission, the permissions are not used for data sources.
2) Only create an JobConf once within one task.

## How was this patch tested?

Manually tests on a partitioned table with 1828 partitions, decrease the time to load the table from 22 seconds to 1.6 seconds (Most of time are spent in merging schema now).

Author: Davies Liu <davies@databricks.com>

Closes #13094 from davies/listing.
2016-05-18 18:46:57 +08:00
Sean Zhong 25b315e6ca [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable
## What changes were proposed in this pull request?

Update the unit test code, examples, and documents to remove calls to deprecated method `dataset.registerTempTable`.

## How was this patch tested?

This PR only changes the unit test code, examples, and comments. It should be safe.
This is a follow up of PR https://github.com/apache/spark/pull/12945 which was merged.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13098 from clockfly/spark-15171-remove-deprecation.
2016-05-18 09:01:59 +08:00
Cheng Lian b674e67c22 [SPARK-14346][SQL] Native SHOW CREATE TABLE for Hive tables/views
## What changes were proposed in this pull request?

This is a follow-up of #12781. It adds native `SHOW CREATE TABLE` support for Hive tables and views. A new field `hasUnsupportedFeatures` is added to `CatalogTable` to indicate whether all table metadata retrieved from the concrete underlying external catalog (i.e. Hive metastore in this case) can be mapped to fields in `CatalogTable`. This flag is useful when the target Hive table contains structures that can't be handled by Spark SQL, e.g., skewed columns and storage handler, etc..

## How was this patch tested?

New test cases are added in `ShowCreateTableSuite` to do round-trip tests.

Author: Cheng Lian <lian@databricks.com>

Closes #13079 from liancheng/spark-14346-show-create-table-for-hive-tables.
2016-05-17 15:56:44 -07:00
Shixiong Zhu 8e8bc9f957 [SPARK-11735][CORE][SQL] Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped
## What changes were proposed in this pull request?

Add a check in the constructor of SQLContext/SparkSession to make sure its SparkContext is not stopped.

## How was this patch tested?

Jenkins unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13154 from zsxwing/check-spark-context-stop.
2016-05-17 14:57:21 -07:00
Dongjoon Hyun 9f176dd391 [MINOR][DOCS] Replace remaining 'sqlContext' in ScalaDoc/JavaDoc.
## What changes were proposed in this pull request?

According to the recent change, this PR replaces all the remaining `sqlContext` usage with `spark` in ScalaDoc/JavaDoc (.scala/.java files) except `SQLContext.scala`, `SparkPlan.scala', and `DatasetHolder.scala`.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13125 from dongjoon-hyun/minor_doc_sparksession.
2016-05-17 20:50:22 +02:00
hyukjinkwon 8d05a7a98b [SPARK-10216][SQL] Avoid creating empty files during overwriting with group by query
## What changes were proposed in this pull request?

Currently, `INSERT INTO` with `GROUP BY` query tries to make at least 200 files (default value of `spark.sql.shuffle.partition`), which results in lots of empty files.

This PR makes it avoid creating empty files during overwriting into Hive table and in internal data sources  with group by query.

This checks whether the given partition has data in it or not and creates/writes file only when it actually has data.

## How was this patch tested?

Unittests in `InsertIntoHiveTableSuite` and `HadoopFsRelationTest`.

Closes #8411

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Keuntae Park <sirpkt@apache.org>

Closes #12855 from HyukjinKwon/pr/8411.
2016-05-17 11:18:51 -07:00
Wenchen Fan 20a89478e1 [SPARK-14346][SQL][FOLLOW-UP] add tests for CREAT TABLE USING with partition and bucket
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/12781 introduced PARTITIONED BY, CLUSTERED BY, and SORTED BY keywords to CREATE TABLE USING. This PR adds tests to make sure those keywords are handled correctly.

This PR also fixes a mistake that we should create non-hive-compatible table if partition or bucket info exists.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13144 from cloud-fan/add-test.
2016-05-17 10:12:51 -07:00
Kousuke Saruta c0c3ec3547 [SPARK-15165] [SQL] Codegen can break because toCommentSafeString is not actually safe
## What changes were proposed in this pull request?

toCommentSafeString method replaces "\u" with "\\\\u" to avoid codegen breaking.
But if the even number of "\" is put before "u", like "\\\\u", in the string literal in the query, codegen can break.

Following code causes compilation error.

```
val df = Seq(...).toDF
df.select("'\\\\\\\\u002A/'").show
```

The reason of the compilation error is because "\\\\\\\\\\\\\\\\u002A/" is translated into "*/" (the end of comment).

Due to this unsafety, arbitrary code can be injected like as follows.

```
val df = Seq(...).toDF
// Inject "System.exit(1)"
df.select("'\\\\\\\\u002A/{System.exit(1);}/*'").show
```

## How was this patch tested?

Added new test cases.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>
Author: sarutak <sarutak@oss.nttdata.co.jp>

Closes #12939 from sarutak/SPARK-15165.
2016-05-17 10:07:01 -07:00
Sean Owen 122302cbf5 [SPARK-15290][BUILD] Move annotations, like @Since / @DeveloperApi, into spark-tags
## What changes were proposed in this pull request?

(See https://github.com/apache/spark/pull/12416 where most of this was already reviewed and committed; this is just the module structure and move part. This change does not move the annotations into test scope, which was the apparently problem last time.)

Rename `spark-test-tags` -> `spark-tags`; move common annotations like `Since` to `spark-tags`

## How was this patch tested?

Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #13074 from srowen/SPARK-15290.
2016-05-17 09:55:53 +01:00
Liwei Lin 95f4fbae52 [SPARK-14942][SQL][STREAMING] Reduce delay between batch construction and execution
## Problem

Currently in `StreamExecution`, [we first run the batch, then construct the next](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L165):
```scala
if (dataAvailable) runBatch()
constructNextBatch()
```

This is good when we run batches ASAP, where data would get processed in the **very next batch**:

![1](https://cloud.githubusercontent.com/assets/15843379/14779964/2786e698-0b0d-11e6-9d2c-bb41513488b2.png)

However, when we run batches at trigger like `ProcessTime("1 minute")`, data - such as _y_ below - may not get processed in the very next batch i.e. _batch 1_, but in _batch 2_:

![2](https://cloud.githubusercontent.com/assets/15843379/14779818/6f3bb064-0b0c-11e6-9f16-c1ce4897186b.png)

## What changes were proposed in this pull request?

This patch reverses the order of `constructNextBatch()` and `runBatch()`. After this patch, data would get processed in the **very next batch**, i.e. _batch 1_:

![3](https://cloud.githubusercontent.com/assets/15843379/14779816/6f36ee62-0b0c-11e6-9e53-bc8397fade18.png)

In addition, this patch alters when we do `currentBatchId += 1`: let's do that when the processing of the current batch's data is completed, so we won't bother passing `currentBatchId + 1` or  `currentBatchId - 1` to states or sinks.

## How was this patch tested?

New added test case. Also this should be covered by existing test suits, e.g. stress tests and others.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #12725 from lw-lin/construct-before-run-3.
2016-05-16 12:59:55 -07:00
Sean Zhong 4a5ee1954a [SPARK-15253][SQL] Support old table schema config key "spark.sql.sources.schema" for DESCRIBE TABLE
## What changes were proposed in this pull request?

"DESCRIBE table" is broken when table schema is stored at key "spark.sql.sources.schema".

Originally, we used spark.sql.sources.schema to store the schema of a data source table.
After SPARK-6024, we removed this flag. Although we are not using spark.sql.sources.schema any more, we need to still support it.

## How was this patch tested?

Unit test.

When using spark2.0 to load a table generated by spark 1.2.
Before change:
`DESCRIBE table` => Schema of this table is inferred at runtime,,

After change:
`DESCRIBE table` => correct output.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13073 from clockfly/spark-15253.
2016-05-16 10:41:20 +08:00
Zheng RuiFeng c7efc56c7b [MINOR] Fix Typos
## What changes were proposed in this pull request?
1,Rename matrix args in BreezeUtil to upper to match the doc
2,Fix several typos in ML and SQL

## How was this patch tested?
manual tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #13078 from zhengruifeng/fix_ann.
2016-05-15 15:59:49 +01:00
Tejas Patil 4210e2a6b7 [TRIVIAL] Add () to SparkSession's builder function
## What changes were proposed in this pull request?

Was trying out `SparkSession` for the first time and the given class doc (when copied as is) did not work over Spark shell:

```
scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate()
<console>:27: error: org.apache.spark.sql.SparkSession.Builder does not take parameters
       SparkSession.builder().master("local").appName("Word Count").getOrCreate()
```

Adding () to the builder method in SparkSession.

## How was this patch tested?

```
scala> SparkSession.builder().master("local").appName("Word Count").getOrCreate()
res0: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38

scala> SparkSession.builder.master("local").appName("Word Count").getOrCreate()
res1: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession65c17e38
```

Author: Tejas Patil <tejasp@fb.com>

Closes #13086 from tejasapatil/doc_correction.
2016-05-13 18:10:22 -07:00
hyukjinkwon 3ded5bc4db [SPARK-15267][SQL] Refactor options for JDBC and ORC data sources and change default compression for ORC
## What changes were proposed in this pull request?

Currently, Parquet, JSON and CSV data sources have a class for thier options, (`ParquetOptions`, `JSONOptions` and `CSVOptions`).

It is convenient to manage options for sources to gather options into a class. Currently, `JDBC`, `Text`, `libsvm` and `ORC` datasources do not have this class. This might be nicer if these options are in a unified format so that options can be added and

This PR refactors the options in Spark internal data sources adding new classes, `OrcOptions`, `TextOptions`, `JDBCOptions` and `LibSVMOptions`.

Also, this PR change the default compression codec for ORC from `NONE` to `SNAPPY`.

## How was this patch tested?

Existing tests should cover this for refactoring and unittests in `OrcHadoopFsRelationSuite` for changing the default compression codec for ORC.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13048 from HyukjinKwon/SPARK-15267.
2016-05-13 09:04:37 -07:00
Reynold Xin e1dc853737 [SPARK-15310][SQL] Rename HiveTypeCoercion -> TypeCoercion
## What changes were proposed in this pull request?
We originally designed the type coercion rules to match Hive, but over time we have diverged. It does not make sense to call it HiveTypeCoercion anymore. This patch renames it TypeCoercion.

## How was this patch tested?
Updated unit tests to reflect the rename.

Author: Reynold Xin <rxin@databricks.com>

Closes #13091 from rxin/SPARK-15310.
2016-05-13 00:15:39 -07:00
hyukjinkwon 51841d77d9 [SPARK-13866] [SQL] Handle decimal type in CSV inference at CSV data source.
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-13866

This PR adds the support to infer `DecimalType`.
Here are the rules between `IntegerType`, `LongType` and `DecimalType`.

#### Infering Types

1. `IntegerType` and then `LongType`are tried first.

  ```scala
  Int.MaxValue => IntegerType
  Long.MaxValue => LongType
  ```

2. If it fails, try `DecimalType`.

  ```scala
  (Long.MaxValue + 1) => DecimalType(20, 0)
  ```
  This does not try to infer this as `DecimalType` when scale is less than 0.

3. if it fails, try `DoubleType`
  ```scala
  0.1 => DoubleType // This is failed to be inferred as `DecimalType` because it has the scale, 1.
  ```

#### Compatible Types (Merging Types)

For merging types, this is the same with JSON data source. If `DecimalType` is not capable, then it becomes `DoubleType`

## How was this patch tested?

Unit tests were used and `./dev/run_tests` for code style test.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #11724 from HyukjinKwon/SPARK-13866.
2016-05-12 22:31:14 -07:00
Reynold Xin eda2800d44 [SPARK-14541][SQL] Support IFNULL, NULLIF, NVL and NVL2
## What changes were proposed in this pull request?
This patch adds support for a few SQL functions to improve compatibility with other databases: IFNULL, NULLIF, NVL and NVL2. In order to do this, this patch introduced a RuntimeReplaceable expression trait that allows replacing an unevaluable expression in the optimizer before evaluation.

Note that the semantics are not completely identical to other databases in esoteric cases.

## How was this patch tested?
Added a new test suite SQLCompatibilityFunctionSuite.

Closes #12373.

Author: Reynold Xin <rxin@databricks.com>

Closes #13084 from rxin/SPARK-14541.
2016-05-12 22:18:39 -07:00
Reynold Xin ba169c3230 [SPARK-15306][SQL] Move object expressions into expressions.objects package
## What changes were proposed in this pull request?
This patch moves all the object related expressions into expressions.objects package, for better code organization.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #13085 from rxin/SPARK-15306.
2016-05-12 21:35:14 -07:00
Herman van Hovell bb1362eb3b [SPARK-10605][SQL] Create native collect_list/collect_set aggregates
## What changes were proposed in this pull request?
We currently use the Hive implementations for the collect_list/collect_set aggregate functions. This has a few major drawbacks: the use of HiveUDAF (which has quite a bit of overhead) and the lack of support for struct datatypes. This PR adds native implementation of these functions to Spark.

The size of the collected list/set may vary, this means we cannot use the fast, Tungsten, aggregation path to perform the aggregation, and that we fallback to the slower sort based path. Another big issue with these operators is that when the size of the collected list/set grows too large, we can start experiencing large GC pauzes and OOMEs.

This `collect*` aggregates implemented in this PR rely on the sort based aggregate path for correctness. They maintain their own internal buffer which holds the rows for one group at a time. The sortbased aggregation path is triggered by disabling `partialAggregation` for these aggregates (which is kinda funny); this technique is also employed in `org.apache.spark.sql.hiveHiveUDAFFunction`.

I have done some performance testing:
```scala
import org.apache.spark.sql.{Dataset, Row}

sql("create function collect_list2 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectList'")

val df = range(0, 10000000).select($"id", (rand(213123L) * 100000).cast("int").as("grp"))
df.select(countDistinct($"grp")).show

def benchmark(name: String, plan: Dataset[Row], maxItr: Int = 5): Unit = {
   // Do not measure planning.
   plan1.queryExecution.executedPlan

   // Execute the plan a number of times and average the result.
   val start = System.nanoTime
   var i = 0
   while (i < maxItr) {
     plan.rdd.foreach(row => Unit)
     i += 1
   }
   val time = (System.nanoTime - start) / (maxItr * 1000000L)
   println(s"[$name] $maxItr iterations completed in an average time of $time ms.")
}

val plan1 = df.groupBy($"grp").agg(collect_list($"id"))
val plan2 = df.groupBy($"grp").agg(callUDF("collect_list2", $"id"))

benchmark("Spark collect_list", plan1)
...
> [Spark collect_list] 5 iterations completed in an average time of 3371 ms.

benchmark("Hive collect_list", plan2)
...
> [Hive collect_list] 5 iterations completed in an average time of 9109 ms.
```
Performance is improved by a factor 2-3.

## How was this patch tested?
Added tests to `DataFrameAggregateSuite`.

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

Closes #12874 from hvanhovell/implode.
2016-05-12 13:56:00 -07:00
gatorsmile be617f3d06 [SPARK-14684][SPARK-15277][SQL] Partition Spec Validation in SessionCatalog and Checking Partition Spec Existence Before Dropping
#### What changes were proposed in this pull request?
~~Currently, multiple partitions are allowed to drop by using a single DDL command: Alter Table Drop Partition. However, the internal implementation could break atomicity. That means, we could just drop a subset of qualified partitions, if hitting an exception when dropping one of qualified partitions~~

~~This PR contains the following behavior changes:~~
~~- disallow dropping multiple partitions by a single command ~~
~~- allow users to input predicates in partition specification and issue a nicer error message if the predicate's comparison operator is not `=`.~~
~~- verify the partition spec in SessionCatalog. This can ensure each partition spec in `Drop Partition` does not correspond to multiple partitions.~~

This PR has two major parts:
- Verify the partition spec in SessionCatalog for fixing the following issue:
  ```scala
  sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')")
  ```
  Above example uses an invalid partition spec. Without this PR, we will drop all the partitions. The reason is Hive megastores getPartitions API returns all the partitions if we provide an invalid spec.

- Re-implemented the `dropPartitions` in `HiveClientImpl`. Now, we always check if all the user-specified partition specs exist before attempting to drop the partitions. Previously, we start drop the partition before completing checking the existence of all the partition specs. If any failure happened after we start to drop the partitions, we will log an error message to indicate which partitions have been dropped and which partitions have not been dropped.

#### How was this patch tested?
Modified the existing test cases and added new test cases.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12801 from gatorsmile/banDropMultiPart.
2016-05-12 11:14:40 -07:00
Liang-Chi Hsieh 470de743ec [SPARK-15094][SPARK-14803][SQL] Remove extra Project added in EliminateSerialization
## What changes were proposed in this pull request?

We will eliminate the pair of `DeserializeToObject` and `SerializeFromObject` in `Optimizer` and add extra `Project`. However, when DeserializeToObject's outputObjectType is ObjectType and its cls can't be processed by unsafe project, it will be failed.

To fix it, we can simply remove the extra `Project` and replace the output attribute of `DeserializeToObject` in another rule.

## How was this patch tested?
`DatasetSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #12926 from viirya/fix-eliminate-serialization-projection.
2016-05-12 10:11:12 -07:00
Sean Zhong 33c6eb5218 [SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView
## What changes were proposed in this pull request?

Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #12945 from clockfly/spark-15171.
2016-05-12 15:51:53 +08:00
Wenchen Fan 46991448aa [SPARK-15160][SQL] support data source table in InMemoryCatalog
## What changes were proposed in this pull request?

This PR adds a new rule to convert `SimpleCatalogRelation` to data source table if its table property contains data source information.

## How was this patch tested?

new test in SQLQuerySuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12935 from cloud-fan/ds-table.
2016-05-11 23:55:42 -07:00
Cheng Lian f036dd7ce7 [SPARK-14346] SHOW CREATE TABLE for data source tables
## What changes were proposed in this pull request?

This PR adds native `SHOW CREATE TABLE` DDL command for data source tables. Support for Hive tables will be added in follow-up PR(s).

To show table creation DDL for data source tables created by CTAS statements, this PR also added partitioning and bucketing support for normal `CREATE TABLE ... USING ...` syntax.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

A new test suite `ShowCreateTableSuite` is added in sql/hive package to test the new feature.

Author: Cheng Lian <lian@databricks.com>

Closes #12781 from liancheng/spark-14346-show-create-table.
2016-05-11 20:44:04 -07:00
Sandeep Singh ff92eb2e80 [SPARK-15080][CORE] Break copyAndReset into copy and reset
## What changes were proposed in this pull request?
Break copyAndReset into two methods copy and reset instead of just one.

## How was this patch tested?
Existing Tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #12936 from techaddict/SPARK-15080.
2016-05-12 11:12:09 +08:00
Bill Chambers 603f4453a1 [SPARK-15264][SPARK-15274][SQL] CSV Reader Error on Blank Column Names
## What changes were proposed in this pull request?

When a CSV begins with:
- `,,`
OR
- `"","",`

meaning that the first column names are either empty or blank strings and `header` is specified to be `true`, then the column name is replaced with `C` + the index number of that given column. For example, if you were to read in the CSV:
```
"","second column"
"hello", "there"
```
Then column names would become `"C0", "second column"`.

This behavior aligns with what currently happens when `header` is specified to be `false` in recent versions of Spark.

### Current Behavior in Spark <=1.6
In Spark <=1.6, a CSV with a blank column name becomes a blank string, `""`, meaning that this column cannot be accessed. However the CSV reads in without issue.

### Current Behavior in Spark 2.0
Spark throws a NullPointerError and will not read in the file.

#### Reproduction in 2.0
https://databricks-prod-cloudfront.cloud.databricks.com/public/4027ec902e239c93eaaa8714f173bcfc/346304/2828750690305044/484361/latest.html

## How was this patch tested?
A new test was added to `CSVSuite` to account for this issue. We then have asserts that test for being able to select both the empty column names as well as the regular column names.

Author: Bill Chambers <bill@databricks.com>
Author: Bill Chambers <wchambers@ischool.berkeley.edu>

Closes #13041 from anabranch/master.
2016-05-11 17:42:13 -07:00
Andrew Or f14c4ba001 [SPARK-15276][SQL] CREATE TABLE with LOCATION should imply EXTERNAL
## What changes were proposed in this pull request?

Before:
```sql
-- uses that location but issues a warning
CREATE TABLE my_tab LOCATION /some/path
-- deletes any existing data in the specified location
DROP TABLE my_tab
```

After:
```sql
-- uses that location but creates an EXTERNAL table instead
CREATE TABLE my_tab LOCATION /some/path
-- does not delete the data at /some/path
DROP TABLE my_tab
```

This patch essentially makes the `EXTERNAL` field optional. This is related to #13032.

## How was this patch tested?

New test in `DDLCommandSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #13060 from andrewor14/location-implies-external.
2016-05-11 17:29:58 -07:00
Andrew Or 8881765ac7 [SPARK-15257][SQL] Require CREATE EXTERNAL TABLE to specify LOCATION
## What changes were proposed in this pull request?

Before:
```sql
-- uses warehouse dir anyway
CREATE EXTERNAL TABLE my_tab
-- doesn't actually delete the data
DROP TABLE my_tab
```
After:
```sql
-- no location is provided, throws exception
CREATE EXTERNAL TABLE my_tab
-- creates an external table using that location
CREATE EXTERNAL TABLE my_tab LOCATION '/path/to/something'
-- doesn't delete the data, which is expected
DROP TABLE my_tab
```

## How was this patch tested?

New test in `DDLCommandSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #13032 from andrewor14/create-external-table-location.
2016-05-11 15:30:53 -07:00
Tathagata Das 81c68eceba [SPARK-15248][SQL] Make MetastoreFileCatalog consider directories from partition specs of a partitioned metastore table
Table partitions can be added with locations different from default warehouse location of a hive table.
`CREATE TABLE parquetTable (a int) PARTITIONED BY (b int) STORED AS parquet `
`ALTER TABLE parquetTable ADD PARTITION (b=1) LOCATION '/partition'`
Querying such a table throws error as the MetastoreFileCatalog does not list the added partition directory, it only lists the default base location.

```
[info] - SPARK-15248: explicitly added partitions should be readable *** FAILED *** (1 second, 8 milliseconds)
[info]   java.util.NoSuchElementException: key not found: file:/Users/tdas/Projects/Spark/spark2/target/tmp/spark-b39ad224-c5d1-4966-8981-fb45a2066d61/partition
[info]   at scala.collection.MapLike$class.default(MapLike.scala:228)
[info]   at scala.collection.AbstractMap.default(Map.scala:59)
[info]   at scala.collection.MapLike$class.apply(MapLike.scala:141)
[info]   at scala.collection.AbstractMap.apply(Map.scala:59)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:59)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:55)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
[info]   at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
[info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
[info]   at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
[info]   at scala.collection.AbstractTraversable.map(Traversable.scala:104)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog.listFiles(PartitioningAwareFileCatalog.scala:55)
[info]   at org.apache.spark.sql.execution.datasources.FileSourceStrategy$.apply(FileSourceStrategy.scala:93)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
[info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:55)
[info]   at org.apache.spark.sql.execution.SparkStrategies$SpecialLimits$.apply(SparkStrategies.scala:55)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
[info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
[info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:77)
[info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:75)
[info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:82)
[info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:82)
[info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:330)
[info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:146)
[info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:159)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:554)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:535)
[info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:125)
[info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTempDir(parquetSuites.scala:726)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7.apply$mcV$sp(parquetSuites.scala:535)
[info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:166)
[info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTable(parquetSuites.scala:726)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply$mcV$sp(parquetSuites.scala:534)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
```

The solution in this PR to get the paths to list from the partition spec and not rely on the default table path alone.

unit tests.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #13022 from tdas/SPARK-15248.
2016-05-11 12:36:25 -07:00
Eric Liang 6d0368ab8d [SPARK-15259] Sort time metric should not include spill and record insertion time
## What changes were proposed in this pull request?

After SPARK-14669 it seems the sort time metric includes both spill and record insertion time. This makes it not very useful since the metric becomes close to the total execution time of the node.

We should track just the time spent for in-memory sort, as before.

## How was this patch tested?

Verified metric in the UI, also unit test on UnsafeExternalRowSorter.

cc davies

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #13035 from ericl/fix-metrics.
2016-05-11 11:25:46 -07:00
Shixiong Zhu 875ef76428 [SPARK-15231][SQL] Document the semantic of saveAsTable and insertInto and don't drop columns silently
## What changes were proposed in this pull request?

This PR adds documents about the different behaviors between `insertInto` and `saveAsTable`, and throws an exception when the user try to add too man columns using `saveAsTable with append`.

## How was this patch tested?

Unit tests added in this PR.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13013 from zsxwing/SPARK-15231.
2016-05-10 23:53:55 -07:00
Davies Liu 1fbe2785df [SPARK-15255][SQL] limit the length of name for cached DataFrame
## What changes were proposed in this pull request?

We use the tree string of an SparkPlan as the name of cached DataFrame, that could be very long, cause the browser to be not responsive. This PR will limit the length of the name to 1000 characters.

## How was this patch tested?

Here is how the UI looks right now:

![ui](https://cloud.githubusercontent.com/assets/40902/15163355/d5640f9c-16bc-11e6-8655-809af8a4fed1.png)

Author: Davies Liu <davies@databricks.com>

Closes #13033 from davies/cache_name.
2016-05-10 22:29:41 -07:00
hyukjinkwon 3ff012051f [SPARK-15250][SQL] Remove deprecated json API in DataFrameReader
## What changes were proposed in this pull request?

This PR removes the old `json(path: String)` API which is covered by the new `json(paths: String*)`.

## How was this patch tested?

Jenkins tests (existing tests should cover this)

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #13040 from HyukjinKwon/SPARK-15250.
2016-05-10 22:21:17 -07:00
Reynold Xin 5a5b83c97b [SPARK-15261][SQL] Remove experimental tag from DataFrameReader/Writer
## What changes were proposed in this pull request?
This patch removes experimental tag from DataFrameReader and DataFrameWriter, and explicitly tags a few methods added for structured streaming as experimental.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #13038 from rxin/SPARK-15261.
2016-05-10 21:54:32 -07:00
Sean Zhong 61e0bdcff2 [SPARK-14476][SQL] Improve the physical plan visualization by adding meta info like table name and file path for data source.
## What changes were proposed in this pull request?
Improve the physical plan visualization by adding meta info like table name and file path for data source.

Meta info InputPaths and TableName are newly added. Example:
```
scala> spark.range(10).write.saveAsTable("tt")
scala> spark.sql("select * from tt").explain()
== Physical Plan ==
WholeStageCodegen
:  +- BatchedScan HadoopFiles[id#13L] Format: ParquetFormat, InputPaths: file:/home/xzhong10/spark-linux/assembly/spark-warehouse/tt, PushedFilters: [], ReadSchema: struct<id:bigint>, TableName: default.tt
```

## How was this patch tested?

manual tests.

Changes for UI:
Before:
![ui_before_change](https://cloud.githubusercontent.com/assets/2595532/15064559/3d423e3c-1388-11e6-8099-7803ef496c4d.jpg)

After:
![fix_long_string](https://cloud.githubusercontent.com/assets/2595532/15133566/8ad09e26-1696-11e6-939c-99b908249b9d.jpg)

![for_load](https://cloud.githubusercontent.com/assets/2595532/15157224/3ba95c98-171d-11e6-885a-de0ee8dec27c.jpg)

Author: Sean Zhong <clockfly@gmail.com>

Closes #12947 from clockfly/spark-14476.
2016-05-10 21:50:53 -07:00
Tathagata Das d9ca9fd3e5 [SPARK-14837][SQL][STREAMING] Added support in file stream source for reading new files added to subdirs
## What changes were proposed in this pull request?
Currently, file stream source can only find new files if they appear in the directory given to the source, but not if they appear in subdirs. This PR add support for providing glob patterns when creating file stream source so that it can find new files in nested directories based on the glob pattern.

## How was this patch tested?

Unit test that tests when new files are discovered with globs and partitioned directories.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #12616 from tdas/SPARK-14837.
2016-05-10 16:43:32 -07:00
Sandeep Singh da02d006bb [SPARK-15249][SQL] Use FunctionResource instead of (String, String) in CreateFunction and CatalogFunction for resource
Use FunctionResource instead of (String, String) in CreateFunction and CatalogFunction for resource
see: TODO's here
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala#L36
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala#L42

Existing tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13024 from techaddict/SPARK-15249.
2016-05-10 14:22:03 -07:00
Herman van Hovell d28c67544b [SPARK-14986][SQL] Return correct result for empty LATERAL VIEW OUTER
## What changes were proposed in this pull request?
A Generate with the `outer` flag enabled should always return one or more rows for every input row. The optimizer currently violates this by rewriting `outer` Generates that do not contain columns of the child plan into an unjoined generate, for example:
```sql
select e from a lateral view outer explode(a.b) as e
```
The result of this is that `outer` Generate does not produce output at all when the Generators' input expression is empty. This PR fixes this.

## How was this patch tested?
Added test case to `SQLQuerySuite`.

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

Closes #12906 from hvanhovell/SPARK-14986.
2016-05-10 12:47:31 -07:00
Subhobrata Dey 89f73f6741 [SPARK-14642][SQL] import org.apache.spark.sql.expressions._ breaks udf under functions
## What changes were proposed in this pull request?

PR fixes the import issue which breaks udf functions.

The following code snippet throws an error

```
scala> import org.apache.spark.sql.functions._
import org.apache.spark.sql.functions._

scala> import org.apache.spark.sql.expressions._
import org.apache.spark.sql.expressions._

scala> udf((v: String) => v.stripSuffix("-abc"))
<console>:30: error: No TypeTag available for String
       udf((v: String) => v.stripSuffix("-abc"))
```

This PR resolves the issue.

## How was this patch tested?

patch tested with unit tests.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: Subhobrata Dey <sbcd90@gmail.com>

Closes #12458 from sbcd90/udfFuncBreak.
2016-05-10 12:32:56 -07:00
Andrew Or 69641066ae [SPARK-15037][HOTFIX] Don't create 2 SparkSessions in constructor
## What changes were proposed in this pull request?

After #12907 `TestSparkSession` creates a spark session in one of the constructors just to get the `SparkContext` from it. This ends up creating 2 `SparkSession`s from one call, which is definitely not what we want.

## How was this patch tested?

Jenkins.

Author: Andrew Or <andrew@databricks.com>

Closes #13031 from andrewor14/sql-test.
2016-05-10 12:07:47 -07:00
Andrew Or cddb9da074 [HOTFIX] SQL test compilation error from merge conflict 2016-05-10 11:46:02 -07:00
gatorsmile 5c6b085578 [SPARK-14603][SQL] Verification of Metadata Operations by Session Catalog
Since we cannot really trust if the underlying external catalog can throw exceptions when there is an invalid metadata operation, let's do it in SessionCatalog.

- [X] The first step is to unify the error messages issued in Hive-specific Session Catalog and general Session Catalog.
- [X] The second step is to verify the inputs of metadata operations for partitioning-related operations. This is moved to a separate PR: https://github.com/apache/spark/pull/12801
- [X] The third step is to add database existence verification in `SessionCatalog`
- [X] The fourth step is to add table existence verification in `SessionCatalog`
- [X] The fifth step is to add function existence verification in `SessionCatalog`

Add test cases and verify the error messages we issued

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12385 from gatorsmile/verifySessionAPIs.
2016-05-10 11:25:55 -07:00
Sandeep Singh ed0b4070fb [SPARK-15037][SQL][MLLIB] Use SparkSession instead of SQLContext in Scala/Java TestSuites
## What changes were proposed in this pull request?
Use SparkSession instead of SQLContext in Scala/Java TestSuites
as this PR already very big working Python TestSuites in a diff PR.

## How was this patch tested?
Existing tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #12907 from techaddict/SPARK-15037.
2016-05-10 11:17:47 -07:00
Wenchen Fan bcfee153b1 [SPARK-12837][CORE] reduce network IO for accumulators
Sending un-updated accumulators back to driver makes no sense, as merging a zero value accumulator is a no-op. We should only send back updated accumulators, to save network IO.

new test in `TaskContextSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12899 from cloud-fan/acc.
2016-05-10 11:16:56 -07:00
Herman van Hovell 2646265368 [SPARK-14773] [SPARK-15179] [SQL] Fix SQL building and enable Hive tests
## What changes were proposed in this pull request?
This PR fixes SQL building for predicate subqueries and correlated scalar subqueries. It also enables most Hive subquery tests.

## How was this patch tested?
Enabled new tests in HiveComparisionSuite.

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

Closes #12988 from hvanhovell/SPARK-14773.
2016-05-10 09:56:07 -07:00
Pete Robbins 2dfb9cd1f7 [SPARK-15154] [SQL] Change key types to Long in tests
## What changes were proposed in this pull request?

As reported in the Jira the 2 tests changed here are using a key of type Integer where the Spark sql code assumes the type is Long. This PR changes the tests to use the correct key types.

## How was this patch tested?

Test builds run on both Big Endian and Little Endian platforms

Author: Pete Robbins <robbinspg@gmail.com>

Closes #13009 from robbinspg/HashedRelationSuiteFix.
2016-05-10 09:53:56 -07:00