Commit graph

2328 commits

Author SHA1 Message Date
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
Cheng Lian 8a12580d25 [SPARK-14127][SQL] "DESC <table>": Extracts schema information from table properties for data source tables
## What changes were proposed in this pull request?

This is a follow-up of #12934 and #12844. This PR adds a set of utility methods in `DDLUtils` to help extract schema information (user-defined schema, partition columns, and bucketing information) from data source table properties. These utility methods are then used in `DescribeTableCommand` to refine output for data source tables. Before this PR, the aforementioned schema information are only shown as table properties, which are hard to read.

Sample output:

```
+----------------------------+---------------------------------------------------------+-------+
|col_name                    |data_type                                                |comment|
+----------------------------+---------------------------------------------------------+-------+
|a                           |bigint                                                   |       |
|b                           |bigint                                                   |       |
|c                           |bigint                                                   |       |
|d                           |bigint                                                   |       |
|# Partition Information     |                                                         |       |
|# col_name                  |                                                         |       |
|d                           |                                                         |       |
|                            |                                                         |       |
|# Detailed Table Information|                                                         |       |
|Database:                   |default                                                  |       |
|Owner:                      |lian                                                     |       |
|Create Time:                |Tue May 10 03:20:34 PDT 2016                             |       |
|Last Access Time:           |Wed Dec 31 16:00:00 PST 1969                             |       |
|Location:                   |file:/Users/lian/local/src/spark/workspace-a/target/...  |       |
|Table Type:                 |MANAGED                                                  |       |
|Table Parameters:           |                                                         |       |
|  rawDataSize               |-1                                                       |       |
|  numFiles                  |1                                                        |       |
|  transient_lastDdlTime     |1462875634                                               |       |
|  totalSize                 |684                                                      |       |
|  spark.sql.sources.provider|parquet                                                  |       |
|  EXTERNAL                  |FALSE                                                    |       |
|  COLUMN_STATS_ACCURATE     |false                                                    |       |
|  numRows                   |-1                                                       |       |
|                            |                                                         |       |
|# Storage Information       |                                                         |       |
|SerDe Library:              |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe       |       |
|InputFormat:                |org.apache.hadoop.mapred.SequenceFileInputFormat         |       |
|OutputFormat:               |org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat|       |
|Compressed:                 |No                                                       |       |
|Num Buckets:                |2                                                        |       |
|Bucket Columns:             |[b]                                                      |       |
|Sort Columns:               |[c]                                                      |       |
|Storage Desc Parameters:    |                                                         |       |
|  path                      |file:/Users/lian/local/src/spark/workspace-a/target/...  |       |
|  serialization.format      |1                                                        |       |
+----------------------------+---------------------------------------------------------+-------+
```

## How was this patch tested?

Test cases are added in `HiveDDLSuite` to check command output.

Author: Cheng Lian <lian@databricks.com>

Closes #13025 from liancheng/spark-14127-extract-schema-info.
2016-05-10 09:00:53 -07:00
gatorsmile 5706472670 [SPARK-15215][SQL] Fix Explain Parsing and Output
#### What changes were proposed in this pull request?
This PR is to address a few existing issues in `EXPLAIN`:
- The `EXPLAIN` options `LOGICAL | FORMATTED | EXTENDED | CODEGEN` should not be 0 or more match. It should 0 or one match. Parser does not allow users to use more than one option in a single command.
- The option `LOGICAL` is not supported. Issue an exception when users specify this option in the command.
- The output of `EXPLAIN ` contains a weird empty line when the output of analyzed plan is empty. We should remove it. For example:
  ```
  == Parsed Logical Plan ==
  CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.  HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false

  == Analyzed Logical Plan ==

  CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.  HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false

  == Optimized Logical Plan ==
  CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.  HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false
  ...
  ```

#### How was this patch tested?
Added and modified a few test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12991 from gatorsmile/explainCreateTable.
2016-05-10 11:53:37 +02:00
gatorsmile f45379173b [SPARK-15187][SQL] Disallow Dropping Default Database
#### What changes were proposed in this pull request?
In Hive Metastore, dropping default database is not allowed. However, in `InMemoryCatalog`, this is allowed.

This PR is to disallow users to drop default database.

#### How was this patch tested?
Previously, we already have a test case in HiveDDLSuite. Now, we also add the same one in DDLSuite

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12962 from gatorsmile/dropDefaultDB.
2016-05-10 11:57:01 +08:00
Reynold Xin 4b4344a813 [SPARK-15229][SQL] Make case sensitivity setting internal
## What changes were proposed in this pull request?
Our case sensitivity support is different from what ANSI SQL standards support. Postgres' behavior is that if an identifier is quoted, then it is treated as case sensitive; otherwise it is folded to lowercase. We will likely need to revisit this in the future and change our behavior. For now, the safest change to do for Spark 2.0 is to make the case sensitive option internal and discourage users from turning it on, effectively making Spark always case insensitive.

## How was this patch tested?
N/A - a small config documentation change.

Author: Reynold Xin <rxin@databricks.com>

Closes #13011 from rxin/SPARK-15229.
2016-05-09 20:03:01 -07:00
Andrew Or 8f932fb88d [SPARK-15234][SQL] Fix spark.catalog.listDatabases.show()
## What changes were proposed in this pull request?

Before:
```
scala> spark.catalog.listDatabases.show()
+--------------------+-----------+-----------+
|                name|description|locationUri|
+--------------------+-----------+-----------+
|Database[name='de...|
|Database[name='my...|
|Database[name='so...|
+--------------------+-----------+-----------+
```

After:
```
+-------+--------------------+--------------------+
|   name|         description|         locationUri|
+-------+--------------------+--------------------+
|default|Default Hive data...|file:/user/hive/w...|
|  my_db|  This is a database|file:/Users/andre...|
|some_db|                    |file:/private/var...|
+-------+--------------------+--------------------+
```

## How was this patch tested?

New test in `CatalogSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #13015 from andrewor14/catalog-show.
2016-05-09 20:02:23 -07:00
xin Wu 980bba0dcf [SPARK-15025][SQL] fix duplicate of PATH key in datasource table options
## What changes were proposed in this pull request?
The issue is that when the user provides the path option with uppercase "PATH" key, `options` contains `PATH` key and will get into the non-external case in the following code in `createDataSourceTables.scala`, where a new key "path" is created with a default path.
```
val optionsWithPath =
      if (!options.contains("path")) {
        isExternal = false
        options + ("path" -> sessionState.catalog.defaultTablePath(tableIdent))
      } else {
        options
      }
```
So before creating hive table, serdeInfo.parameters will contain both "PATH" and "path" keys and different directories. and Hive table's dataLocation contains the value of "path".

The fix in this PR is to convert `options` in the code above to `CaseInsensitiveMap` before checking for containing "path" key.

## How was this patch tested?
A testcase is added

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

Closes #12804 from xwu0226/SPARK-15025.
2016-05-09 17:18:48 -07:00
Josh Rosen c3350cadb8 [SPARK-14972] Improve performance of JSON schema inference's compatibleType method
This patch improves the performance of `InferSchema.compatibleType` and `inferField`. The net result of this patch is a 6x speedup in local benchmarks running against cached data with a massive nested schema.

The key idea is to remove unnecessary sorting in `compatibleType`'s `StructType` merging code. This code takes two structs, merges the fields with matching names, and copies over the unique fields, producing a new schema which is the union of the two structs' schemas. Previously, this code performed a very inefficient `groupBy()` to match up fields with the same name, but this is unnecessary because `inferField` already sorts structs' fields by name: since both lists of fields are sorted, we can simply merge them in a single pass.

This patch also speeds up the existing field sorting in `inferField`: the old sorting code allocated unnecessary intermediate collections, while the new code uses mutable collects and performs in-place sorting.

I rewrote inefficient `equals()` implementations in `StructType` and `Metadata`, significantly reducing object allocations in those methods.

Finally, I replaced a `treeAggregate` call with `fold`: I doubt that `treeAggregate` will benefit us very much because the schemas would have to be enormous to realize large savings in network traffic. Since most schemas are probably fairly small in serialized form, they should typically fit within a direct task result and therefore can be incrementally merged at the driver as individual tasks finish. This change eliminates an entire (short) scheduler stage.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #12750 from JoshRosen/schema-inference-speedups.
2016-05-09 13:11:18 -07:00
Wenchen Fan 2adb11f6db [SPARK-15173][SQL] DataFrameWriter.insertInto should work with datasource table stored in hive
When we parse `CREATE TABLE USING`, we should build a `CreateTableUsing` plan with the `managedIfNoPath` set to true. Then we will add default table path to options when write it to hive.

new test in `SQLQuerySuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12949 from cloud-fan/bug.
2016-05-09 12:58:27 -07:00
Andrew Or 7bf9b12019 [SPARK-15166][SQL] Move some hive-specific code from SparkSession
## What changes were proposed in this pull request?

This also simplifies the code being moved.

## How was this patch tested?

Existing tests.

Author: Andrew Or <andrew@databricks.com>

Closes #12941 from andrewor14/move-code.
2016-05-09 11:24:58 -07:00
jerryshao ee6a8d7eaa [MINOR][SQL] Enhance the exception message if checkpointLocation is not set
Enhance the exception message when `checkpointLocation` is not set, previously the message is:

```
java.util.NoSuchElementException: None.get
  at scala.None$.get(Option.scala:347)
  at scala.None$.get(Option.scala:345)
  at org.apache.spark.sql.DataFrameWriter$$anonfun$8.apply(DataFrameWriter.scala:338)
  at org.apache.spark.sql.DataFrameWriter$$anonfun$8.apply(DataFrameWriter.scala:338)
  at scala.collection.MapLike$class.getOrElse(MapLike.scala:128)
  at scala.collection.AbstractMap.getOrElse(Map.scala:59)
  at org.apache.spark.sql.DataFrameWriter.startStream(DataFrameWriter.scala:337)
  at org.apache.spark.sql.DataFrameWriter.startStream(DataFrameWriter.scala:277)
  ... 48 elided
```

This is not so meaningful, so changing to make it more specific.

Local verified.

Author: jerryshao <sshao@hortonworks.com>

Closes #12998 from jerryshao/improve-exception-message.
2016-05-09 11:14:40 -07:00
Cheng Lian 671b382a80 [SPARK-14127][SQL] Makes 'DESC [EXTENDED|FORMATTED] <table>' support data source tables
## What changes were proposed in this pull request?

This is a follow-up of PR #12844. It makes the newly updated `DescribeTableCommand` to support data sources tables.

## How was this patch tested?

A test case is added to check `DESC [EXTENDED | FORMATTED] <table>` output.

Author: Cheng Lian <lian@databricks.com>

Closes #12934 from liancheng/spark-14127-desc-table-follow-up.
2016-05-09 10:53:32 -07:00
gatorsmile b1e01fd519 [SPARK-15199][SQL] Disallow Dropping Build-in Functions
#### What changes were proposed in this pull request?
As Hive and the major RDBMS behave, the built-in functions are not allowed to drop. In the current implementation, users can drop the built-in functions. However, after dropping the built-in functions, users are unable to add them back.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12975 from gatorsmile/dropBuildInFunction.
2016-05-09 10:49:54 -07:00
Wenchen Fan beb16ec556 [SPARK-15093][SQL] create/delete/rename directory for InMemoryCatalog operations if needed
## What changes were proposed in this pull request?

following operations have file system operation now:

1. CREATE DATABASE: create a dir
2. DROP DATABASE: delete the dir
3. CREATE TABLE: create a dir
4. DROP TABLE: delete the dir
5. RENAME TABLE: rename the dir
6. CREATE PARTITIONS: create a dir
7. RENAME PARTITIONS: rename the dir
8. DROP PARTITIONS: drop the dir

## How was this patch tested?

new tests in `ExternalCatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12871 from cloud-fan/catalog.
2016-05-09 10:47:45 -07:00
gatorsmile a59ab594ca [SPARK-15184][SQL] Fix Silent Removal of An Existent Temp Table by Rename Table
#### What changes were proposed in this pull request?
Currently, if we rename a temp table `Tab1` to another existent temp table `Tab2`. `Tab2` will be silently removed. This PR is to detect it and issue an exception message.

In addition, this PR also detects another issue in the rename table command. When the destination table identifier does have database name, we should not ignore them. That might mean users could rename a regular table.

#### How was this patch tested?
Added two related test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12959 from gatorsmile/rewriteTable.
2016-05-09 13:05:18 +08:00
Herman van Hovell df89f1d43d [SPARK-15122] [SQL] Fix TPC-DS 41 - Normalize predicates before pulling them out
## What changes were proposed in this pull request?
The official TPC-DS 41 query currently fails because it contains a scalar subquery with a disjunctive correlated predicate (the correlated predicates were nested in ORs). This makes the `Analyzer` pull out the entire predicate which is wrong and causes the following (correct) analysis exception: `The correlated scalar subquery can only contain equality predicates`

This PR fixes this by first simplifing (or normalizing) the correlated predicates before pulling them out of the subquery.

## How was this patch tested?
Manual testing on TPC-DS 41, and added a test to SubquerySuite.

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

Closes #12954 from hvanhovell/SPARK-15122.
2016-05-06 21:06:03 -07:00
Kevin Yu 607a27a0d1 [SPARK-15051][SQL] Create a TypedColumn alias
## What changes were proposed in this pull request?

Currently when we create an alias against a TypedColumn from user-defined Aggregator(for example: agg(aggSum.toColumn as "a")), spark is using the alias' function from Column( as), the alias function will return a column contains a TypedAggregateExpression, which is unresolved because the inputDeserializer is not defined. Later the aggregator function (agg) will inject the inputDeserializer back to the TypedAggregateExpression, but only if the aggregate columns are TypedColumn, in the above case, the TypedAggregateExpression will remain unresolved because it is under column and caused the
problem reported by this jira [15051](https://issues.apache.org/jira/browse/SPARK-15051?jql=project%20%3D%20SPARK).

This PR propose to create an alias function for TypedColumn,  it will return a TypedColumn. It is using the similar code path  as Column's alia function.

For the spark build in aggregate function, like max, it is working with alias, for example

val df1 = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j")
checkAnswer(df1.agg(max("j") as "b"), Row(3) :: Nil)

Thanks for comments.

## How was this patch tested?

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

Add test cases in DatasetAggregatorSuite.scala
run the sql related queries against this patch.

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

Closes #12893 from kevinyu98/spark-15051.
2016-05-07 11:13:48 +08:00
Tathagata Das f7b7ef4166 [SPARK-14997][SQL] Fixed FileCatalog to return correct set of files when there is no partitioning scheme in the given paths
## What changes were proposed in this pull request?
Lets says there are json files in the following directories structure
```
xyz/file0.json
xyz/subdir1/file1.json
xyz/subdir2/file2.json
xyz/subdir1/subsubdir1/file3.json
```
`sqlContext.read.json("xyz")` should read only file0.json according to behavior in Spark 1.6.1. However in current master, all the 4 files are read.

The fix is to make FileCatalog return only the children files of the given path if there is not partitioning detected (instead of all the recursive list of files).

Closes #12774

## How was this patch tested?

unit tests

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

Closes #12856 from tdas/SPARK-14997.
2016-05-06 15:04:16 -07:00
gatorsmile 5c8fad7b9b [SPARK-15108][SQL] Describe Permanent UDTF
#### What changes were proposed in this pull request?
When Describe a UDTF, the command returns a wrong result. The command is unable to find the function, which has been created and cataloged in the catalog but not in the functionRegistry.

This PR is to correct it. If the function is not in the functionRegistry, we will check the catalog for collecting the information of the UDTF function.

#### How was this patch tested?
Added test cases to verify the results

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12885 from gatorsmile/showFunction.
2016-05-06 11:43:07 -07:00
hyukjinkwon fa928ff9a3 [SPARK-14962][SQL] Do not push down isnotnull/isnull on unsuportted types in ORC
## What changes were proposed in this pull request?

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

ORC filters were being pushed down for all types for both `IsNull` and `IsNotNull`.

This is apparently OK because both `IsNull` and `IsNotNull` do not take a type as an argument (Hive 1.2.x) during building filters (`SearchArgument`) in Spark-side but they do not filter correctly because stored statistics always produces `null` for not supported types (eg `ArrayType`) in ORC-side. So, it is always `true` for `IsNull` which ends up with always `false` for `IsNotNull`. (Please see [RecordReaderImpl.java#L296-L318](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L296-L318)  and [RecordReaderImpl.java#L359-L365](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L359-L365) in Hive 1.2)

This looks prevented in Hive 1.3.x >= by forcing to give a type ([`PredicateLeaf.Type`](e085b7e9bd/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java (L50-L56))) when building a filter ([`SearchArgument`](26b5c7b56a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java (L260))) but Hive 1.2.x seems not doing this.

This PR prevents ORC filter creation for `IsNull` and `IsNotNull` on unsupported types. `OrcFilters` resembles `ParquetFilters`.

## How was this patch tested?

Unittests in `OrcQuerySuite` and `OrcFilterSuite` and `sbt scalastyle`.

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

Closes #12777 from HyukjinKwon/SPARK-14962.
2016-05-07 01:46:45 +08:00
Jacek Laskowski bbb7773437 [SPARK-15152][DOC][MINOR] Scaladoc and Code style Improvements
## What changes were proposed in this pull request?

Minor doc and code style fixes

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #12928 from jaceklaskowski/SPARK-15152.
2016-05-05 16:34:27 -07:00
Dongjoon Hyun 2c170dd3d7 [SPARK-15134][EXAMPLE] Indent SparkSession builder patterns and update binary_classification_metrics_example.py
## What changes were proposed in this pull request?

This issue addresses the comments in SPARK-15031 and also fix java-linter errors.
- Use multiline format in SparkSession builder patterns.
- Update `binary_classification_metrics_example.py` to use `SparkSession`.
- Fix Java Linter errors (in SPARK-13745, SPARK-15031, and so far)

## How was this patch tested?

After passing the Jenkins tests and run `dev/lint-java` manually.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12911 from dongjoon-hyun/SPARK-15134.
2016-05-05 14:37:50 -07:00
Shixiong Zhu bb9991dec5 [SPARK-15135][SQL] Make sure SparkSession thread safe
## What changes were proposed in this pull request?

Went through SparkSession and its members and fixed non-thread-safe classes used by SparkSession

## How was this patch tested?

Existing unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12915 from zsxwing/spark-session-thread-safe.
2016-05-05 14:36:47 -07:00
Sandeep Singh ed6f3f8a5f [SPARK-15072][SQL][REPL][EXAMPLES] Remove SparkSession.withHiveSupport
## What changes were proposed in this pull request?
Removing the `withHiveSupport` method of `SparkSession`, instead use `enableHiveSupport`

## How was this patch tested?
ran tests locally

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #12851 from techaddict/SPARK-15072.
2016-05-05 14:35:15 -07:00
gatorsmile 8cba57a75c [SPARK-14124][SQL][FOLLOWUP] Implement Database-related DDL Commands
#### What changes were proposed in this pull request?

First, a few test cases failed in mac OS X  because the property value of `java.io.tmpdir` does not include a trailing slash on some platform. Hive always removes the last trailing slash. For example, what I got in the web:
```
Win NT  --> C:\TEMP\
Win XP  --> C:\TEMP
Solaris --> /var/tmp/
Linux   --> /var/tmp
```
Second, a couple of test cases are added to verify if the commands work properly.

#### How was this patch tested?
Added a test case for it and correct the previous test cases.

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

Closes #12081 from gatorsmile/mkdir.
2016-05-05 14:34:24 -07:00
NarineK 22226fcc92 [SPARK-15110] [SPARKR] Implement repartitionByColumn for SparkR DataFrames
## What changes were proposed in this pull request?

Implement repartitionByColumn on DataFrame.
This will allow us to run R functions on each partition identified by column groups with dapply() method.

## How was this patch tested?

Unit tests

Author: NarineK <narine.kokhlikyan@us.ibm.com>

Closes #12887 from NarineK/repartitionByColumns.
2016-05-05 12:00:55 -07:00
hyukjinkwon ac12b35d31 [SPARK-15148][SQL] Upgrade Univocity library from 2.0.2 to 2.1.0
## What changes were proposed in this pull request?

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

Mainly it improves the performance roughtly about 30%-40% according to the [release note](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.1.0). For the details of the purpose is described in the JIRA.

This PR upgrades Univocity library from 2.0.2 to 2.1.0.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12923 from HyukjinKwon/SPARK-15148.
2016-05-05 11:26:40 -07:00
Wenchen Fan 55cc1c991a [SPARK-14139][SQL] RowEncoder should preserve schema nullability
## What changes were proposed in this pull request?

The problem is: In `RowEncoder`, we use `Invoke` to get the field of an external row, which lose the nullability information. This PR creates a `GetExternalRowField` expression, so that we can preserve the nullability info.

TODO: simplify the null handling logic in `RowEncoder`, to remove so many if branches, in follow-up PR.

## How was this patch tested?

new tests in `RowEncoderSuite`

Note that, This PR takes over https://github.com/apache/spark/pull/11980, with a little simplification, so all credits should go to koertkuipers

Author: Wenchen Fan <wenchen@databricks.com>
Author: Koert Kuipers <koert@tresata.com>

Closes #12364 from cloud-fan/nullable.
2016-05-06 01:08:04 +08:00
Kousuke Saruta 1a9b341581 [SPARK-15132][MINOR][SQL] Debug log for generated code should be printed with proper indentation
## What changes were proposed in this pull request?

Similar to #11990, GenerateOrdering and GenerateColumnAccessor should print debug log for generated code with proper indentation.

## How was this patch tested?

Manually checked.

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

Closes #12908 from sarutak/SPARK-15132.
2016-05-04 22:18:55 -07:00
Tathagata Das bde27b89a2 [SPARK-15131][SQL] Shutdown StateStore management thread when SparkContext has been shutdown
## What changes were proposed in this pull request?

Make sure that whenever the StateStoreCoordinator cannot be contacted, assume that the SparkContext and RpcEnv on the driver has been shutdown, and therefore stop the StateStore management thread, and unload all loaded stores.

## How was this patch tested?

Updated unit tests.

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

Closes #12905 from tdas/SPARK-15131.
2016-05-04 21:19:53 -07:00
gatorsmile ef55e46c92 [SPARK-14993][SQL] Fix Partition Discovery Inconsistency when Input is a Path to Parquet File
#### What changes were proposed in this pull request?
When we load a dataset, if we set the path to ```/path/a=1```, we will not take `a` as the partitioning column. However, if we set the path to ```/path/a=1/file.parquet```, we take `a` as the partitioning column and it shows up in the schema.

This PR is to fix the behavior inconsistency issue.

The base path contains a set of paths that are considered as the base dirs of the input datasets. The partitioning discovery logic will make sure it will stop when it reaches any base path.

By default, the paths of the dataset provided by users will be base paths. Below are three typical cases,
**Case 1**```sqlContext.read.parquet("/path/something=true/")```: the base path will be
`/path/something=true/`, and the returned DataFrame will not contain a column of `something`.
**Case 2**```sqlContext.read.parquet("/path/something=true/a.parquet")```: the base path will be
still `/path/something=true/`, and the returned DataFrame will also not contain a column of
`something`.
**Case 3**```sqlContext.read.parquet("/path/")```: the base path will be `/path/`, and the returned
DataFrame will have the column of `something`.

Users also can override the basePath by setting `basePath` in the options to pass the new base
path to the data source. For example,
```sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")```,
and the returned DataFrame will have the column of `something`.

The related PRs:
- https://github.com/apache/spark/pull/9651
- https://github.com/apache/spark/pull/10211

#### How was this patch tested?
Added a couple of test cases

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

Closes #12828 from gatorsmile/readPartitionedTable.
2016-05-04 18:47:27 -07:00
Sean Zhong 8fb1463d6a [SPARK-6339][SQL] Supports CREATE TEMPORARY VIEW tableIdentifier AS query
## What changes were proposed in this pull request?

This PR support new SQL syntax CREATE TEMPORARY VIEW.
Like:
```
CREATE TEMPORARY VIEW viewName AS SELECT * from xx
CREATE OR REPLACE TEMPORARY VIEW viewName AS SELECT * from xx
CREATE TEMPORARY VIEW viewName (c1 COMMENT 'blabla', c2 COMMENT 'blabla') AS SELECT * FROM xx
```

## How was this patch tested?

Unit tests.

Author: Sean Zhong <clockfly@gmail.com>

Closes #12872 from clockfly/spark-6399.
2016-05-04 18:27:25 -07:00
sethah b281377647 [MINOR][SQL] Fix typo in DataFrameReader csv documentation
## What changes were proposed in this pull request?
Typo fix

## How was this patch tested?
No tests

My apologies for the tiny PR, but I stumbled across this today and wanted to get it corrected for 2.0.

Author: sethah <seth.hendrickson16@gmail.com>

Closes #12912 from sethah/csv_typo.
2016-05-04 16:46:13 -07:00
Reynold Xin 6ae9fc00ed [SPARK-15126][SQL] RuntimeConfig.set should return Unit
## What changes were proposed in this pull request?
Currently we return RuntimeConfig itself to facilitate chaining. However, it makes the output in interactive environments (e.g. notebooks, scala repl) weird because it'd show the response of calling set as a RuntimeConfig itself.

## How was this patch tested?
Updated unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12902 from rxin/SPARK-15126.
2016-05-04 14:26:05 -07:00
Tathagata Das 0fd3a47484 [SPARK-15103][SQL] Refactored FileCatalog class to allow StreamFileCatalog to infer partitioning
## What changes were proposed in this pull request?

File Stream Sink writes the list of written files in a metadata log. StreamFileCatalog reads the list of the files for processing. However StreamFileCatalog does not infer partitioning like HDFSFileCatalog.

This PR enables that by refactoring HDFSFileCatalog to create an abstract class PartitioningAwareFileCatalog, that has all the functionality to infer partitions from a list of leaf files.
- HDFSFileCatalog has been renamed to ListingFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from recursive directory scanning.
- StreamFileCatalog has been renamed to MetadataLogFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from the metadata log.
- The above two classes has been moved into their own files as they are not interfaces that should be in fileSourceInterfaces.scala.

## How was this patch tested?
- FileStreamSinkSuite was update to see if partitioning gets inferred, and on reading whether the partitions get pruned correctly based on the query.
- Other unit tests are unchanged and pass as expected.

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

Closes #12879 from tdas/SPARK-15103.
2016-05-04 11:02:48 -07:00
Reynold Xin 6274a520fa [SPARK-15115][SQL] Reorganize whole stage codegen benchmark suites
## What changes were proposed in this pull request?
We currently have a single suite that is very large, making it difficult to maintain and play with specific primitives. This patch reorganizes the file by creating multiple benchmark suites in a single package.

Most of the changes are straightforward move of code. On top of the code moving, I did:
1. Use SparkSession instead of SQLContext.
2. Turned most benchmark scenarios into a their own test cases, rather than having multiple scenarios in a single test case, which takes forever to run.

## How was this patch tested?
This is a test only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #12891 from rxin/SPARK-15115.
2016-05-04 11:00:01 -07:00
Liang-Chi Hsieh b85d21fb9d [SPARK-14951] [SQL] Support subexpression elimination in TungstenAggregate
## What changes were proposed in this pull request?

We can support subexpression elimination in TungstenAggregate by using current `EquivalentExpressions` which is already used in subexpression elimination for expression codegen.

However, in wholestage codegen, we can't wrap the common expression's codes in functions as before, we simply generate the code snippets for common expressions. These code snippets are inserted before the common expressions are actually used in generated java codes.

For multiple `TypedAggregateExpression` used in aggregation operator, since their input type should be the same. So their `inputDeserializer` will be the same too. This patch can also reduce redundant input deserialization.

## How was this patch tested?
Existing tests.

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

Closes #12729 from viirya/subexpr-elimination-tungstenaggregate.
2016-05-04 10:54:51 -07:00
Reynold Xin d864c55cf8 [SPARK-15109][SQL] Accept Dataset[_] in joins
## What changes were proposed in this pull request?
This patch changes the join API in Dataset so they can accept any Dataset, rather than just DataFrames.

## How was this patch tested?
N/A.

Author: Reynold Xin <rxin@databricks.com>

Closes #12886 from rxin/SPARK-15109.
2016-05-04 10:38:27 -07:00
Liwei Lin e597ec6f1c [SPARK-15022][SPARK-15023][SQL][STREAMING] Add support for testing against the ProcessingTime(intervalMS > 0) trigger and ManualClock
## What changes were proposed in this pull request?

Currently in `StreamTest`, we have a `StartStream` which will start a streaming query against trigger `ProcessTime(intervalMS = 0)` and `SystemClock`.

We also need to test cases against `ProcessTime(intervalMS > 0)`, which often requires `ManualClock`.

This patch:
- fixes an issue of `ProcessingTimeExecutor`, where for a batch it should run `batchRunner` only once but might run multiple times under certain conditions;
- adds support for testing against the `ProcessingTime(intervalMS > 0)` trigger and `AdvanceManualClock`, by specifying them as fields for `StartStream`, and by adding an `AdvanceClock` action;
- adds a test, which takes advantage of the new `StartStream` and `AdvanceManualClock`, to test against [PR#[SPARK-14942] Reduce delay between batch construction and execution ](https://github.com/apache/spark/pull/12725).

## How was this patch tested?

N/A

Author: Liwei Lin <lwlin7@gmail.com>

Closes #12797 from lw-lin/add-trigger-test-support.
2016-05-04 10:25:14 -07:00
Cheng Lian f152fae306 [SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] <table>" DDL command
## What changes were proposed in this pull request?

This PR implements native `DESC [EXTENDED | FORMATTED] <table>` DDL command. Sample output:

```
scala> spark.sql("desc extended src").show(100, truncate = false)
+----------------------------+---------------------------------+-------+
|col_name                    |data_type                        |comment|
+----------------------------+---------------------------------+-------+
|key                         |int                              |       |
|value                       |string                           |       |
|                            |                                 |       |
|# Detailed Table Information|CatalogTable(`default`.`src`, ...|       |
+----------------------------+---------------------------------+-------+

scala> spark.sql("desc formatted src").show(100, truncate = false)
+----------------------------+----------------------------------------------------------+-------+
|col_name                    |data_type                                                 |comment|
+----------------------------+----------------------------------------------------------+-------+
|key                         |int                                                       |       |
|value                       |string                                                    |       |
|                            |                                                          |       |
|# Detailed Table Information|                                                          |       |
|Database:                   |default                                                   |       |
|Owner:                      |lian                                                      |       |
|Create Time:                |Mon Jan 04 17:06:00 CST 2016                              |       |
|Last Access Time:           |Thu Jan 01 08:00:00 CST 1970                              |       |
|Location:                   |hdfs://localhost:9000/user/hive/warehouse_hive121/src     |       |
|Table Type:                 |MANAGED                                                   |       |
|Table Parameters:           |                                                          |       |
|  transient_lastDdlTime     |1451898360                                                |       |
|                            |                                                          |       |
|# Storage Information       |                                                          |       |
|SerDe Library:              |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe        |       |
|InputFormat:                |org.apache.hadoop.mapred.TextInputFormat                  |       |
|OutputFormat:               |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat|       |
|Num Buckets:                |-1                                                        |       |
|Bucket Columns:             |[]                                                        |       |
|Sort Columns:               |[]                                                        |       |
|Storage Desc Parameters:    |                                                          |       |
|  serialization.format      |1                                                         |       |
+----------------------------+----------------------------------------------------------+-------+
```

## How was this patch tested?

A test case is added to `HiveDDLSuite` to check command output.

Author: Cheng Lian <lian@databricks.com>

Closes #12844 from liancheng/spark-14127-desc-table.
2016-05-04 16:44:09 +08:00
Wenchen Fan 6c12e801e8 [SPARK-15029] improve error message for Generate
## What changes were proposed in this pull request?

This PR improve the error message for `Generate` in 3 cases:

1. generator is nested in expressions, e.g. `SELECT explode(list) + 1 FROM tbl`
2. generator appears more than one time in SELECT, e.g. `SELECT explode(list), explode(list) FROM tbl`
3. generator appears in other operator which is not project, e.g. `SELECT * FROM tbl SORT BY explode(list)`

## How was this patch tested?

new tests in `AnalysisErrorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12810 from cloud-fan/bug.
2016-05-04 00:10:20 -07:00
Cheng Lian bc3760d405 [SPARK-14237][SQL] De-duplicate partition value appending logic in various buildReader() implementations
## What changes were proposed in this pull request?

Currently, various `FileFormat` data sources share approximately the same code for partition value appending. This PR tries to eliminate this duplication.

A new method `buildReaderWithPartitionValues()` is added to `FileFormat` with a default implementation that appends partition values to `InternalRow`s produced by the reader function returned by `buildReader()`.

Special data sources like Parquet, which implements partition value appending inside `buildReader()` because of the vectorized reader, and the Text data source, which doesn't support partitioning, override `buildReaderWithPartitionValues()` and simply delegate to `buildReader()`.

This PR brings two benefits:

1. Apparently, it de-duplicates partition value appending logic

2. Now the reader function returned by `buildReader()` is only required to produce `InternalRow`s rather than `UnsafeRow`s if the data source doesn't override `buildReaderWithPartitionValues()`.

   Because the safe-to-unsafe conversion is also performed while appending partition values. This makes 3rd-party data sources (e.g. spark-avro) easier to implement since they no longer need to access private APIs involving `UnsafeRow`.

## How was this patch tested?

Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #12866 from liancheng/spark-14237-simplify-partition-values-appending.
2016-05-04 14:16:57 +08:00
Reynold Xin 695f0e9195 [SPARK-15107][SQL] Allow varying # iterations by test case in Benchmark
## What changes were proposed in this pull request?
This patch changes our micro-benchmark util to allow setting different iteration numbers for different test cases. For some of our benchmarks, turning off whole-stage codegen can make the runtime 20X slower, making it very difficult to run a large number of times without substantially shortening the input cardinality.

With this change, I set the default num iterations to 2 for whole stage codegen off, and 5 for whole stage codegen on. I also updated some results.

## How was this patch tested?
N/A - this is a test util.

Author: Reynold Xin <rxin@databricks.com>

Closes #12884 from rxin/SPARK-15107.
2016-05-03 22:56:40 -07:00
Andrew Or 6ba17cd147 [SPARK-14414][SQL] Make DDL exceptions more consistent
## What changes were proposed in this pull request?

Just a bunch of small tweaks on DDL exception messages.

## How was this patch tested?

`DDLCommandSuite` et al.

Author: Andrew Or <andrew@databricks.com>

Closes #12853 from andrewor14/make-exceptions-consistent.
2016-05-03 18:07:53 -07:00